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

[01/48] incubator-geode git commit: New unit tests & minor bugfixes

Repository: incubator-geode
Updated Branches:
  refs/heads/feature/GEODE-217 0e62bc04b -> afed66577


New unit tests & minor bugfixes

1) new unit tests
2) detection of IDs w/o UUIDs in them in UDP messaging
3) removal of JoinResponse messages - view messages are considered a join-response so we don't need to send both
4) detection of new members admitted by a different coordinator during becomeCoordinator processes (problem found by Jianxia)
5) adjusting the join-timeout if member-timeout is large


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

Branch: refs/heads/feature/GEODE-217
Commit: a5906e5be3165b6afaa8424631a7a7f504c73f27
Parents: 6e32ffe
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Thu Dec 10 09:14:51 2015 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Thu Dec 10 09:16:12 2015 -0800

----------------------------------------------------------------------
 .../internal/InternalDistributedSystem.java     |  19 ++-
 .../distributed/internal/ReplyProcessor21.java  |   2 +-
 .../internal/membership/NetView.java            |  19 +++
 .../internal/membership/gms/GMSMember.java      |   8 ++
 .../internal/membership/gms/ServiceConfig.java  |  21 ++-
 .../membership/gms/membership/GMSJoinLeave.java | 126 ++++++++++++------
 .../gms/mgr/GMSMembershipManager.java           |  18 ++-
 .../internal/DistributionManagerDUnitTest.java  |  43 +++++-
 .../membership/MembershipJUnitTest.java         | 130 +++++++------------
 .../internal/membership/NetViewJUnitTest.java   |  66 +++++++++-
 .../gms/fd/GMSHealthMonitorJUnitTest.java       |  54 ++++----
 .../gms/membership/GMSJoinLeaveJUnitTest.java   |  87 ++++++++++---
 .../gms/mgr/GMSMembershipManagerJUnitTest.java  |  74 +++++++++++
 .../internal/DataSerializableJUnitTest.java     |  35 ++---
 14 files changed, 511 insertions(+), 191 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a5906e5b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystem.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystem.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystem.java
index 261b8a9..8f604b4 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystem.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystem.java
@@ -285,6 +285,19 @@ public final class InternalDistributedSystem
       }
     }
   }
+  
+  
+  /**
+   * creates a non-functional instance for testing
+   * @param nonDefault - non-default distributed system properties
+   */
+  public static InternalDistributedSystem newInstanceForTesting(DM dm, Properties nonDefault) {
+    InternalDistributedSystem sys = new InternalDistributedSystem(nonDefault);
+    sys.config = new RuntimeDistributionConfigImpl(sys);
+    sys.dm = dm;
+    sys.isConnected = true;
+    return sys;
+  }
 
   /**
    * Returns a connection to the distributed system that is suitable
@@ -529,10 +542,8 @@ public final class InternalDistributedSystem
       }
     }
 
-    if (this.isLoner) {
-      this.config = new RuntimeDistributionConfigImpl(this);
-    } else {
-      this.config = new RuntimeDistributionConfigImpl(this);
+    this.config = new RuntimeDistributionConfigImpl(this);
+    if (!this.isLoner) {
       this.attemptingToReconnect = (reconnectAttemptCounter > 0);
     }
     try {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a5906e5b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/ReplyProcessor21.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/ReplyProcessor21.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/ReplyProcessor21.java
index d12e78d..aa5f66c 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/ReplyProcessor21.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/ReplyProcessor21.java
@@ -757,7 +757,7 @@ public class ReplyProcessor21
       else {
         if (msecs > timeout) {
           if (!latch.await(timeout)) {
-            timeout(false, false);
+            timeout(isSevereAlertProcessingEnabled() && (severeAlertTimeout > 0), false);
             // after timeout alert, wait remaining time
             if (!latch.await(msecs-timeout)) {
               logger.info(LocalizedMessage.create(LocalizedStrings.ReplyProcessor21_WAIT_FOR_REPLIES_TIMING_OUT_AFTER_0_SEC, Long.valueOf(msecs / 1000)));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a5906e5b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/NetView.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/NetView.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/NetView.java
index a90a45d..6a68619 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/NetView.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/NetView.java
@@ -230,6 +230,10 @@ public class NetView implements DataSerializableFixedID {
     this.crashedMembers.addAll(mbr);
   }
 
+  public void addShutdownMembers(Set<InternalDistributedMember> mbr) {
+    this.shutdownMembers.addAll(mbr);
+  }
+
   public boolean remove(InternalDistributedMember mbr) {
     this.hashedMembers.remove(mbr);
     int idx = this.members.indexOf(mbr);
@@ -518,6 +522,21 @@ public class NetView implements DataSerializableFixedID {
     sb.append("]");
     return sb.toString();
   }
+  
+  /**
+   * Returns the ID from this view that is equal to the argument.
+   * If no such ID exists the argument is returned.
+   */
+  public synchronized InternalDistributedMember getCanonicalID(InternalDistributedMember id) {
+    if (hashedMembers.contains(id)) {
+      for (InternalDistributedMember m: this.members) {
+        if (id.equals(m)) {
+          return m;
+        }
+      }
+    }
+    return id;
+  }
 
   @Override
   public synchronized boolean equals(Object arg0) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a5906e5b/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 05b3aee..b1a4883 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
@@ -374,6 +374,14 @@ public class GMSMember implements NetMember, DataSerializableFixedID {
   public void setPort(int p) {
     this.udpPort = p;
   }
+  
+  /**
+   * checks to see if this address has UUID information needed
+   * to send messages via JGroups
+   */
+  public boolean hasUUID() {
+    return !(this.uuidLSBs == 0 && this.uuidMSBs == 0);
+  }
 
   @Override
   public Version[] getSerializationVersions() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a5906e5b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/ServiceConfig.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/ServiceConfig.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/ServiceConfig.java
index 1d67bbf..a412dfa 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/ServiceConfig.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/ServiceConfig.java
@@ -20,12 +20,17 @@ import java.net.InetAddress;
 
 import com.gemstone.gemfire.distributed.Locator;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.internal.membership.gms.membership.GMSJoinLeave;
 import com.gemstone.gemfire.internal.SocketCreator;
 import com.gemstone.gemfire.internal.admin.remote.RemoteTransportConfig;
 
 public class ServiceConfig {
+
+  /** stall time to wait for concurrent join/leave/remove requests to be received */
+  public static final long MEMBER_REQUEST_COLLECTION_INTERVAL = Long.getLong("gemfire.member-request-collection-interval", 500);
+
   /** various settings from Geode configuration */
-  private int joinTimeout;
+  private long joinTimeout;
   private int[] membershipPortRange;
   private int udpRecvBufferSize;
   private int udpSendBufferSize;
@@ -47,7 +52,7 @@ public class ServiceConfig {
   }
 
 
-  public int getJoinTimeout() {
+  public long getJoinTimeout() {
     return joinTimeout;
   }
 
@@ -128,11 +133,19 @@ public class ServiceConfig {
     this.dconfig = theConfig;
     this.transport = transport;
     
-    int defaultJoinTimeout = 24000;
+    long defaultJoinTimeout = 24000;
     if (theConfig.getLocators().length() > 0 && !Locator.hasLocators()) {
       defaultJoinTimeout = 60000;
     }
-    joinTimeout = Integer.getInteger("p2p.joinTimeout", defaultJoinTimeout).intValue();
+    
+    // we need to have enough time to figure out that the coordinator has crashed &
+    // find a new one
+    long minimumJoinTimeout = dconfig.getMemberTimeout() * 2 + MEMBER_REQUEST_COLLECTION_INTERVAL;
+    if (defaultJoinTimeout < minimumJoinTimeout) {
+      defaultJoinTimeout = minimumJoinTimeout;
+    };
+    
+    joinTimeout = Long.getLong("p2p.joinTimeout", defaultJoinTimeout).longValue();
     
     // if network partition detection is enabled, we must connect to the locators
     // more frequently in order to make sure we're not isolated from them

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a5906e5b/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 3e767ae..ccc9d8c 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
@@ -25,6 +25,7 @@ import static com.gemstone.gemfire.internal.DataSerializableFixedID.LEAVE_REQUES
 import static com.gemstone.gemfire.internal.DataSerializableFixedID.NETWORK_PARTITION_MESSAGE;
 import static com.gemstone.gemfire.internal.DataSerializableFixedID.REMOVE_MEMBER_REQUEST;
 import static com.gemstone.gemfire.internal.DataSerializableFixedID.VIEW_ACK_MESSAGE;
+import static com.gemstone.gemfire.distributed.internal.membership.gms.ServiceConfig.MEMBER_REQUEST_COLLECTION_INTERVAL;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
@@ -96,9 +97,6 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
   /** amount of time to sleep before trying to join after a failed attempt */
   private static final int JOIN_RETRY_SLEEP = Integer.getInteger("gemfire.join-retry-sleep", 1000);
 
-  /** stall time to wait for concurrent join/leave/remove requests to be received */
-  public static final long MEMBER_REQUEST_COLLECTION_INTERVAL = Long.getLong("gemfire.member-request-collection-interval", 500);
-
   /** time to wait for a leave request to be transmitted by jgroups */
   private static final long LEAVE_MESSAGE_SLEEP_TIME = Long.getLong("gemfire.leave-message-sleep-time", 1000);
 
@@ -664,7 +662,7 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
       }
       if (viewCreator == null || viewCreator.isShutdown()) {
         viewCreator = new ViewCreator("Geode Membership View Creator", Services.getThreadGroup());
-        viewCreator.setInitialView(newView, leaving, removals);
+        viewCreator.setInitialView(newView, newView.getNewMembers(), leaving, removals);
         viewCreator.setDaemon(true);
         viewCreator.start();
         startViewBroadcaster();
@@ -672,13 +670,6 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
     }
   }
 
-  private void sendJoinResponses(List<InternalDistributedMember> newMbrs, NetView newView) {
-    for (InternalDistributedMember mbr : newMbrs) {
-      JoinResponseMessage response = new JoinResponseMessage(mbr, newView);
-      services.getMessenger().send(response);
-    }
-  }
-
   private void sendRemoveMessages(List<InternalDistributedMember> removals, List<String> reasons, NetView newView) {
     Iterator<String> reason = reasons.iterator();
     for (InternalDistributedMember mbr : removals) {
@@ -747,12 +738,6 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
 
     // only wait for responses during preparation
     if (preparing) {
-      // send join responses after other members at least have
-      // a prepared view announcing the new member
-      if (!(isNetworkPartition && quorumRequired)) {
-        sendJoinResponses(newMembers, view);
-      }
-
       logger.debug("waiting for view responses");
 
       Set<InternalDistributedMember> failedToRespond = rp.waitForResponses();
@@ -1330,22 +1315,12 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
       stopCoordinatorServices();
       if (view != null) {
         if (view.size() > 1) {
-          if (this.isCoordinator) {
-            logger.debug("JoinLeave stopping coordination services");
-            NetView newView = new NetView(view, view.getViewId() + 1);
-            newView.remove(localAddress);
-            InstallViewMessage m = new InstallViewMessage(newView, services.getAuthenticator().getCredentials(this.localAddress));
-            m.setRecipients(newView.getMembers());
-            services.getMessenger().send(m);
-            waitForProcessing = true;
-          } else {
-            List<InternalDistributedMember> coords = view.getPreferredCoordinators(Collections.<InternalDistributedMember> emptySet(), localAddress, 5);
+          List<InternalDistributedMember> coords = view.getPreferredCoordinators(Collections.<InternalDistributedMember> emptySet(), localAddress, 5);
 
-            logger.debug("JoinLeave sending a leave request to {}", coords);
-            LeaveRequestMessage m = new LeaveRequestMessage(coords, this.localAddress, "this member is shutting down");
-            services.getMessenger().send(m);
-            waitForProcessing = true;
-          }
+          logger.debug("JoinLeave sending a leave request to {}", coords);
+          LeaveRequestMessage m = new LeaveRequestMessage(coords, this.localAddress, "this member is shutting down");
+          services.getMessenger().send(m);
+          waitForProcessing = true;
         } // view.size
       } // view != null
     }
@@ -1690,8 +1665,21 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
     boolean shutdown = false;
     volatile boolean waiting = false;
 
+    /**
+     * initial view to install.  guarded by synch on ViewCreator
+     */
     NetView initialView;
+    /**
+     * initial joining members.  guarded by synch on ViewCreator
+     */
+    List<InternalDistributedMember> initialJoins = Collections.<InternalDistributedMember>emptyList();
+    /**
+     * initial leaving members  guarded by synch on ViewCreator
+     */
     Set<InternalDistributedMember> initialLeaving;
+    /**
+     * initial crashed members.  guarded by synch on ViewCreator
+     */
     Set<InternalDistributedMember> initialRemovals;
 
     ViewCreator(String name, ThreadGroup tg) {
@@ -1723,22 +1711,72 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
      * @param leaving - members leaving in this view
      * @param removals - members crashed in this view
      */
-    void setInitialView(NetView newView, Set<InternalDistributedMember> leaving, Set<InternalDistributedMember> removals) {
+    synchronized void setInitialView(NetView newView, List<InternalDistributedMember> newMembers,
+        Set<InternalDistributedMember> leaving, Set<InternalDistributedMember> removals) {
       this.initialView = newView;
+      this.initialJoins = newMembers;
       this.initialLeaving = leaving;
       this.initialRemovals = removals;
     }
 
     private void sendInitialView() {
-      if (initialView != null) {
-        try {
-          prepareAndSendView(initialView, Collections.<InternalDistributedMember>emptyList(), initialLeaving,
-              initialRemovals);
-        } finally {
-          this.initialView = null;
-          this.initialLeaving = null;
-          this.initialRemovals = null;
+      if (initialView == null) {
+        return;
+      }
+      NetView v = preparedView;
+      if (v != null) {
+        processPreparedView(v);
+      }
+      try {
+        NetView iView;
+        List<InternalDistributedMember> iJoins;
+        Set<InternalDistributedMember> iLeaves;
+        Set<InternalDistributedMember> iRemoves;
+        synchronized(this) {
+          iView = initialView;
+          iJoins = initialJoins;
+          iLeaves = initialLeaving;
+          iRemoves = initialRemovals;
+        }
+        if (iView != null) {
+          prepareAndSendView(iView, iJoins, iLeaves, iRemoves);
+        }
+      } finally {
+        setInitialView(null, null, null, null);
+      }
+    }
+
+    /**
+     * During initial view processing a prepared view was discovered.
+     * This method will extract its new members and create a new
+     * initial view containing them.
+     * 
+     * @param v The prepared view
+     */
+    private void processPreparedView(NetView v) {
+      assert initialView != null;
+      if (currentView == null || currentView.getViewId() < v.getViewId()) {
+        // we have a prepared view that is newer than the current view
+        // form a new View ID
+        int viewId = Math.max(initialView.getViewId(),v.getViewId());
+        viewId += 1;
+        NetView newView = new NetView(initialView, viewId);
+
+        // add the new members from the prepared view to the new view,
+        // preserving their failure-detection ports
+        List<InternalDistributedMember> newMembers;
+        if (currentView != null) {
+          newMembers = v.getNewMembers(currentView);
+        } else {
+          newMembers = v.getMembers();
         }
+        for (InternalDistributedMember newMember: newMembers) {
+          newView.add(newMember);
+          newView.setFailureDetectionPort(newMember, v.getFailureDetectionPort(newMember));
+        }
+
+        // use the new view as the initial view
+        setInitialView(newView, newMembers, initialLeaving, initialRemovals);
       }
     }
 
@@ -2007,6 +2045,12 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
           logger.info("adding these crashed members from a conflicting view to the crash-set for the next view: {}\nconflicting view: {}", unresponsive,
               conflictingView);
           failures.addAll(conflictingView.getCrashedMembers());
+          // this member may have been kicked out of the conflicting view
+          if (failures.contains(localAddress)) {
+            forceDisconnect("I am no longer a member of the distributed system");
+            shutdown = true;
+            return;
+          }
           List<InternalDistributedMember> newMembers = conflictingView.getNewMembers();
           if (!newMembers.isEmpty()) {
             logger.info("adding these new members from a conflicting view to the new view: {}", newMembers);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a5906e5b/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 4e108be..bbe7ab3 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,6 +39,7 @@ 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;
@@ -72,6 +73,7 @@ import com.gemstone.gemfire.distributed.internal.membership.MembershipManager;
 import com.gemstone.gemfire.distributed.internal.membership.MembershipTestHook;
 import com.gemstone.gemfire.distributed.internal.membership.NetView;
 import com.gemstone.gemfire.distributed.internal.membership.QuorumChecker;
+import com.gemstone.gemfire.distributed.internal.membership.gms.GMSMember;
 import com.gemstone.gemfire.distributed.internal.membership.gms.GMSUtil;
 import com.gemstone.gemfire.distributed.internal.membership.gms.Services;
 import com.gemstone.gemfire.distributed.internal.membership.gms.SuspectMember;
@@ -1429,7 +1431,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
     }
   }
   
-  
   protected boolean isJoining() {
     return this.isJoining;
   }
@@ -1971,6 +1972,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
     boolean sendViaMessenger = isForceUDPCommunications(); // enable when bug #46438 is fixed: || msg.sendViaUDP();
 
     if (useMcast || tcpDisabled || sendViaMessenger) {
+      checkAddressesForUUIDs(destinations);
       result = services.getMessenger().send(msg);
     }
     else {
@@ -1991,6 +1993,20 @@ public class GMSMembershipManager implements MembershipManager, Manager
     forceUseUDPMessaging.set(null);
   }
   
+  void checkAddressesForUUIDs(InternalDistributedMember[] addresses) {
+    for (int i=0; i<addresses.length; i++) {
+      GMSMember id = (GMSMember)addresses[i].getNetMember();
+      if (!id.hasUUID()) {
+        latestViewLock.readLock().lock();
+        try {
+          addresses[i] = latestView.getCanonicalID(addresses[i]);
+        } finally {
+          latestViewLock.readLock().unlock();
+        }
+      }
+    }
+  }
+  
   private boolean isForceUDPCommunications() {
     Boolean forced = forceUseUDPMessaging.get();
     return forced == Boolean.TRUE;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a5906e5b/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 51771cb..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
@@ -19,9 +19,8 @@ package com.gemstone.gemfire.distributed.internal;
 import java.net.InetAddress;
 import java.util.Properties;
 
-import junit.framework.Assert;
-
 import org.apache.logging.log4j.Logger;
+import org.junit.Assert;
 
 import com.gemstone.gemfire.LogWriter;
 import com.gemstone.gemfire.admin.AdminDistributedSystem;
@@ -43,7 +42,9 @@ 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;
+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;
@@ -538,4 +539,42 @@ public class DistributionManagerDUnitTest extends DistributedTestCase {
     props.setProperty(DistributionConfig.BIND_ADDRESS_NAME, InetAddress.getLocalHost().getCanonicalHostName());
     getSystem().disconnect();
   }
+  
+  /**
+   * install a new view and show that waitForViewInstallation works as expected
+   */
+  public void testWaitForViewInstallation() {
+    getSystem(new Properties());
+    
+    MembershipManager mgr = system.getDM().getMembershipManager(); 
+
+    final NetView v = mgr.getView();
+    
+    final boolean[] passed = new boolean[1];
+    Thread t = new Thread("wait for view installation") {
+      public void run() {
+        try {
+          ((DistributionManager)system.getDM()).waitForViewInstallation(v.getViewId()+1);
+          synchronized(passed) {
+            passed[0] = true;
+          }
+        } catch (InterruptedException e) {
+          // failed
+        }
+      }
+    };
+    t.setDaemon(true);
+    t.start();
+    
+    pause(2000);
+
+    NetView newView = new NetView(v, v.getViewId()+1);
+    ((Manager)mgr).installView(newView);
+
+    pause(2000);
+    
+    synchronized(passed) {
+      Assert.assertTrue(passed[0]);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a5906e5b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/MembershipJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/MembershipJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/MembershipJUnitTest.java
index bee2367..7a4971f 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/MembershipJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/MembershipJUnitTest.java
@@ -16,28 +16,17 @@
  */
 package com.gemstone.gemfire.distributed.internal.membership;
 
-import static org.mockito.Mockito.isA;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+import static org.mockito.Matchers.isA;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 import java.io.File;
 import java.net.InetAddress;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
 import java.util.Properties;
-import java.util.Set;
-
-import junit.framework.TestCase;
 
 import org.apache.logging.log4j.Level;
 import org.junit.AfterClass;
@@ -47,12 +36,10 @@ import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.GemFireConfigException;
 import com.gemstone.gemfire.distributed.Locator;
-import com.gemstone.gemfire.distributed.internal.DM;
 import com.gemstone.gemfire.distributed.internal.DMStats;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.DistributionConfigImpl;
 import com.gemstone.gemfire.distributed.internal.DistributionManager;
-import com.gemstone.gemfire.distributed.internal.DistributionMessage;
 import com.gemstone.gemfire.distributed.internal.InternalLocator;
 import com.gemstone.gemfire.distributed.internal.SerialAckedMessage;
 import com.gemstone.gemfire.distributed.internal.membership.gms.GMSUtil;
@@ -90,70 +77,7 @@ public class MembershipJUnitTest {
 //    LogService.setBaseLogLevel(baseLogLevel);
   }
   
-  /**
-   * Test that failed weight calculations are correctly performed.  See bug #47342
-   * @throws Exception
-   */
-  public void testFailedWeight() throws Exception {
-    // in #47342 a new view was created that contained a member that was joining but
-    // was no longer reachable.  The member was included in the failed-weight and not
-    // in the previous view-weight, causing a spurious network partition to be declared
-    InternalDistributedMember members[] = new InternalDistributedMember[] {
-        new InternalDistributedMember("localhost", 1), new InternalDistributedMember("localhost", 2), new InternalDistributedMember("localhost", 3),
-        new InternalDistributedMember("localhost", 4), new InternalDistributedMember("localhost", 5), new InternalDistributedMember("localhost", 6)};
-    int i = 0;
-    // weight 3
-    members[i].setVmKind(DistributionManager.LOCATOR_DM_TYPE);
-    members[i++].getNetMember().setPreferredForCoordinator(true);
-    // weight 3
-    members[i].setVmKind(DistributionManager.LOCATOR_DM_TYPE);
-    members[i++].getNetMember().setPreferredForCoordinator(true);
-    // weight 15 (cache+leader)
-    members[i].setVmKind(DistributionManager.NORMAL_DM_TYPE);
-    members[i++].getNetMember().setPreferredForCoordinator(false);
-    // weight 0
-    members[i].setVmKind(DistributionManager.ADMIN_ONLY_DM_TYPE);
-    members[i++].getNetMember().setPreferredForCoordinator(false);
-    // weight 0
-    members[i].setVmKind(DistributionManager.ADMIN_ONLY_DM_TYPE);
-    members[i++].getNetMember().setPreferredForCoordinator(false);
-    // weight 10
-    members[i].setVmKind(DistributionManager.NORMAL_DM_TYPE);
-    members[i++].getNetMember().setPreferredForCoordinator(false);
-    
-    List<InternalDistributedMember> vmbrs = new ArrayList<>(members.length);
-    for (i=0; i<members.length; i++) {
-      vmbrs.add(members[i]);
-    }
-    Set<InternalDistributedMember> empty = Collections.emptySet();
-    NetView lastView = new NetView(members[0], 4, vmbrs, empty, empty);
-    InternalDistributedMember leader = members[2];
-    assertTrue(!leader.getNetMember().preferredForCoordinator());
-    
-    InternalDistributedMember joiningMember = new InternalDistributedMember("localhost", 7);
-    joiningMember.setVmKind(DistributionManager.NORMAL_DM_TYPE);
-    joiningMember.getNetMember().setPreferredForCoordinator(false);
-    
-    // have the joining member and another cache process (weight 10) in the failed members
-    // collection and check to make sure that the joining member is not included in failed
-    // weight calcs.
-    Set<InternalDistributedMember> failedMembers = new HashSet<>(3);
-    failedMembers.add(joiningMember);
-    failedMembers.add(members[members.length-1]); // cache
-    failedMembers.add(members[members.length-2]); // admin
-    List<InternalDistributedMember> newMbrs = new ArrayList<InternalDistributedMember>(lastView.getMembers());
-    newMbrs.removeAll(failedMembers);
-    NetView newView = new NetView(members[0], 5, newMbrs, empty, failedMembers);
-    
-    int failedWeight = newView.getCrashedMemberWeight(lastView);
-//    System.out.println("last view = " + lastView);
-//    System.out.println("failed mbrs = " + failedMembers);
-//    System.out.println("failed weight = " + failedWeight);
-    assertEquals("failure weight calculation is incorrect", 10, failedWeight);
-    Set<InternalDistributedMember> actual = newView.getActualCrashedMembers(lastView);
-    assertTrue(!actual.contains(members[members.length-2]));
-  }
-  
+ 
 //  @Test
 //  public void testRepeat() throws Exception {
 //    for (int i=0; i<50; i++) {
@@ -292,6 +216,48 @@ public class MembershipJUnitTest {
       }
     }
   }
+  
+  @Test
+  public void testJoinTimeoutSetting() throws Exception {
+    long timeout = 30000;
+    Properties nonDefault = new Properties();
+    nonDefault.put(DistributionConfig.MEMBER_TIMEOUT_NAME, ""+timeout);
+    DistributionConfigImpl config = new DistributionConfigImpl(nonDefault);
+    RemoteTransportConfig transport = new RemoteTransportConfig(config,
+        DistributionManager.NORMAL_DM_TYPE);
+    ServiceConfig sc = new ServiceConfig(transport, config);
+    assertEquals(2 * timeout + ServiceConfig.MEMBER_REQUEST_COLLECTION_INTERVAL, sc.getJoinTimeout());
+    
+    nonDefault.clear();
+    config = new DistributionConfigImpl(nonDefault);
+    transport = new RemoteTransportConfig(config,
+        DistributionManager.NORMAL_DM_TYPE);
+    sc = new ServiceConfig(transport, config);
+    assertEquals(24000, sc.getJoinTimeout());
+    
+
+    nonDefault.clear();
+    nonDefault.put(DistributionConfig.LOCATORS_NAME, SocketCreator.getLocalHost().getHostAddress()+"["+12345+"]");
+    config = new DistributionConfigImpl(nonDefault);
+    transport = new RemoteTransportConfig(config,
+        DistributionManager.NORMAL_DM_TYPE);
+    sc = new ServiceConfig(transport, config);
+    assertEquals(60000, sc.getJoinTimeout());
+    
+
+    timeout = 2000;
+    System.setProperty("p2p.joinTimeout", ""+timeout);
+    try {
+      config = new DistributionConfigImpl(nonDefault);
+      transport = new RemoteTransportConfig(config,
+          DistributionManager.NORMAL_DM_TYPE);
+      sc = new ServiceConfig(transport, config);
+      assertEquals(timeout, sc.getJoinTimeout());
+    } finally {
+      System.getProperties().remove("p2p.joinTimeout");
+    }
+    
+  }
 
   @Test
   public void testMulticastDiscoveryNotAllowed() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a5906e5b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/NetViewJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/NetViewJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/NetViewJUnitTest.java
index 603c7bf..9e39d0f 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/NetViewJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/NetViewJUnitTest.java
@@ -191,4 +191,68 @@ public class NetViewJUnitTest {
     assertEquals(100, view.getNewMembers(copy).size());
   }
   
-}
+  /**
+   * Test that failed weight calculations are correctly performed.  See bug #47342
+   * @throws Exception
+   */
+  @Test
+  public void testFailedWeight() throws Exception {
+    // in #47342 a new view was created that contained a member that was joining but
+    // was no longer reachable.  The member was included in the failed-weight and not
+    // in the previous view-weight, causing a spurious network partition to be declared
+    InternalDistributedMember members[] = new InternalDistributedMember[] {
+        new InternalDistributedMember("localhost", 1), new InternalDistributedMember("localhost", 2), new InternalDistributedMember("localhost", 3),
+        new InternalDistributedMember("localhost", 4), new InternalDistributedMember("localhost", 5), new InternalDistributedMember("localhost", 6)};
+    int i = 0;
+    // weight 3
+    members[i].setVmKind(DistributionManager.LOCATOR_DM_TYPE);
+    members[i++].getNetMember().setPreferredForCoordinator(true);
+    // weight 3
+    members[i].setVmKind(DistributionManager.LOCATOR_DM_TYPE);
+    members[i++].getNetMember().setPreferredForCoordinator(true);
+    // weight 15 (cache+leader)
+    members[i].setVmKind(DistributionManager.NORMAL_DM_TYPE);
+    members[i++].getNetMember().setPreferredForCoordinator(false);
+    // weight 0
+    members[i].setVmKind(DistributionManager.ADMIN_ONLY_DM_TYPE);
+    members[i++].getNetMember().setPreferredForCoordinator(false);
+    // weight 0
+    members[i].setVmKind(DistributionManager.ADMIN_ONLY_DM_TYPE);
+    members[i++].getNetMember().setPreferredForCoordinator(false);
+    // weight 10
+    members[i].setVmKind(DistributionManager.NORMAL_DM_TYPE);
+    members[i++].getNetMember().setPreferredForCoordinator(false);
+    
+    List<InternalDistributedMember> vmbrs = new ArrayList<>(members.length);
+    for (i=0; i<members.length; i++) {
+      vmbrs.add(members[i]);
+    }
+    Set<InternalDistributedMember> empty = Collections.emptySet();
+    NetView lastView = new NetView(members[0], 4, vmbrs, empty, empty);
+    InternalDistributedMember leader = members[2];
+    assertTrue(!leader.getNetMember().preferredForCoordinator());
+    
+    InternalDistributedMember joiningMember = new InternalDistributedMember("localhost", 7);
+    joiningMember.setVmKind(DistributionManager.NORMAL_DM_TYPE);
+    joiningMember.getNetMember().setPreferredForCoordinator(false);
+    
+    // have the joining member and another cache process (weight 10) in the failed members
+    // collection and check to make sure that the joining member is not included in failed
+    // weight calcs.
+    Set<InternalDistributedMember> failedMembers = new HashSet<>(3);
+    failedMembers.add(joiningMember);
+    failedMembers.add(members[members.length-1]); // cache
+    failedMembers.add(members[members.length-2]); // admin
+    List<InternalDistributedMember> newMbrs = new ArrayList<InternalDistributedMember>(lastView.getMembers());
+    newMbrs.removeAll(failedMembers);
+    NetView newView = new NetView(members[0], 5, newMbrs, empty, failedMembers);
+    
+    int failedWeight = newView.getCrashedMemberWeight(lastView);
+//    System.out.println("last view = " + lastView);
+//    System.out.println("failed mbrs = " + failedMembers);
+//    System.out.println("failed weight = " + failedWeight);
+    assertEquals("failure weight calculation is incorrect", 10, failedWeight);
+    Set<InternalDistributedMember> actual = newView.getActualCrashedMembers(lastView);
+    assertTrue(!actual.contains(members[members.length-2]));
+  }
+ }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a5906e5b/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 585ff17..c4ac3a6 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,11 +20,7 @@ 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.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 static org.mockito.Mockito.*;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -46,8 +42,6 @@ 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;
@@ -292,7 +286,6 @@ public class GMSHealthMonitorJUnitTest {
 
     NetView v = new NetView(mockMembers.get(0), 2, mockMembers, new HashSet<InternalDistributedMember>(), new HashSet<InternalDistributedMember>());
 
-    MethodExecuted messageSent = new MethodExecuted();
     // 3rd is current member
     when(messenger.getMemberID()).thenReturn(mockMembers.get(3));
 
@@ -300,15 +293,13 @@ public class GMSHealthMonitorJUnitTest {
 
     gmsHealthMonitor.suspect(mockMembers.get(1), "Not responding");
 
-    when(messenger.send(isA(SuspectMembersMessage.class))).thenAnswer(messageSent);
-
     try {
       // suspect thread timeout is 200 ms
       Thread.sleep(100l);
     } catch (InterruptedException e) {
     }
 
-    assertTrue("SuspectMembersMessage shouldn't have sent", !messageSent.isMethodExecuted());
+    verify(messenger, atLeastOnce()).send(isA(SuspectMembersMessage.class));
   }
 
   /***
@@ -544,6 +535,33 @@ public class GMSHealthMonitorJUnitTest {
     int byteReply = dis.read();
     Assert.assertEquals(expectedResult, byteReply);
   }
+  
+  @Test
+  public void testBeSickAndPlayDead() throws Exception {
+    NetView v = new NetView(mockMembers.get(0), 2, mockMembers, new HashSet<InternalDistributedMember>(), new HashSet<InternalDistributedMember>());
+    gmsHealthMonitor.installView(v);
+    gmsHealthMonitor.beSick();
+    
+    // a sick member will not respond to a heartbeat request
+    HeartbeatRequestMessage req = new HeartbeatRequestMessage(mockMembers.get(0), 10);
+    req.setSender(mockMembers.get(0));
+    gmsHealthMonitor.processMessage(req);
+    verify(messenger, never()).send(isA(HeartbeatMessage.class));
+    
+    // a sick member will not record a heartbeat from another member
+    HeartbeatMessage hb = new HeartbeatMessage(-1);
+    hb.setSender(mockMembers.get(0));
+    gmsHealthMonitor.processMessage(hb);
+    assertTrue(gmsHealthMonitor.memberTimeStamps.get(hb.getSender()) == null);
+    
+    // a sick member will not take action on a Suspect message from another member
+    SuspectMembersMessage smm = mock(SuspectMembersMessage.class);
+    Error err = new AssertionError("expected suspect message to be ignored");
+    when(smm.getMembers()).thenThrow(err);
+    when(smm.getSender()).thenThrow(err);
+    when(smm.getDSFID()).thenCallRealMethod();
+    gmsHealthMonitor.processMessage(smm);
+  }
 
   private GMSMember createGMSMember(short version, int viewId, long msb, long lsb) {
     GMSMember gmsMember = new GMSMember();
@@ -560,18 +578,4 @@ public class GMSHealthMonitorJUnitTest {
     return baos.toByteArray();
   }
 
-
-  private class MethodExecuted implements Answer {
-    private boolean methodExecuted = false;
-
-    public boolean isMethodExecuted() {
-      return methodExecuted;
-    }
-
-    @Override
-    public Object answer(InvocationOnMock invocation) throws Throwable {
-      methodExecuted = true;
-      return null;
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a5906e5b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeaveJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeaveJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeaveJUnitTest.java
index 9895f68..01c0695 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeaveJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeaveJUnitTest.java
@@ -16,6 +16,7 @@
  */
 package com.gemstone.gemfire.distributed.internal.membership.gms.membership;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Matchers.any;
@@ -43,12 +44,9 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.mockito.ArgumentCaptor;
 import org.mockito.internal.verification.Times;
-import org.mockito.internal.verification.api.VerificationData;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 import org.mockito.verification.Timeout;
-import org.mockito.verification.VerificationMode;
-import org.mockito.verification.VerificationWithTimeout;
 
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
@@ -63,6 +61,7 @@ import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.Manag
 import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.Messenger;
 import com.gemstone.gemfire.distributed.internal.membership.gms.locator.FindCoordinatorResponse;
 import com.gemstone.gemfire.distributed.internal.membership.gms.membership.GMSJoinLeave.SearchState;
+import com.gemstone.gemfire.distributed.internal.membership.gms.membership.GMSJoinLeave.ViewCreator;
 import com.gemstone.gemfire.distributed.internal.membership.gms.membership.GMSJoinLeave.ViewReplyProcessor;
 import com.gemstone.gemfire.distributed.internal.membership.gms.messages.InstallViewMessage;
 import com.gemstone.gemfire.distributed.internal.membership.gms.messages.JoinRequestMessage;
@@ -72,8 +71,6 @@ import com.gemstone.gemfire.distributed.internal.membership.gms.messages.Network
 import com.gemstone.gemfire.distributed.internal.membership.gms.messages.RemoveMemberMessage;
 import com.gemstone.gemfire.distributed.internal.membership.gms.messages.ViewAckMessage;
 import com.gemstone.gemfire.internal.Version;
-import com.gemstone.gemfire.internal.admin.remote.AddStatListenerResponse;
-import com.gemstone.gemfire.internal.admin.remote.StatListenerMessage;
 import com.gemstone.gemfire.security.AuthenticationFailedException;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
@@ -297,7 +294,7 @@ public class GMSJoinLeaveJUnitTest {
     MethodExecuted removeMessageSent = new MethodExecuted();
     when(messenger.send(any(RemoveMemberMessage.class))).thenAnswer(removeMessageSent);
     gmsJoinLeave.remove(mockMembers[0], "removing for test");
-    Thread.sleep(GMSJoinLeave.MEMBER_REQUEST_COLLECTION_INTERVAL*2);
+    Thread.sleep(ServiceConfig.MEMBER_REQUEST_COLLECTION_INTERVAL*2);
     assertTrue(removeMessageSent.methodExecuted);
   }
   
@@ -655,9 +652,6 @@ public class GMSJoinLeaveJUnitTest {
   public void testNetworkPartitionMessageReceived() throws Exception {
     initMocks();
     gmsJoinLeave.becomeCoordinatorForTest();
-    List<InternalDistributedMember> members = Arrays.asList(mockMembers);
-    Set<InternalDistributedMember> empty = Collections.<InternalDistributedMember>emptySet();
-    NetView v = new NetView(mockMembers[0], 2, members, empty, empty);
     NetworkPartitionMessage message = new NetworkPartitionMessage();
     gmsJoinLeave.processMessage(message);
     verify(manager).forceDisconnect(any(String.class));
@@ -805,7 +799,7 @@ public class GMSJoinLeaveJUnitTest {
     gmsJoinLeave.memberShutdown(mockMembers[2], "Shutdown");
     
     //Install a view that still contains one of the left members (as if something like a new member, triggered a new view before coordinator leaves)
-    NetView netView = new NetView(mockMembers[0], 3/*new view id*/, createMemberList(mockMembers[0], gmsJoinLeaveMemberId, mockMembers[1], mockMembers[3]), new HashSet(), new HashSet());
+    NetView netView = new NetView(mockMembers[0], 3/*new view id*/, createMemberList(mockMembers[0], gmsJoinLeaveMemberId, mockMembers[1], mockMembers[3]), new HashSet<InternalDistributedMember>(), new HashSet<InternalDistributedMember>());
     InstallViewMessage installViewMessage = new InstallViewMessage(netView, credentials, false);
     gmsJoinLeave.processMessage(installViewMessage);
     
@@ -866,11 +860,11 @@ public class GMSJoinLeaveJUnitTest {
     int viewRequests = gmsJoinLeave.getViewRequests().size();
     
     assertTrue( "There should be 1 viewRequest but found " + viewRequests, viewRequests == 1);
-    Thread.sleep(2 * GMSJoinLeave.MEMBER_REQUEST_COLLECTION_INTERVAL);
+    Thread.sleep(2 * ServiceConfig.MEMBER_REQUEST_COLLECTION_INTERVAL);
     
     viewRequests = gmsJoinLeave.getViewRequests().size();
-    assertTrue( "There should be 0 viewRequest but found " + viewRequests, viewRequests == 0);
-    }finally {
+    assertEquals( "Found view requests: " + gmsJoinLeave.getViewRequests(), 0, viewRequests);
+    } finally {
       System.getProperties().remove(GMSJoinLeave.BYPASS_DISCOVERY_PROPERTY);
     }
   }
@@ -946,7 +940,7 @@ public class GMSJoinLeaveJUnitTest {
     InternalDistributedMember ids = new InternalDistributedMember("localhost", 97898);
     ids.getNetMember().setPreferredForCoordinator(true);
     gmsJoinLeave.processMessage(reqMsg);
-    ArgumentCaptor<JoinResponseMessage> ac = new ArgumentCaptor<>();
+    ArgumentCaptor<JoinResponseMessage> ac = ArgumentCaptor.forClass(JoinResponseMessage.class);
     verify(messenger).send(ac.capture());
     
     assertTrue("Should have asked for becoming a coordinator", ac.getValue().getBecomeCoordinator());
@@ -964,12 +958,75 @@ public class GMSJoinLeaveJUnitTest {
         msg.setSender(gmsJoinLeaveMemberId);
         gmsJoinLeave.processMessage(msg);
       }
-      Timeout to = new Timeout(2 * GMSJoinLeave.MEMBER_REQUEST_COLLECTION_INTERVAL, new Times(1));
+      Timeout to = new Timeout(2 * ServiceConfig.MEMBER_REQUEST_COLLECTION_INTERVAL, new Times(1));
       verify(messenger, to).send( isA(NetworkPartitionMessage.class));
                  
     }finally {
       System.getProperties().remove(GMSJoinLeave.BYPASS_DISCOVERY_PROPERTY);
     }    
   }
+  
+  @Test
+  public void testViewIgnoredAfterShutdown() throws Exception {
+    try {
+      initMocks(true);
+      System.setProperty(GMSJoinLeave.BYPASS_DISCOVERY_PROPERTY, "true");
+      gmsJoinLeave.join();
+      installView(1, gmsJoinLeaveMemberId, createMemberList(mockMembers[0], mockMembers[1], mockMembers[2], gmsJoinLeaveMemberId, mockMembers[3]));
+      gmsJoinLeave.stop();
+      for(int i = 1; i < 4; i++) {
+        RemoveMemberMessage msg = new RemoveMemberMessage(gmsJoinLeaveMemberId, mockMembers[i], "crashed");
+        msg.setSender(gmsJoinLeaveMemberId);
+        gmsJoinLeave.processMessage(msg);
+      }
+      Timeout to = new Timeout(2 * ServiceConfig.MEMBER_REQUEST_COLLECTION_INTERVAL, never());
+      verify(messenger, to).send( isA(NetworkPartitionMessage.class));
+                 
+    }finally {
+      System.getProperties().remove(GMSJoinLeave.BYPASS_DISCOVERY_PROPERTY);
+    }    
+  }
+  
+  @Test
+  public void testPreparedViewFoundDuringBecomeCoordinator() throws Exception {
+    initMocks(false);
+    prepareAndInstallView(mockMembers[0], createMemberList(mockMembers[0], gmsJoinLeaveMemberId));
+    
+    // a new member is joining
+    NetView preparedView = new NetView(gmsJoinLeave.getView(), gmsJoinLeave.getView().getViewId()+5);
+    mockMembers[1].setVmViewId(preparedView.getViewId());
+    preparedView.add(mockMembers[1]);
+    
+    InstallViewMessage msg = new InstallViewMessage(preparedView, null, true);
+    gmsJoinLeave.processMessage(msg);
+    
+    gmsJoinLeave.becomeCoordinatorForTest();
+
+    Thread.sleep(2000);
+    ViewCreator vc = gmsJoinLeave.getViewCreator();
+    
+    ViewAckMessage vack = new ViewAckMessage(gmsJoinLeaveMemberId, gmsJoinLeave.getPreparedView().getViewId(), true);
+    vack.setSender(mockMembers[0]);
+    gmsJoinLeave.processMessage(vack);
+    vack = new ViewAckMessage(gmsJoinLeaveMemberId, gmsJoinLeave.getPreparedView().getViewId(), true);
+    vack.setSender(mockMembers[1]);
+    gmsJoinLeave.processMessage(vack);
+    vack = new ViewAckMessage(gmsJoinLeaveMemberId, gmsJoinLeave.getPreparedView().getViewId(), true);
+    vack.setSender(gmsJoinLeaveMemberId);
+    gmsJoinLeave.processMessage(vack);
+    
+    int tries = 0;
+    while (!vc.waiting) {
+      if (tries > 30) {
+        Assert.fail("view creator never finished");
+      }
+      tries++;
+      Thread.sleep(1000);
+    }
+    NetView newView = gmsJoinLeave.getView();
+    System.out.println("new view is " + newView);
+    assertTrue(newView.contains(mockMembers[1]));
+    assertTrue(newView.getViewId() > preparedView.getViewId());
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a5906e5b/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 2b59ca5..e133625 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
@@ -25,9 +25,11 @@ import java.util.Date;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Properties;
+import java.util.Random;
 import java.util.Set;
 import java.util.Timer;
 
+import org.jgroups.util.UUID;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -35,16 +37,22 @@ 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;
 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;
 import com.gemstone.gemfire.distributed.internal.direct.DirectChannel;
 import com.gemstone.gemfire.distributed.internal.membership.DistributedMembershipListener;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.distributed.internal.membership.NetView;
+import com.gemstone.gemfire.distributed.internal.membership.gms.GMSMember;
 import com.gemstone.gemfire.distributed.internal.membership.gms.ServiceConfig;
 import com.gemstone.gemfire.distributed.internal.membership.gms.Services;
 import com.gemstone.gemfire.distributed.internal.membership.gms.Services.Stopper;
@@ -72,6 +80,7 @@ public class GMSMembershipManagerJUnitTest {
   private Services services;
   private ServiceConfig mockConfig;
   private DistributionConfig distConfig;
+  private Properties distProperties;
   private Authenticator authenticator;
   private HealthMonitor healthMonitor;
   private InternalDistributedMember myMemberId;
@@ -88,6 +97,8 @@ public class GMSMembershipManagerJUnitTest {
   @Before
   public void initMocks() throws Exception {
     Properties nonDefault = new Properties();
+    nonDefault.put(DistributionConfig.ACK_WAIT_THRESHOLD_NAME, "1");
+    nonDefault.put(DistributionConfig.ACK_SEVERE_ALERT_THRESHOLD_NAME, "10");
     nonDefault.put(DistributionConfig.DISABLE_TCP_NAME, "true");
     nonDefault.put(DistributionConfig.MCAST_PORT_NAME, "0");
     nonDefault.put(DistributionConfig.MCAST_TTL_NAME, "0");
@@ -96,6 +107,7 @@ public class GMSMembershipManagerJUnitTest {
     nonDefault.put(DistributionConfig.MEMBER_TIMEOUT_NAME, "2000");
     nonDefault.put(DistributionConfig.LOCATORS_NAME, "localhost[10344]");
     distConfig = new DistributionConfigImpl(nonDefault);
+    distProperties = nonDefault;
     RemoteTransportConfig tconfig = new RemoteTransportConfig(distConfig,
         DistributionManager.NORMAL_DM_TYPE);
     
@@ -129,9 +141,13 @@ public class GMSMembershipManagerJUnitTest {
     Timer t = new Timer(true);
     when(services.getTimer()).thenReturn(t);
     
+    Random r = new Random();
     mockMembers = new InternalDistributedMember[5];
     for (int i = 0; i < mockMembers.length; i++) {
       mockMembers[i] = new InternalDistributedMember("localhost", 8888 + i);
+      GMSMember m = (GMSMember)mockMembers[i].getNetMember();
+      UUID uuid = new UUID(r.nextLong(), r.nextLong());
+      m.setUUID(uuid);
     }
     members = new ArrayList<>(Arrays.asList(mockMembers));
 
@@ -366,5 +382,63 @@ public class GMSMembershipManagerJUnitTest {
     verify(dc).send(isA(GMSMembershipManager.class), isA(mockMembers.getClass()), isA(DistributionMessage.class), anyInt(), anyInt());
   }
   
+  /**
+   * This test ensures that the membership manager can accept an ID that
+   * does not have a UUID and replace it with one that does have a UUID
+   * from the current membership view.
+   */
+  @Test
+  public void testAddressesWithoutUUIDs() throws Exception {
+    manager.start();
+    manager.started();
+    manager.isJoining = true;
+
+    List<InternalDistributedMember> viewmembers = Arrays.asList(new InternalDistributedMember[] {mockMembers[0], mockMembers[1], myMemberId});
+    manager.installView(new NetView(myMemberId, 2, viewmembers, emptyMembersSet, emptyMembersSet));
+    
+    InternalDistributedMember[] destinations = new InternalDistributedMember[viewmembers.size()];
+    for (int i=0; i<destinations.length; i++) {
+      InternalDistributedMember id = viewmembers.get(i);
+      destinations[i] = new InternalDistributedMember(id.getHost(), id.getPort());
+    }
+    manager.checkAddressesForUUIDs(destinations);
+    // each destination w/o a UUID should have been replaced with the corresponding
+    // ID from the membership view
+    for (int i=0; i<destinations.length; i++) {
+      assertTrue(viewmembers.get(i) == destinations[i]);
+    }
+  }
+  
+  @Test
+  public void testReplyProcessorInitiatesSuspicion() throws Exception {
+    DM dm = mock(DM.class);
+    DMStats stats = mock(DMStats.class);
+    
+    InternalDistributedSystem system = InternalDistributedSystem.newInstanceForTesting(dm, distProperties);
+
+    when(dm.getStats()).thenReturn(stats);
+    when(dm.getSystem()).thenReturn(system);
+    when(dm.getCancelCriterion()).thenReturn(stopper);
+    when(dm.getMembershipManager()).thenReturn(manager);
+    when(dm.getViewMembers()).thenReturn(members);
+    when(dm.getDistributionManagerIds()).thenReturn(new HashSet(members));
+    when(dm.addMembershipListenerAndGetDistributionManagerIds(any(MembershipListener.class))).thenReturn(new HashSet(members));
+    
+    manager.start();
+    manager.started();
+    manager.isJoining = true;
+
+    List<InternalDistributedMember> viewmembers = Arrays.asList(new InternalDistributedMember[] {mockMembers[0], mockMembers[1], myMemberId});
+    manager.installView(new NetView(myMemberId, 2, viewmembers, emptyMembersSet, emptyMembersSet));
+
+    List<InternalDistributedMember> mbrs = new ArrayList<>(1);
+    mbrs.add(mockMembers[0]);
+    ReplyProcessor21 rp = new ReplyProcessor21(dm, mbrs);
+    rp.enableSevereAlertProcessing();
+    boolean result = rp.waitForReplies(2000);
+    assertFalse(result);  // the wait should have timed out
+    verify(healthMonitor, atLeastOnce()).checkIfAvailable(isA(InternalDistributedMember.class), isA(String.class), isA(Boolean.class));
+  }
+  
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a5906e5b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/DataSerializableJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/DataSerializableJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/DataSerializableJUnitTest.java
index 91a3411..5f69393 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/DataSerializableJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/DataSerializableJUnitTest.java
@@ -3545,21 +3545,26 @@ public class DataSerializableJUnitTest extends TestCase
   }
 
   public void testObjectEnum() throws Exception {
-    System.getProperties().setProperty("DataSerializer.DEBUG", "true");
-    DAY_OF_WEEK e = DAY_OF_WEEK.SUN;
-    MONTH m = MONTH.FEB;
-    DataOutputStream out = getDataOutput();
-    DataSerializer.writeObject(e, out);
-    DataSerializer.writeObject(m, out);
-    out.flush();
-
-    DataInput in = getDataInput();
-    DAY_OF_WEEK e2 = (DAY_OF_WEEK)DataSerializer.readObject(in);
-    MONTH m2 = (MONTH)DataSerializer.readObject(in);
-    assertEquals(e, e2);
-    assertEquals(m, m2);
-    // Make sure there's nothing left in the stream
-    assertEquals(0, in.skipBytes(1));
+    final String propName = "DataSerializer.DEBUG";
+    System.setProperty(propName, "true");
+    try {
+      DAY_OF_WEEK e = DAY_OF_WEEK.SUN;
+      MONTH m = MONTH.FEB;
+      DataOutputStream out = getDataOutput();
+      DataSerializer.writeObject(e, out);
+      DataSerializer.writeObject(m, out);
+      out.flush();
+  
+      DataInput in = getDataInput();
+      DAY_OF_WEEK e2 = (DAY_OF_WEEK)DataSerializer.readObject(in);
+      MONTH m2 = (MONTH)DataSerializer.readObject(in);
+      assertEquals(e, e2);
+      assertEquals(m, m2);
+      // Make sure there's nothing left in the stream
+      assertEquals(0, in.skipBytes(1));
+    } finally {
+      System.getProperties().remove(propName);
+    }
   }
   
   /**


[24/48] incubator-geode git commit: GEODE-647: fix GfshParserJUnitTest

Posted by kl...@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-217
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++) {


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

Posted by kl...@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-217
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;
     }
   }
 }



[08/48] incubator-geode git commit: GEODE-608: Refactor rat configuration into a separate script

Posted by kl...@apache.org.
GEODE-608: Refactor rat configuration into a separate script

Split out the rat configuration and update 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/3da5bccb
Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/3da5bccb
Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/3da5bccb

Branch: refs/heads/feature/GEODE-217
Commit: 3da5bccb5b63185f2d503772fab9733aa08da035
Parents: e45539a
Author: Anthony Baker <ab...@pivotal.io>
Authored: Sat Dec 5 10:47:43 2015 -0800
Committer: Anthony Baker <ab...@pivotal.io>
Committed: Thu Dec 10 09:44:57 2015 -0800

----------------------------------------------------------------------
 build.gradle      | 112 +-----------------------------------------
 gradle/rat.gradle | 129 +++++++++++++++++++++++++++++++++++++++++++++++++
 2 files changed, 130 insertions(+), 111 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3da5bccb/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index 6045b72..9042976 100755
--- a/build.gradle
+++ b/build.gradle
@@ -10,118 +10,8 @@ buildscript {
   }
 }
 
+apply from: 'gradle/rat.gradle'
 apply plugin: 'wrapper'
-apply plugin: "org.nosphere.apache.rat"
-
-rat {
-  excludes = [
-    // git
-    '.git/**',
-    '**/.gitignore',
-    
-    // gradle
-    '**/.gradle/**',
-    'gradlew',
-    'gradlew.bat',
-    'gradle/wrapper/gradle-wrapper.properties',
-    '**/build/**',
-    
-    // IDE
-    'etc/eclipseFormatterProfile.xml',
-    'etc/intellijIdeaCodeStyle.xml',
-    '**/.project',
-    '**/.classpath',
-    '**/.settings/**',
-    '**/build-eclipse/**',
-    '*.iml',
-    '.idea/**',
-
-    // text files
-    '**/*.fig',
-    '**/*.txt',
-    '**/*.md',
-    '**/*.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',
-    
-    // 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',
-    '**/META-INF/services/com.gemstone.gemfire.internal.cache.xmlcache.XmlParser',
-    '**/META-INF/services/org.springframework.shell.core.CommandMarker',
-
-    // --- Other Licenses ---
-    
-    // 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',
-
-    // Public Domain http://meyerweb.com/eric/tools/css/reset/
-    'gemfire-web-api/src/main/webapp/docs/css/reset.css',
-
-    // JSON License - permissive, used for Good, not Evil
-    'gemfire-json/src/main/java/org/json/CDL.java',
-    'gemfire-json/src/main/java/org/json/Cookie.java',
-    'gemfire-json/src/main/java/org/json/CookieList.java',
-    'gemfire-json/src/main/java/org/json/CDL.java',
-    'gemfire-json/src/main/java/org/json/Cookie.java',
-    'gemfire-json/src/main/java/org/json/CookieList.java',
-    'gemfire-json/src/main/java/org/json/HTTP.java',
-    'gemfire-json/src/main/java/org/json/HTTPTokener.java',
-    'gemfire-json/src/main/java/org/json/JSONArray.java',
-    'gemfire-json/src/main/java/org/json/JSONException.java',
-    'gemfire-json/src/main/java/org/json/JSONML.java',
-    'gemfire-json/src/main/java/org/json/JSONObject.java',
-    'gemfire-json/src/main/java/org/json/JSONString.java',
-    'gemfire-json/src/main/java/org/json/JSONStringer.java',
-    'gemfire-json/src/main/java/org/json/JSONTokener.java',
-    'gemfire-json/src/main/java/org/json/JSONWriter.java',
-    'gemfire-json/src/main/java/org/json/XML.java',
-    'gemfire-json/src/main/java/org/json/XMLTokener.java',
-
-    // MIT License
-    'gemfire-web-api/src/main/webapp/docs/lib/backbone-min.js',
-    'gemfire-web-api/src/main/webapp/docs/lib/jquery-1.8.0.min.js',
-    'gemfire-web-api/src/main/webapp/docs/lib/jquery.ba-bbq.min.js',
-    'gemfire-web-api/src/main/webapp/docs/lib/jquery.slideto.min.js',
-    'gemfire-web-api/src/main/webapp/docs/lib/jquery.wiggle.min.js',
-    'gemfire-web-api/src/main/webapp/docs/lib/underscore-min.js',
-    'gemfire-site/src/jbake/**',
-
-    // MIT or ISC
-    'gemfire-web-api/src/main/webapp/docs/lib/shred.bundle.js',
-    'gemfire-web-api/src/main/webapp/docs/lib/shred/content.js',
-
-    // BSD License
-    'gemfire-web-api/src/main/webapp/docs/lib/highlight.7.3.pack.js',
-
-    // Apache License
-    'gemfire-web-api/src/main/webapp/docs/o2c.html',
-    'gemfire-web-api/src/main/webapp/docs/index.html',
-    'gemfire-web-api/src/main/webapp/docs/lib/swagger-oauth.js',
-    'gemfire-web-api/src/main/webapp/docs/lib/swagger.js',
-    'gemfire-web-api/src/main/webapp/docs/css/screen.css',
-    'gemfire-web-api/src/main/webapp/docs/swagger-ui.js',
-    'gemfire-web-api/src/main/webapp/docs/swagger-ui.min.js'
-  ]
-}
 
 // Load all properties in dependency-version.properties as project properties, so all projects can read them
 Properties dependencyVersions = new Properties()

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3da5bccb/gradle/rat.gradle
----------------------------------------------------------------------
diff --git a/gradle/rat.gradle b/gradle/rat.gradle
new file mode 100644
index 0000000..96d1944
--- /dev/null
+++ b/gradle/rat.gradle
@@ -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.
+ */
+apply plugin: "org.nosphere.apache.rat"
+
+rat {
+  excludes = [
+    // git
+    '.git/**',
+    '**/.gitignore',
+    
+    // gradle
+    '**/.gradle/**',
+    'gradlew',
+    'gradlew.bat',
+    'gradle/wrapper/gradle-wrapper.properties',
+    '**/build/**',
+    
+    // IDE
+    'etc/eclipseFormatterProfile.xml',
+    'etc/intellijIdeaCodeStyle.xml',
+    '**/.project',
+    '**/.classpath',
+    '**/.settings/**',
+    '**/build-eclipse/**',
+    '*.iml',
+    '.idea/**',
+
+    // text files
+    '**/*.fig',
+    '**/*.txt',
+    '**/*.md',
+    '**/*.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',
+    '**/log4j2*.xml',
+ 
+    // 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',
+    '**/META-INF/services/com.gemstone.gemfire.internal.cache.xmlcache.XmlParser',
+    '**/META-INF/services/org.springframework.shell.core.CommandMarker',
+
+    // --- Other Licenses ---
+    
+    // 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',
+
+    // Public Domain http://meyerweb.com/eric/tools/css/reset/
+    'gemfire-web-api/src/main/webapp/docs/css/reset.css',
+
+    // JSON License - permissive, used for Good, not Evil
+    'gemfire-json/src/main/java/org/json/CDL.java',
+    'gemfire-json/src/main/java/org/json/Cookie.java',
+    'gemfire-json/src/main/java/org/json/CookieList.java',
+    'gemfire-json/src/main/java/org/json/CDL.java',
+    'gemfire-json/src/main/java/org/json/Cookie.java',
+    'gemfire-json/src/main/java/org/json/CookieList.java',
+    'gemfire-json/src/main/java/org/json/HTTP.java',
+    'gemfire-json/src/main/java/org/json/HTTPTokener.java',
+    'gemfire-json/src/main/java/org/json/JSONArray.java',
+    'gemfire-json/src/main/java/org/json/JSONException.java',
+    'gemfire-json/src/main/java/org/json/JSONML.java',
+    'gemfire-json/src/main/java/org/json/JSONObject.java',
+    'gemfire-json/src/main/java/org/json/JSONString.java',
+    'gemfire-json/src/main/java/org/json/JSONStringer.java',
+    'gemfire-json/src/main/java/org/json/JSONTokener.java',
+    'gemfire-json/src/main/java/org/json/JSONWriter.java',
+    'gemfire-json/src/main/java/org/json/XML.java',
+    'gemfire-json/src/main/java/org/json/XMLTokener.java',
+
+    // MIT License
+    'gemfire-web-api/src/main/webapp/docs/lib/backbone-min.js',
+    'gemfire-web-api/src/main/webapp/docs/lib/jquery-1.8.0.min.js',
+    'gemfire-web-api/src/main/webapp/docs/lib/jquery.ba-bbq.min.js',
+    'gemfire-web-api/src/main/webapp/docs/lib/jquery.slideto.min.js',
+    'gemfire-web-api/src/main/webapp/docs/lib/jquery.wiggle.min.js',
+    'gemfire-web-api/src/main/webapp/docs/lib/underscore-min.js',
+    'gemfire-site/src/jbake/**',
+
+    // MIT or ISC
+    'gemfire-web-api/src/main/webapp/docs/lib/shred.bundle.js',
+    'gemfire-web-api/src/main/webapp/docs/lib/shred/content.js',
+
+    // BSD License
+    'gemfire-web-api/src/main/webapp/docs/lib/highlight.7.3.pack.js',
+
+    // Apache License
+    'gemfire-web-api/src/main/webapp/docs/o2c.html',
+    'gemfire-web-api/src/main/webapp/docs/index.html',
+    'gemfire-web-api/src/main/webapp/docs/lib/swagger-oauth.js',
+    'gemfire-web-api/src/main/webapp/docs/lib/swagger.js',
+    'gemfire-web-api/src/main/webapp/docs/css/screen.css',
+    'gemfire-web-api/src/main/webapp/docs/swagger-ui.js',
+    'gemfire-web-api/src/main/webapp/docs/swagger-ui.min.js'
+  ]
+}
+


[12/48] incubator-geode git commit: GEODE-644: add unit test for DataAsAddress

Posted by kl...@apache.org.
GEODE-644: add unit test for DataAsAddress


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

Branch: refs/heads/feature/GEODE-217
Commit: 56b3757129fb20ab047d8c164fa965eff50099c4
Parents: 7cbb5db
Author: Sai Boorlagadda <sb...@pivotal.io>
Authored: Tue Dec 8 15:08:02 2015 -0800
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Thu Dec 10 11:11:30 2015 -0800

----------------------------------------------------------------------
 .../gemfire/internal/offheap/DataAsAddress.java |  14 +-
 .../offheap/OffHeapRegionEntryHelper.java       |  51 +--
 .../offheap/AbstractStoredObjectTestBase.java   | 203 ++++++++++
 .../offheap/DataAsAddressJUnitTest.java         | 368 +++++++++++++++++++
 4 files changed, 593 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/56b37571/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/DataAsAddress.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/DataAsAddress.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/DataAsAddress.java
index 5b14389..dff1d8f 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/DataAsAddress.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/DataAsAddress.java
@@ -59,7 +59,13 @@ public class DataAsAddress extends AbstractStoredObject {
   }
 
   public byte[] getDecompressedBytes(RegionEntryContext r) {
-    return OffHeapRegionEntryHelper.encodedAddressToBytes(this.address, true, r);
+    byte[] bytes = OffHeapRegionEntryHelper.decodeAddressToBytes(getEncodedAddress(), true, true);
+    if (isCompressed()) {
+        long time = r.getCachePerfStats().startDecompression();
+        bytes = r.getCompressor().decompress(bytes);
+        r.getCachePerfStats().endDecompression(time);
+    }
+    return bytes;
   }
 
   /**
@@ -67,17 +73,17 @@ public class DataAsAddress extends AbstractStoredObject {
    * Otherwise return the serialize bytes in us in a byte array.
    */
   public byte[] getRawBytes() {
-    return OffHeapRegionEntryHelper.encodedAddressToRawBytes(this.address);
+    return OffHeapRegionEntryHelper.decodeAddressToBytes(getEncodedAddress(), true, false);
   }
   
   @Override
   public byte[] getSerializedValue() {
-    return OffHeapRegionEntryHelper.encodedAddressToBytes(this.address);
+    return OffHeapRegionEntryHelper.decodeAddressToBytes(this.address);
   }
 
   @Override
   public Object getDeserializedValue(Region r, RegionEntry re) {
-    return OffHeapRegionEntryHelper.encodedAddressToObject(this.address);
+    return OffHeapRegionEntryHelper.decodeAddressToObject(this.address);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/56b37571/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionEntryHelper.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionEntryHelper.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionEntryHelper.java
index d1a81f0..1731b01 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionEntryHelper.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapRegionEntryHelper.java
@@ -87,10 +87,6 @@ public class OffHeapRegionEntryHelper {
     if (v == Token.NOT_AVAILABLE) return NOT_AVAILABLE_ADDRESS;
     throw new IllegalStateException("Can not convert " + v + " to an off heap address.");
   }
-
-  static Object encodedAddressToObject(long ohAddress) {
-    return encodedAddressToObject(ohAddress, true, true);
-  }
   
   //TODO:Asif:Check if this is a valid equality conditions
   public static boolean isAddressInvalidOrRemoved(long address) {
@@ -280,23 +276,20 @@ public class OffHeapRegionEntryHelper {
     }
     return 0L;
   }
-  
-  public static Object encodedAddressToObject(long addr, boolean decompress, boolean deserialize) {
-    boolean isSerialized = (addr & SERIALIZED_BIT) != 0;
-    byte[] bytes = encodedAddressToBytes(addr, decompress, false);
-    if (isSerialized) {
-      if (deserialize) {
-        return EntryEventImpl.deserialize(bytes);
+
+  static Object decodeAddressToObject(long ohAddress) {
+      byte[] bytes = decodeAddressToBytes(ohAddress, true, false);
+
+      boolean isSerialized = (ohAddress & SERIALIZED_BIT) != 0;
+      if (isSerialized) {
+         return EntryEventImpl.deserialize(bytes);
       } else {
-        return CachedDeserializableFactory.create(bytes);
+          return bytes;
       }
-    } else {
-      return bytes;
-    }
   }
-  
-  static byte[] encodedAddressToBytes(long addr) {
-    byte[] result = encodedAddressToBytes(addr, true, false);
+
+  static byte[] decodeAddressToBytes(long addr) {
+    byte[] result = decodeAddressToBytes(addr, true, false);
     boolean isSerialized = (addr & SERIALIZED_BIT) != 0;
     if (!isSerialized) {
       result = EntryEventImpl.serialize(result);
@@ -304,15 +297,7 @@ public class OffHeapRegionEntryHelper {
     return result;
   }
 
-  /**
-   * If the address contains a byte[] return it.
-   * Otherwise return the serialize bytes in the address in a byte array.
-   */
-  static byte[] encodedAddressToRawBytes(long addr) {
-    return encodedAddressToBytes(addr, true, false);
-  }
-
-  private static byte[] encodedAddressToBytes(long addr, boolean decompress, boolean compressedOk) {
+  static byte[] decodeAddressToBytes(long addr, boolean decompress, boolean compressedOk) {
     assert (addr & ENCODED_BIT) != 0;
     boolean isCompressed = (addr & COMPRESSED_BIT) != 0;
     int size = (int) ((addr & SIZE_MASK) >> SIZE_SHIFT);
@@ -344,18 +329,6 @@ public class OffHeapRegionEntryHelper {
     }
     return bytes;
   }
-  public static byte[] encodedAddressToBytes(long addr, boolean decompress, RegionEntryContext context) {
-    byte[] bytes = encodedAddressToBytes(addr, decompress, true);
-    if (decompress) {
-      boolean isCompressed = (addr & COMPRESSED_BIT) != 0;
-      if (isCompressed) {
-        long time = context.getCachePerfStats().startDecompression();
-        bytes = context.getCompressor().decompress(bytes);
-        context.getCachePerfStats().endDecompression(time);      
-      }
-    }
-    return bytes;
-  }
 
   /**
    * The previous value at the address in 're' will be @Released and then the

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/56b37571/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
new file mode 100644
index 0000000..a5fac48
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/AbstractStoredObjectTestBase.java
@@ -0,0 +1,203 @@
+/*
+ * 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 com.gemstone.gemfire.internal.DataSerializableFixedID;
+import com.gemstone.gemfire.internal.offheap.StoredObject;
+import org.junit.Test;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+public abstract class AbstractStoredObjectTestBase {
+
+    /* Returns Value as an Object Eg: Integer or UserDefinedRegionValue */
+    protected abstract Object getValue();
+
+    /* Returns Value as an ByteArray (not serialized) */
+    protected abstract byte[] getValueAsByteArray();
+
+    protected abstract Object convertByteArrayToObject(byte[] valueInByteArray);
+
+    protected abstract Object convertSerializedByteArrayToObject(byte[] valueInSerializedByteArray);
+
+    protected abstract StoredObject createValueAsUnserializedStoredObject(Object value);
+
+    protected abstract StoredObject createValueAsSerializedStoredObject(Object value);
+
+    @Test
+    public void getValueAsDeserializedHeapObjectShouldReturnDeserializedValueIfValueIsSerialized() {
+        Object regionEntryValue = getValue();
+        StoredObject storedObject = createValueAsSerializedStoredObject(regionEntryValue);
+
+        Integer actualRegionEntryValue = (Integer) storedObject.getValueAsDeserializedHeapObject();
+        assertEquals(regionEntryValue, actualRegionEntryValue);
+    }
+
+    @Test
+    public void getValueAsDeserializedHeapObjectShouldReturnValueAsIsIfNotSerialized() {
+        byte[] regionEntryValue = getValueAsByteArray();
+        StoredObject storedObject = createValueAsUnserializedStoredObject(regionEntryValue);
+
+        byte[] deserializedValue = (byte[]) storedObject.getValueAsDeserializedHeapObject();
+        assertArrayEquals(regionEntryValue, deserializedValue);
+    }
+
+    @Test
+    public void getValueAsHeapByteArrayShouldReturnSerializedByteArrayIfValueIsSerialized() {
+        Object regionEntryValue = getValue();
+        StoredObject storedObject = createValueAsSerializedStoredObject(regionEntryValue);
+
+        byte[] valueInSerializedByteArray = (byte[]) storedObject.getValueAsHeapByteArray();
+        Object actualRegionEntryValue = convertSerializedByteArrayToObject(valueInSerializedByteArray);
+
+        assertEquals(regionEntryValue, actualRegionEntryValue);
+    }
+
+    @Test
+    public void getValueAsHeapByteArrayShouldReturnDeserializedByteArrayIfValueIsNotSerialized() {
+        Object regionEntryValue = getValue();
+
+        StoredObject storedObject = createValueAsUnserializedStoredObject(regionEntryValue);
+
+        byte[] valueInByteArray = (byte[]) storedObject.getValueAsHeapByteArray();
+
+        Object actualRegionEntryValue = convertByteArrayToObject(valueInByteArray);
+
+        assertEquals(regionEntryValue, actualRegionEntryValue);
+    }
+
+    @Test
+    public void getStringFormShouldReturnStringFromDeserializedValue() {
+        Object regionEntryValue = getValue();
+        StoredObject storedObject = createValueAsSerializedStoredObject(regionEntryValue);
+
+        String stringForm = storedObject.getStringForm();
+        assertEquals(String.valueOf(regionEntryValue), stringForm);
+    }
+
+    @Test
+    public void getValueShouldReturnSerializedValue() {
+        Object regionEntryValue = getValue();
+        StoredObject storedObject = createValueAsSerializedStoredObject(regionEntryValue);
+
+        byte[] valueAsSerializedByteArray = (byte[]) storedObject.getValue();
+
+        Object actualValue = convertSerializedByteArrayToObject(valueAsSerializedByteArray);
+
+        assertEquals(regionEntryValue, actualValue);
+    }
+
+    @Test(expected = IllegalStateException.class)
+    public void getValueShouldThrowExceptionIfValueIsNotSerialized() {
+        Object regionEntryValue = getValue();
+        StoredObject storedObject = createValueAsUnserializedStoredObject(regionEntryValue);
+
+        byte[] deserializedValue = (byte[]) storedObject.getValue();
+    }
+
+    @Test
+    public void getDeserializedWritableCopyShouldReturnDeserializedValue() {
+        byte[] regionEntryValue = getValueAsByteArray();
+        StoredObject storedObject = createValueAsSerializedStoredObject(regionEntryValue);
+
+        assertArrayEquals(regionEntryValue, (byte[]) storedObject.getDeserializedWritableCopy(null, null));
+    }
+
+    @Test
+    public void writeValueAsByteArrayWritesToProvidedDataOutput() throws IOException {
+        byte[] regionEntryValue = getValueAsByteArray();
+        StoredObject storedObject = createValueAsSerializedStoredObject(regionEntryValue);
+
+        DataOutput dataOutput = mock(DataOutput.class);
+        storedObject.writeValueAsByteArray(dataOutput);
+
+        verify(dataOutput, times(1)).write(storedObject.getSerializedValue(), 0 , storedObject.getSerializedValue().length);
+    }
+
+    @Test
+    public void sendToShouldWriteSerializedValueToDataOutput() throws IOException {
+        Object regionEntryValue = getValue();
+        StoredObject storedObject = createValueAsSerializedStoredObject(regionEntryValue);
+
+        DataOutput dataOutput = mock(DataOutput.class);
+        storedObject.sendTo(dataOutput);
+
+        verify(dataOutput, times(1)).write(storedObject.getSerializedValue());
+    }
+
+    @Test
+    public void sendToShouldWriteDeserializedObjectToDataOutput() throws IOException {
+        byte[] regionEntryValue = getValueAsByteArray();
+        StoredObject storedObject = createValueAsUnserializedStoredObject(regionEntryValue);
+
+        DataOutput dataOutput = mock(DataOutput.class);
+        storedObject.sendTo(dataOutput);
+
+        verify(dataOutput, times(1)).write(regionEntryValue, 0, regionEntryValue.length);
+    }
+
+    @Test
+    public void sendAsByteArrayShouldWriteSerializedValueToDataOutput() throws IOException {
+        Object regionEntryValue = getValue();
+        StoredObject storedObject = createValueAsSerializedStoredObject(regionEntryValue);
+
+        DataOutput dataOutput = mock(DataOutput.class);
+        storedObject.sendAsByteArray(dataOutput);
+
+        verify(dataOutput, times(1)).write(storedObject.getSerializedValue(), 0, storedObject.getSerializedValue().length);
+    }
+
+    @Test
+    public void sendAsByteArrayShouldWriteDeserializedObjectToDataOutput() throws IOException {
+        byte[] regionEntryValue = getValueAsByteArray();
+        StoredObject storedObject = createValueAsUnserializedStoredObject(regionEntryValue);
+
+        DataOutput dataOutput = mock(DataOutput.class);
+        storedObject.sendAsByteArray(dataOutput);
+
+        verify(dataOutput, times(1)).write(regionEntryValue, 0, regionEntryValue.length);
+    }
+
+    @Test
+    public void sendAsCachedDeserializableShouldWriteSerializedValueToDataOutputAndSetsHeader() throws IOException {
+        Object regionEntryValue = getValue();
+        StoredObject storedObject = createValueAsSerializedStoredObject(regionEntryValue);
+
+        DataOutput dataOutput = mock(DataOutput.class);
+        storedObject.sendAsCachedDeserializable(dataOutput);
+
+        verify(dataOutput, times(1)).writeByte((DataSerializableFixedID.VM_CACHED_DESERIALIZABLE));
+        verify(dataOutput, times(1)).write(storedObject.getSerializedValue(), 0, storedObject.getSerializedValue().length);
+    }
+
+    @Test(expected = IllegalStateException.class)
+    public void sendAsCachedDeserializableShouldThrowExceptionIfValueIsNotSerialized() throws IOException {
+        Object regionEntryValue = getValue();
+        StoredObject storedObject = createValueAsUnserializedStoredObject(regionEntryValue);
+
+        DataOutput dataOutput = mock(DataOutput.class);
+        storedObject.sendAsCachedDeserializable(dataOutput);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/56b37571/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/DataAsAddressJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/DataAsAddressJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/DataAsAddressJUnitTest.java
new file mode 100644
index 0000000..8a251fd
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/DataAsAddressJUnitTest.java
@@ -0,0 +1,368 @@
+/*
+ * 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 com.gemstone.gemfire.compression.Compressor;
+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.DataAsAddress;
+
+import com.gemstone.gemfire.internal.offheap.OffHeapRegionEntryHelper;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mock;
+
+import java.nio.ByteBuffer;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.mockito.Mockito.*;
+
+@Category(UnitTest.class)
+public class DataAsAddressJUnitTest extends AbstractStoredObjectTestBase {
+
+    @Override
+    public Object getValue() {
+        return Integer.valueOf(123456789);
+    }
+
+    @Override
+    public byte[] getValueAsByteArray() {
+        return convertValueToByteArray(getValue());
+    }
+
+    private byte[] convertValueToByteArray(Object value) {
+        return ByteBuffer.allocate(Integer.SIZE / Byte.SIZE).putInt((Integer) value).array();
+    }
+
+    @Override
+    public Object convertByteArrayToObject(byte[] valueInByteArray) {
+        return ByteBuffer.wrap(valueInByteArray).getInt();
+    }
+
+    @Override
+    public Object convertSerializedByteArrayToObject(byte[] valueInSerializedByteArray) {
+       return EntryEventImpl.deserialize(valueInSerializedByteArray);
+    }
+
+    @Override
+    public DataAsAddress createValueAsUnserializedStoredObject(Object value) {
+        byte[] valueInByteArray;
+        if(value instanceof Integer) {
+            valueInByteArray = convertValueToByteArray(value);
+        } else {
+            valueInByteArray = (byte[]) value;
+        }
+        //encode a non-serialized entry value to address
+        long encodedAddress = OffHeapRegionEntryHelper.encodeDataAsAddress(valueInByteArray, false, false);
+        return new DataAsAddress(encodedAddress);
+    }
+
+    @Override
+    public DataAsAddress createValueAsSerializedStoredObject(Object value) {
+        byte[] valueInSerializedByteArray = EntryEventImpl.serialize(value);
+        //encode a serialized entry value to address
+        long encodedAddress = OffHeapRegionEntryHelper.encodeDataAsAddress(valueInSerializedByteArray, true, false);
+        return new DataAsAddress(encodedAddress);
+    }
+
+    public DataAsAddress createValueAsCompressedStoredObject(Object value) {
+        byte[] valueInSerializedByteArray = EntryEventImpl.serialize(value);
+        //encode a serialized, compressed entry value to address
+        long encodedAddress = OffHeapRegionEntryHelper.encodeDataAsAddress(valueInSerializedByteArray, true, true);
+        return new DataAsAddress(encodedAddress);
+    }
+
+    public DataAsAddress createValueAsUncompressedStoredObject(Object value) {
+        byte[] valueInSerializedByteArray = EntryEventImpl.serialize(value);
+        //encode a serialized, uncompressed entry value to address
+        long encodedAddress = OffHeapRegionEntryHelper.encodeDataAsAddress(valueInSerializedByteArray, true, false);
+        return new DataAsAddress(encodedAddress);
+    }
+
+    @Test
+    public void shouldReturnCorrectEncodingAddress() {
+
+        DataAsAddress address1 = new DataAsAddress(10001L);
+        assertNotNull(address1);
+        assertEquals("Encoding address should be:", 10001, address1.getEncodedAddress());
+
+        DataAsAddress address2 = new DataAsAddress(10002L);
+        assertNotNull(address2);
+        assertEquals("Returning always 10001 expected 10002", 10002, address2.getEncodedAddress());
+    }
+
+    @Test
+    public void twoAddressesShouldBeEqualIfEncodingAddressIsSame() {
+        DataAsAddress address1 = new DataAsAddress(10001L);
+        DataAsAddress address2 = new DataAsAddress(10001L);
+
+        assertEquals("Two addresses are equal if encoding address is same", true, address1.equals(address2));
+    }
+
+    @Test
+    public void twoAddressesShouldNotBeEqualIfEncodingAddressIsNotSame() {
+        DataAsAddress address1 = new DataAsAddress(10001L);
+        DataAsAddress address2 = new DataAsAddress(10002L);
+
+        assertEquals("Two addresses are not equal if encoding address is not same", false, address1.equals(address2));
+    }
+
+    @Test
+    public void twoAddressesAreNotEqualIfTheyAreNotTypeDataAsAddress() {
+        DataAsAddress address1 = new DataAsAddress(10001L);
+        Long address2 = new Long(10002L);
+
+        assertEquals("Two addresses are not equal if encoding address is not same", false, address1.equals(address2));
+    }
+
+    @Test
+    public void addressHashCodeShouldBe() {
+        DataAsAddress address1 = new DataAsAddress(10001L);
+        assertEquals("", 10001, address1.hashCode());
+    }
+
+    @Test
+    public void getSizeInBytesAlwaysReturnsZero() {
+        DataAsAddress address1 = new DataAsAddress(10001L);
+        DataAsAddress address2 = new DataAsAddress(10002L);
+
+        assertEquals("getSizeInBytes", 0, address1.getSizeInBytes());
+        assertEquals("getSizeInBytes", 0, address2.getSizeInBytes());
+    }
+
+    @Test
+    public void getValueSizeInBytesAlwaysReturnsZero() {
+        DataAsAddress address1 = new DataAsAddress(10001L);
+        DataAsAddress address2 = new DataAsAddress(10002L);
+
+        assertEquals("getSizeInBytes", 0, address1.getValueSizeInBytes());
+        assertEquals("getSizeInBytes", 0, address2.getValueSizeInBytes());
+    }
+
+    @Test
+    public void retainShouldAlwaysBeTrue() {
+        DataAsAddress address1 = new DataAsAddress(10001L);
+        DataAsAddress address2 = new DataAsAddress(10002L);
+
+        assertEquals("retain", true, address1.retain());
+        assertEquals("retain", true, address2.retain());
+    }
+
+    @Test
+    public void dataAsAddressShouldImplementReleaseToAdhereToStoredObject() {
+        DataAsAddress address = new DataAsAddress(10001L);
+        address.release();
+    }
+
+    @Test
+    public void isCompressedShouldReturnTrueIfCompressed() {
+        Object regionEntryValue = getValue();
+
+        DataAsAddress offheapAddress = createValueAsCompressedStoredObject(regionEntryValue);
+
+        assertEquals("Should return true as it is compressed", true, offheapAddress.isCompressed());
+    }
+
+    @Test
+    public void isCompressedShouldReturnFalseIfNotCompressed() {
+        Object regionEntryValue = getValue();
+
+        DataAsAddress offheapAddress = createValueAsUncompressedStoredObject(regionEntryValue);
+
+        assertEquals("Should return false as it is compressed", false, offheapAddress.isCompressed());
+    }
+
+    @Test
+    public void isSerializedShouldReturnTrueIfSeriazlied() {
+        Object regionEntryValue = getValue();
+
+        DataAsAddress offheapAddress = createValueAsSerializedStoredObject(regionEntryValue);
+
+        assertEquals("Should return true as it is serialized", true, offheapAddress.isSerialized());
+    }
+
+    @Test
+    public void isSerializedShouldReturnFalseIfNotSeriazlied() {
+        Object regionEntryValue = getValue();
+
+        DataAsAddress offheapAddress = createValueAsUnserializedStoredObject(regionEntryValue);
+
+        assertEquals("Should return false as it is serialized", false, offheapAddress.isSerialized());
+    }
+
+    @Test
+    public void getDecompressedBytesShouldReturnDecompressedBytesIfCompressed() {
+        Object regionEntryValue = getValue();
+        byte[] regionEntryValueAsBytes =  convertValueToByteArray(regionEntryValue);
+
+        //encode a non-serialized and compressed entry value to address - last argument is to let that it is compressed
+        long encodedAddress = OffHeapRegionEntryHelper.encodeDataAsAddress(regionEntryValueAsBytes, false, true);
+        DataAsAddress offheapAddress = new DataAsAddress(encodedAddress);
+
+        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 = offheapAddress.getDecompressedBytes(regionContext);
+
+        //verify the thing happened
+        verify(cacheStats, atLeastOnce()).startDecompression();
+        verify(compressor, times(1)).decompress(regionEntryValueAsBytes);
+        verify(cacheStats, atLeastOnce()).endDecompression(startTime);
+
+        assertArrayEquals(regionEntryValueAsBytes, bytes);
+    }
+
+    @Test
+    public void getDecompressedBytesShouldNotTryToDecompressIfNotCompressed() {
+        Object regionEntryValue = getValue();
+
+        DataAsAddress offheapAddress = createValueAsUncompressedStoredObject(regionEntryValue);
+
+        //mock the thing
+        RegionEntryContext regionContext = mock(RegionEntryContext.class);
+        Compressor compressor = mock(Compressor.class);
+        when(regionContext.getCompressor()).thenReturn(compressor);
+
+        //invoke the thing
+        byte[] actualValueInBytes = offheapAddress.getDecompressedBytes(regionContext);
+
+        //createValueAsUncompressedStoredObject does uses a serialized value - so convert it to object
+        Object actualRegionValue = convertSerializedByteArrayToObject(actualValueInBytes);
+
+        //verify the thing happened
+        verify(regionContext, never()).getCompressor();
+        assertEquals(regionEntryValue, actualRegionValue);
+    }
+
+    @Test
+    public void getRawBytesShouldReturnAByteArray() {
+        byte[] regionEntryValueAsBytes = getValueAsByteArray();
+
+        DataAsAddress offheapAddress = createValueAsUnserializedStoredObject(regionEntryValueAsBytes);
+        byte[] actual = offheapAddress.getRawBytes();
+
+        assertArrayEquals(regionEntryValueAsBytes, actual);
+    }
+
+    @Test
+    public void getSerializedValueShouldReturnASerializedByteArray() {
+        Object regionEntryValue = getValue();
+
+        DataAsAddress offheapAddress = createValueAsSerializedStoredObject(regionEntryValue);
+
+        byte[] actualSerializedValue = offheapAddress.getSerializedValue();
+
+        Object actualRegionEntryValue = convertSerializedByteArrayToObject(actualSerializedValue);
+
+        assertEquals(regionEntryValue, actualRegionEntryValue);
+    }
+
+    @Test
+    public void getDeserializedObjectShouldReturnADeserializedObject() {
+        Object regionEntryValue = getValue();
+
+        DataAsAddress offheapAddress = createValueAsSerializedStoredObject(regionEntryValue);
+
+        Integer actualRegionEntryValue = (Integer) offheapAddress.getDeserializedValue(null, null);
+
+        assertEquals(regionEntryValue, actualRegionEntryValue);
+    }
+
+    @Test
+    public void getDeserializedObjectShouldReturnAByteArrayAsIsIfNotSerialized() {
+        byte[] regionEntryValueAsBytes = getValueAsByteArray();
+
+        DataAsAddress offheapAddress = createValueAsUnserializedStoredObject(regionEntryValueAsBytes);
+
+        byte[] deserializeValue = (byte[]) offheapAddress.getDeserializedValue(null, null);
+
+        assertArrayEquals(regionEntryValueAsBytes, deserializeValue);
+    }
+
+    @Test
+    public void fillSerializedValueShouldFillWrapperWithSerializedValueIfValueIsSerialized() {
+        Object regionEntryValue = getValue();
+        byte[] serializedRegionEntryValue = EntryEventImpl.serialize(regionEntryValue);
+
+        //encode a serialized entry value to address
+        long encodedAddress = OffHeapRegionEntryHelper.encodeDataAsAddress(serializedRegionEntryValue, true, false);
+
+        DataAsAddress offheapAddress = new DataAsAddress(encodedAddress);
+
+        //mock the things
+        BytesAndBitsForCompactor wrapper = mock(BytesAndBitsForCompactor.class);
+
+        byte userBits = 1;
+        offheapAddress.fillSerializedValue(wrapper, userBits);
+
+        verify(wrapper, times(1)).setData(serializedRegionEntryValue, userBits, serializedRegionEntryValue.length, true);
+    }
+
+    @Test
+    public void fillSerializedValueShouldFillWrapperWithDeserializedValueIfValueIsNotSerialized() {
+        Object regionEntryValue = getValue();
+        byte[] regionEntryValueAsBytes =  convertValueToByteArray(regionEntryValue);
+
+        //encode a un serialized entry value to address
+        long encodedAddress = OffHeapRegionEntryHelper.encodeDataAsAddress(regionEntryValueAsBytes, false, false);
+
+        DataAsAddress offheapAddress = new DataAsAddress(encodedAddress);
+
+        //mock the things
+        BytesAndBitsForCompactor wrapper = mock(BytesAndBitsForCompactor.class);
+
+        byte userBits = 1;
+        offheapAddress.fillSerializedValue(wrapper, userBits);
+
+        verify(wrapper, times(1)).setData(regionEntryValueAsBytes, userBits, regionEntryValueAsBytes.length, true);
+    }
+
+    @Test
+    public void getStringFormShouldCatchExceptionAndReturnErrorMessageAsString() {
+        Object regionEntryValueAsBytes = getValue();
+
+        byte[] serializedValue = EntryEventImpl.serialize(regionEntryValueAsBytes);
+
+        //store -127 (DSCODE.ILLEGAL) - in order the deserialize to throw exception
+        serializedValue[0] = -127;
+
+        //encode a un serialized entry value to address
+        long encodedAddress = OffHeapRegionEntryHelper.encodeDataAsAddress(serializedValue, true, false);
+
+        DataAsAddress offheapAddress = new DataAsAddress(encodedAddress);
+
+        String errorMessage = offheapAddress.getStringForm();
+
+        assertEquals(true, errorMessage.contains("Could not convert object to string because "));
+    }
+}


[27/48] incubator-geode git commit: StateFlushOperation flushing/waiting for multicast messages

Posted by kl...@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-217
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


[07/48] incubator-geode git commit: GEODE-608: Exclude cache element xml to prevent failing test

Posted by kl...@apache.org.
GEODE-608: Exclude cache element xml to prevent failing test


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

Branch: refs/heads/feature/GEODE-217
Commit: c9f677b49deed0aac6741f731f3b842821a7d512
Parents: 3da5bcc
Author: Anthony Baker <ab...@pivotal.io>
Authored: Wed Dec 9 14:30:30 2015 -0800
Committer: Anthony Baker <ab...@pivotal.io>
Committed: Thu Dec 10 09:44:57 2015 -0800

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


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c9f677b4/gradle/rat.gradle
----------------------------------------------------------------------
diff --git a/gradle/rat.gradle b/gradle/rat.gradle
index 96d1944..ab2da4c 100644
--- a/gradle/rat.gradle
+++ b/gradle/rat.gradle
@@ -58,6 +58,7 @@ rat {
     'gemfire-spark-connector/project/plugins.sbt',
     'gemfire-spark-connector/project/build.properties',
     '**/log4j2*.xml',
+    'gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/domain/CacheElementJUnitTest.xml',
  
     // ANTLR generated files
     'gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/parse/OQLLexer.java',


[35/48] incubator-geode git commit: Ignoring tags file from ctags in rat

Posted by kl...@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-217
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',


[19/48] incubator-geode git commit: More JoinLeeve tests

Posted by kl...@apache.org.
More JoinLeeve tests


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

Branch: refs/heads/feature/GEODE-217
Commit: 7dfce7cd92c7f2d9e89374dd7799eb87dd8711b3
Parents: 81c472f
Author: Hitesh Khamesra <hi...@yahoo.com>
Authored: Wed Dec 9 16:26:43 2015 -0800
Committer: Hitesh Khamesra <hi...@yahoo.com>
Committed: Thu Dec 10 15:31:14 2015 -0800

----------------------------------------------------------------------
 .../gms/locator/FindCoordinatorRequest.java     | 33 ++++++++++
 .../membership/gms/membership/GMSJoinLeave.java | 31 ++++++++--
 .../gms/membership/GMSJoinLeaveJUnitTest.java   | 63 ++++++++++++++++++++
 3 files changed, 123 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7dfce7cd/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/locator/FindCoordinatorRequest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/locator/FindCoordinatorRequest.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/locator/FindCoordinatorRequest.java
index f1ec2a0..5c0a1d1 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/locator/FindCoordinatorRequest.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/locator/FindCoordinatorRequest.java
@@ -112,4 +112,37 @@ public class FindCoordinatorRequest extends HighPriorityDistributionMessage
     throw new IllegalStateException("this message should not be executed");
   }
 
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + lastViewId;
+    result = prime * result + ((memberID == null) ? 0 : memberID.hashCode());
+    result = prime * result + ((rejectedCoordinators == null) ? 0 : rejectedCoordinators.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    FindCoordinatorRequest other = (FindCoordinatorRequest) obj;
+    if (lastViewId != other.lastViewId)
+      return false;
+    if (memberID == null) {
+      if (other.memberID != null)
+        return false;
+    } else if (!memberID.equals(other.memberID))
+      return false;
+    if (rejectedCoordinators == null) {
+      if (other.rejectedCoordinators != null)
+        return false;
+    } else if (!rejectedCoordinators.equals(other.rejectedCoordinators))
+      return false;
+    return true;
+  }  
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7dfce7cd/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 e1821db..3a3486b 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,7 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
    * @param coord
    * @return true if the attempt succeeded, false if it timed out
    */
-  private boolean attemptToJoin() {
+   boolean attemptToJoin() {
     SearchState state = searchState;
 
     // send a join request to the coordinator and wait for a response
@@ -826,6 +826,15 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
     }
   }
 
+  private TcpClientWrapper tcpClientWrapper = new TcpClientWrapper();
+  
+  /***
+   * testing purpose
+   * @param tcpClientWrapper
+   */
+  void setTcpClientWrapper(TcpClientWrapper tcpClientWrapper) {
+    this.tcpClientWrapper = tcpClientWrapper;
+  }
   /**
    * This contacts the locators to find out who the current coordinator is.
    * All locators are contacted. If they don't agree then we choose the oldest
@@ -861,9 +870,7 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
     do {
       for (InetSocketAddress addr : locators) {
         try {
-          Object o = TcpClient.requestToServer(
-              addr.getAddress(), addr.getPort(), request, connectTimeout, 
-              true);
+          Object o = tcpClientWrapper.sendCoordinatorFindRequest(addr, request, connectTimeout);
           FindCoordinatorResponse response = (o instanceof FindCoordinatorResponse) ? (FindCoordinatorResponse)o : null;
           if (response != null) {
             state.locatorsContacted++;
@@ -937,6 +944,15 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
     }
     return true;
   }
+  
+  protected class TcpClientWrapper {
+    protected Object sendCoordinatorFindRequest(InetSocketAddress addr, FindCoordinatorRequest request, int connectTimeout) 
+        throws ClassNotFoundException, IOException{
+      return TcpClient.requestToServer(
+          addr.getAddress(), addr.getPort(), request, connectTimeout, 
+          true);
+    }
+  }    
 
   boolean findCoordinatorFromView() {
     ArrayList<FindCoordinatorResponse> result;
@@ -1051,6 +1067,13 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
   JoinResponseMessage[] getJoinResponseMessage() {
     return joinResponse;
   }
+  /***
+   * for testing purpose
+   * @param jrm
+   */
+  void setJoinResponseMessage(JoinResponseMessage jrm) {
+    joinResponse[0] = jrm;
+  }
 
   private void processFindCoordinatorRequest(FindCoordinatorRequest req) {
     FindCoordinatorResponse resp;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7dfce7cd/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeaveJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeaveJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeaveJUnitTest.java
index 01c0695..05b0996 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeaveJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeaveJUnitTest.java
@@ -28,6 +28,7 @@ import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
 import java.io.IOException;
+import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -59,9 +60,11 @@ import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.Authe
 import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.HealthMonitor;
 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.locator.FindCoordinatorRequest;
 import com.gemstone.gemfire.distributed.internal.membership.gms.locator.FindCoordinatorResponse;
 import com.gemstone.gemfire.distributed.internal.membership.gms.membership.GMSJoinLeave.SearchState;
 import com.gemstone.gemfire.distributed.internal.membership.gms.membership.GMSJoinLeave.ViewCreator;
+import com.gemstone.gemfire.distributed.internal.membership.gms.membership.GMSJoinLeave.TcpClientWrapper;
 import com.gemstone.gemfire.distributed.internal.membership.gms.membership.GMSJoinLeave.ViewReplyProcessor;
 import com.gemstone.gemfire.distributed.internal.membership.gms.messages.InstallViewMessage;
 import com.gemstone.gemfire.distributed.internal.membership.gms.messages.JoinRequestMessage;
@@ -99,6 +102,7 @@ public class GMSJoinLeaveJUnitTest {
     when(mockDistConfig.getEnableNetworkPartitionDetection()).thenReturn(enableNetworkPartition);
     when(mockDistConfig.getLocators()).thenReturn("localhost[8888]");
     mockConfig = mock(ServiceConfig.class);
+    when(mockDistConfig.getStartLocator()).thenReturn("localhost[12345]");
     when(mockConfig.getDistributionConfig()).thenReturn(mockDistConfig);
     when(mockDistConfig.getLocators()).thenReturn("localhost[12345]");
     when(mockDistConfig.getMcastPort()).thenReturn(0);
@@ -1028,5 +1032,64 @@ public class GMSJoinLeaveJUnitTest {
     assertTrue(newView.contains(mockMembers[1]));
     assertTrue(newView.getViewId() > preparedView.getViewId());
   }
+
+  private NetView createView() {
+    List<InternalDistributedMember> mbrs = new LinkedList<>();
+    Set<InternalDistributedMember> shutdowns = new HashSet<>();
+    Set<InternalDistributedMember> crashes = new HashSet<>();
+    mbrs.add(mockMembers[0]);
+    mbrs.add(mockMembers[1]);
+    mbrs.add(mockMembers[2]);
+    mbrs.add(gmsJoinLeaveMemberId);
+    
+    //prepare the view
+    NetView netView = new NetView(mockMembers[0], 1, mbrs, shutdowns, crashes);
+    return netView;
+  }
+  
+  @Test
+  public void testCoordinatorFindRequestSuccess()  throws Exception {
+    try{
+      initMocks(false);
+      HashSet<InternalDistributedMember> registrants = new HashSet<>();
+      registrants.add(mockMembers[0]);
+      FindCoordinatorResponse fcr = new FindCoordinatorResponse(mockMembers[0], mockMembers[0], false, null, registrants, false, true);
+      NetView view = createView();
+      JoinResponseMessage jrm = new JoinResponseMessage(mockMembers[0], view);
+      gmsJoinLeave.setJoinResponseMessage(jrm);
+      
+      TcpClientWrapper tcpClientWrapper = mock(TcpClientWrapper.class);
+      gmsJoinLeave.setTcpClientWrapper(tcpClientWrapper);
+      FindCoordinatorRequest fcreq = new FindCoordinatorRequest(gmsJoinLeaveMemberId, new HashSet<>(), -1);
+      int connectTimeout = (int)services.getConfig().getMemberTimeout() * 2;
+      when(tcpClientWrapper.sendCoordinatorFindRequest(new InetSocketAddress("localhost", 12345), fcreq, connectTimeout)).thenReturn(fcr);
+      assertTrue("Should be able to join ", gmsJoinLeave.join());
+    }finally{
+      
+    }   
+  }
+  
+  @Test
+  public void testCoordinatorFindRequestFailure()  throws Exception {
+    try{
+      initMocks(false);
+      HashSet<InternalDistributedMember> registrants = new HashSet<>();
+      registrants.add(mockMembers[0]);
+      FindCoordinatorResponse fcr = new FindCoordinatorResponse(mockMembers[0], mockMembers[0], false, null, registrants, false, true);
+      NetView view = createView();
+      JoinResponseMessage jrm = new JoinResponseMessage(mockMembers[0], view);
+      gmsJoinLeave.setJoinResponseMessage(jrm);
+      
+      TcpClientWrapper tcpClientWrapper = mock(TcpClientWrapper.class);
+      gmsJoinLeave.setTcpClientWrapper(tcpClientWrapper);
+      FindCoordinatorRequest fcreq = new FindCoordinatorRequest(gmsJoinLeaveMemberId, new HashSet<>(), -1);
+      int connectTimeout = (int)services.getConfig().getMemberTimeout() * 2;
+      //passing wrong port here, so ot will fail
+      when(tcpClientWrapper.sendCoordinatorFindRequest(new InetSocketAddress("localhost", 12346), fcreq, connectTimeout)).thenReturn(fcr);
+      assertFalse("Should not be able to join ", gmsJoinLeave.join());
+    }finally{
+      
+    }   
+  }
 }
 


[39/48] 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 kl...@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>


[16/48] incubator-geode git commit: Closes #51 *move SameHashDifferentTrace to test code*

Posted by kl...@apache.org.
Closes #51 *move SameHashDifferentTrace to test 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/24c170af
Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/24c170af
Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/24c170af

Branch: refs/heads/feature/GEODE-217
Commit: 24c170afb4dba6f745dd24ab30de99b24474a3b5
Parents: 11c62f2
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Thu Dec 10 14:24:27 2015 -0800
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Thu Dec 10 14:24:27 2015 -0800

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

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



[43/48] 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 kl...@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;
-}


[33/48] incubator-geode git commit: Revert "Removing TCPConduit's Stub ID class"

Posted by kl...@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


[46/48] 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 kl...@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-217
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/*


[05/48] incubator-geode git commit: GEODE-18: Added missing source headers

Posted by kl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/loaderNotLoader.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/loaderNotLoader.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/loaderNotLoader.xml
index d24f8d0..b0bf8f4 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/loaderNotLoader.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/loaderNotLoader.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <!DOCTYPE cache PUBLIC
     "-//GemStone Systems, Inc.//GemFire Declarative Caching 4.0//EN"
     "http://www.gemstone.com/dtd/cache4_0.dtd">

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/malformed.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/malformed.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/malformed.xml
index 3c414be..8052ba5 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/malformed.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/malformed.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <!DOCTYPE cache PUBLIC
     "-//GemStone Systems, Inc.//GemFire Declarative Caching 4.0//EN"
     "http://www.gemstone.com/dtd/cache4_0.dtd">

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/namedAttributes.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/namedAttributes.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/namedAttributes.xml
index b6025fa..2d3b4a5 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/namedAttributes.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/namedAttributes.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <!DOCTYPE cache PUBLIC
     "-//GemStone Systems, Inc.//GemFire Declarative Caching 4.1//EN"
     "http://www.gemstone.com/dtd/cache4_1.dtd">

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/partitionedRegion.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/partitionedRegion.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/partitionedRegion.xml
index ee52010..773cb7b 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/partitionedRegion.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/partitionedRegion.xml
@@ -1,4 +1,21 @@
 <?xml version="1.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.
+-->
+
 <!DOCTYPE cache PUBLIC
     "-//GemStone Systems, Inc.//GemFire Declarative Caching 5.0//EN"
     "http://www.gemstone.com/dtd/cache5_0.dtd">

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/partitionedRegion51.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/partitionedRegion51.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/partitionedRegion51.xml
index 1181245..826e500 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/partitionedRegion51.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/partitionedRegion51.xml
@@ -1,4 +1,21 @@
 <?xml version="1.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.
+-->
+
 <!DOCTYPE cache PUBLIC
     "-//GemStone Systems, Inc.//GemFire Declarative Caching 5.1//EN"
     "http://www.gemstone.com/dtd/cache5_1.dtd">

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/sameRootRegion.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/sameRootRegion.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/sameRootRegion.xml
index b9dbfd4..f34061a 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/sameRootRegion.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/sameRootRegion.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <!DOCTYPE cache PUBLIC
     "-//GemStone Systems, Inc.//GemFire Declarative Caching 4.1//EN"
     "http://www.gemstone.com/dtd/cache4_1.dtd">

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/sameSubregion.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/sameSubregion.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/sameSubregion.xml
index 09d3014..26f95bb 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/sameSubregion.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/sameSubregion.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <!DOCTYPE cache PUBLIC
     "-//GemStone Systems, Inc.//GemFire Declarative Caching 4.1//EN"
     "http://www.gemstone.com/dtd/cache4_1.dtd">

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/unknownNamedAttributes.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/unknownNamedAttributes.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/unknownNamedAttributes.xml
index 1e51336..7e570e8 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/unknownNamedAttributes.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/unknownNamedAttributes.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <!DOCTYPE cache PUBLIC
     "-//GemStone Systems, Inc.//GemFire Declarative Caching 4.1//EN"
     "http://www.gemstone.com/dtd/cache4_1.dtd">

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/distributed/internal/SharedConfigurationJUnitTest.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/distributed/internal/SharedConfigurationJUnitTest.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/distributed/internal/SharedConfigurationJUnitTest.xml
index 65d5ddc..68ce00f 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/distributed/internal/SharedConfigurationJUnitTest.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/distributed/internal/SharedConfigurationJUnitTest.xml
@@ -1,4 +1,21 @@
 <?xml version="1.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.
+-->
+
 <!DOCTYPE cache PUBLIC
     "-//GemStone Systems, Inc.//GemFire Declarative Caching 8.0//EN"
     "http://www.gemstone.com/dtd/cache8_0.dtd">

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/BackupJUnitTest.cache.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/BackupJUnitTest.cache.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/BackupJUnitTest.cache.xml
index 4fabee0..01570f4 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/BackupJUnitTest.cache.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/BackupJUnitTest.cache.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <!DOCTYPE cache PUBLIC "-//GemStone Systems, Inc.//GemFire Declarative Caching 5.1//EN" "http://www.gemstone.com/dtd/cache5_1.dtd">
 <cache>
   <!--  nothing special here, we just want to make sure this file gets backed up, byte for byte. -->

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/DiskRegCacheXmlJUnitTest.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/DiskRegCacheXmlJUnitTest.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/DiskRegCacheXmlJUnitTest.xml
index fe575ae..b6818d0 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/DiskRegCacheXmlJUnitTest.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/DiskRegCacheXmlJUnitTest.xml
@@ -1,4 +1,20 @@
 <?xml version="1.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.
+-->
 
 <!-- DiskRegion.xml
      Configures the Disk Regions to Overflow / Persist /PersistWithOverflow its data to disk.  

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/PartitionRegionCacheExample1.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/PartitionRegionCacheExample1.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/PartitionRegionCacheExample1.xml
index ccdebae..e6e907e 100755
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/PartitionRegionCacheExample1.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/PartitionRegionCacheExample1.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <!DOCTYPE cache PUBLIC "-//GemStone Systems, Inc.//GemFire Declarative Caching 5.1//EN" "http://www.gemstone.com/dtd/cache5_1.dtd">
 <cache>
 	<!-- Create a new root region as partition region -->

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/PartitionRegionCacheExample2.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/PartitionRegionCacheExample2.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/PartitionRegionCacheExample2.xml
index de1db01..6434103 100755
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/PartitionRegionCacheExample2.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/PartitionRegionCacheExample2.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <!DOCTYPE cache PUBLIC "-//GemStone Systems, Inc.//GemFire Declarative Caching 5.1//EN" "http://www.gemstone.com/dtd/cache5_1.dtd">
 <cache search-timeout="60" lock-lease="300">
   <region name="root">

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/incorrect_bytes_threshold.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/incorrect_bytes_threshold.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/incorrect_bytes_threshold.xml
index 6d757d3..b734ce1 100755
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/incorrect_bytes_threshold.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/incorrect_bytes_threshold.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <!DOCTYPE cache PUBLIC "-//GemStone Systems, Inc.//GemFire Declarative Caching 6.5//EN" "http://www.gemstone.com/dtd/cache6_5.dtd">
 <cache lock-lease="120" lock-timeout="60" search-timeout="300" is-server="false" copy-on-read="false">
   <cache-transaction-manager/>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/incorrect_dir.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/incorrect_dir.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/incorrect_dir.xml
index 25ce234..71aba12 100755
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/incorrect_dir.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/incorrect_dir.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <!DOCTYPE cache PUBLIC "-//GemStone Systems, Inc.//GemFire Declarative Caching 6.5//EN" "http://www.gemstone.com/dtd/cache6_5.dtd">
 <cache lock-lease="120" lock-timeout="60" search-timeout="300" is-server="false" copy-on-read="false">
   <cache-transaction-manager/>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/incorrect_dir_size.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/incorrect_dir_size.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/incorrect_dir_size.xml
index 71bf3be..a03139f 100755
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/incorrect_dir_size.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/incorrect_dir_size.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <!DOCTYPE cache PUBLIC "-//GemStone Systems, Inc.//GemFire Declarative Caching 6.5//EN" "http://www.gemstone.com/dtd/cache6_5.dtd">
 <cache lock-lease="120" lock-timeout="60" search-timeout="300" is-server="false" copy-on-read="false">
   <cache-transaction-manager/>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/incorrect_max_oplog_size.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/incorrect_max_oplog_size.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/incorrect_max_oplog_size.xml
index f7a6e4b..71dc62f 100755
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/incorrect_max_oplog_size.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/incorrect_max_oplog_size.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <!DOCTYPE cache PUBLIC "-//GemStone Systems, Inc.//GemFire Declarative Caching 6.5//EN" "http://www.gemstone.com/dtd/cache6_5.dtd">
 <cache lock-lease="120" lock-timeout="60" search-timeout="300" is-server="false" copy-on-read="false">
   <cache-transaction-manager/>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/incorrect_roll_oplogs_value.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/incorrect_roll_oplogs_value.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/incorrect_roll_oplogs_value.xml
index 34e59b2..f3d8594 100755
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/incorrect_roll_oplogs_value.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/incorrect_roll_oplogs_value.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <!DOCTYPE cache PUBLIC "-//GemStone Systems, Inc.//GemFire Declarative Caching 6.5//EN" "http://www.gemstone.com/dtd/cache6_5.dtd">
 <cache lock-lease="120" lock-timeout="60" search-timeout="300" is-server="false" copy-on-read="false">
   <cache-transaction-manager/>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/incorrect_sync_value.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/incorrect_sync_value.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/incorrect_sync_value.xml
index acaeb58..990fa24 100755
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/incorrect_sync_value.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/incorrect_sync_value.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <!DOCTYPE cache PUBLIC "-//GemStone Systems, Inc.//GemFire Declarative Caching 6.5//EN" "http://www.gemstone.com/dtd/cache6_5.dtd">
 <cache lock-lease="120" lock-timeout="60" search-timeout="300" is-server="false" copy-on-read="false">
   <cache-transaction-manager/>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/incorrect_time_interval.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/incorrect_time_interval.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/incorrect_time_interval.xml
index 40fbaac..a55b690 100755
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/incorrect_time_interval.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/incorrect_time_interval.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <!DOCTYPE cache PUBLIC "-//GemStone Systems, Inc.//GemFire Declarative Caching 6.5//EN" "http://www.gemstone.com/dtd/cache6_5.dtd">
 <cache lock-lease="120" lock-timeout="60" search-timeout="300" is-server="false" copy-on-read="false">
   <cache-transaction-manager/>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/mixed_diskstore_diskdir.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/mixed_diskstore_diskdir.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/mixed_diskstore_diskdir.xml
index c73190f..6fcb012 100755
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/mixed_diskstore_diskdir.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/mixed_diskstore_diskdir.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <!DOCTYPE cache PUBLIC "-//GemStone Systems, Inc.//GemFire Declarative Caching 6.5//EN" "http://www.gemstone.com/dtd/cache6_5.dtd">
 <cache lock-lease="120" lock-timeout="60" search-timeout="300" is-server="false" copy-on-read="false">
   <cache-transaction-manager/>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/mixed_diskstore_diskwriteattrs.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/mixed_diskstore_diskwriteattrs.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/mixed_diskstore_diskwriteattrs.xml
index 38f75fc..6d23a22 100755
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/mixed_diskstore_diskwriteattrs.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/faultyDiskXMLsForTesting/mixed_diskstore_diskwriteattrs.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <!DOCTYPE cache PUBLIC "-//GemStone Systems, Inc.//GemFire Declarative Caching 6.5//EN" "http://www.gemstone.com/dtd/cache6_5.dtd">
 <cache lock-lease="120" lock-timeout="60" search-timeout="300" is-server="false" copy-on-read="false">
   <cache-transaction-manager/>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelJUnitTest.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelJUnitTest.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelJUnitTest.xml
index 38ad448..76f0c15 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelJUnitTest.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelJUnitTest.xml
@@ -1,4 +1,20 @@
 <?xml version="1.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.
+-->
 
 <!-- redundancylevel.xml
      Configures a region as a client region. The region's loader and writer 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlParserJUnitTest.testDTDFallbackWithNonEnglishLocal.cache.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlParserJUnitTest.testDTDFallbackWithNonEnglishLocal.cache.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlParserJUnitTest.testDTDFallbackWithNonEnglishLocal.cache.xml
index 89e93f6..7fb89e4 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlParserJUnitTest.testDTDFallbackWithNonEnglishLocal.cache.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlParserJUnitTest.testDTDFallbackWithNonEnglishLocal.cache.xml
@@ -1,4 +1,21 @@
 <?xml version="1.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.
+-->
+
 <!DOCTYPE cache PUBLIC
   "-//GemStone Systems, Inc.//GemFire Declarative Caching 6.5//EN"
   "http://www.gemstone.com/dtd/cache6_5.dtd">

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/jta/cachejta.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/jta/cachejta.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/jta/cachejta.xml
index 29273f1..d31a634 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/jta/cachejta.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/jta/cachejta.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?> 
+<!--
+  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.
+-->
+
 <!DOCTYPE cache PUBLIC
   "-//GemStone Systems, Inc.//GemFire Declarative Caching 4.1//EN"
   "http://www.gemstone.com/dtd/cache4_1.dtd">

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeNewNamed.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeNewNamed.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeNewNamed.xml
index 3055f31..e8e8d72 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeNewNamed.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeNewNamed.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <cache
     xmlns="http://schema.pivotal.io/gemfire/cache"
     xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeNewUnnamed.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeNewUnnamed.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeNewUnnamed.xml
index fb50e6b..b27039d 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeNewUnnamed.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeNewUnnamed.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <cache
     xmlns="http://schema.pivotal.io/gemfire/cache"
     xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeNewUnnamedExtension.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeNewUnnamedExtension.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeNewUnnamedExtension.xml
index 38f791d..32cf0c2 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeNewUnnamedExtension.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeNewUnnamedExtension.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <cache
     xmlns="http://schema.pivotal.io/gemfire/cache"
     xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeReplaceNamed.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeReplaceNamed.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeReplaceNamed.xml
index 4f5b1a1..69e919b 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeReplaceNamed.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeReplaceNamed.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <cache
     xmlns="http://schema.pivotal.io/gemfire/cache"
     xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeReplaceUnnamed.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeReplaceUnnamed.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeReplaceUnnamed.xml
index 4db3799..ed88234 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeReplaceUnnamed.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeReplaceUnnamed.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <cache
     xmlns="http://schema.pivotal.io/gemfire/cache"
     xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeReplaceUnnamedExtension.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeReplaceUnnamedExtension.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeReplaceUnnamedExtension.xml
index a3238d8..a2cecf0 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeReplaceUnnamedExtension.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeReplaceUnnamedExtension.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <cache
     xmlns="http://schema.pivotal.io/gemfire/cache"
     xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testDeleteNodeNamed.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testDeleteNodeNamed.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testDeleteNodeNamed.xml
index f9d548b..3f74bcb 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testDeleteNodeNamed.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testDeleteNodeNamed.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <cache
     xmlns="http://schema.pivotal.io/gemfire/cache"
     xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testDeleteNodeUnnamed.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testDeleteNodeUnnamed.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testDeleteNodeUnnamed.xml
index b325fbf..85d6acc 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testDeleteNodeUnnamed.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testDeleteNodeUnnamed.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <cache
     xmlns="http://schema.pivotal.io/gemfire/cache"
     xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testDeleteNodeUnnamedExtension.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testDeleteNodeUnnamedExtension.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testDeleteNodeUnnamedExtension.xml
index 53d8d6d..07afbce 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testDeleteNodeUnnamedExtension.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testDeleteNodeUnnamedExtension.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <cache
     xmlns="http://schema.pivotal.io/gemfire/cache"
     xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.xml
index bdefa30..bdf18e4 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <cache
     xmlns="http://schema.pivotal.io/gemfire/cache"
     xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapAttribute.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapAttribute.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapAttribute.xml
index 832d756..257eb8e 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapAttribute.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapAttribute.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <cache
     xmlns="http://schema.pivotal.io/gemfire/cache"
     xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapEmptyAttribute.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapEmptyAttribute.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapEmptyAttribute.xml
index 05843bf..968ca6a 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapEmptyAttribute.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapEmptyAttribute.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <cache
     xmlns="http://schema.pivotal.io/gemfire/cache"
     xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapMapOfStringListOfStringAttribute.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapMapOfStringListOfStringAttribute.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapMapOfStringListOfStringAttribute.xml
index 87b3cac..bf2b2d8 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapMapOfStringListOfStringAttribute.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapMapOfStringListOfStringAttribute.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <cache
     xmlns="http://schema.pivotal.io/gemfire/cache"
     xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapNullAttribute.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapNullAttribute.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapNullAttribute.xml
index 95d11b3..d32cb2c 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapNullAttribute.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapNullAttribute.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <cache
     xmlns="http://schema.pivotal.io/gemfire/cache"
     xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testQuerySingleElement.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testQuerySingleElement.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testQuerySingleElement.xml
index 76b851e..d5f41d6 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testQuerySingleElement.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testQuerySingleElement.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <cache
     xmlns="http://schema.pivotal.io/gemfire/cache"
     xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/jta/cachejta.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/jta/cachejta.xml b/gemfire-core/src/test/resources/jta/cachejta.xml
index 87727fc..9a36ee6 100644
--- a/gemfire-core/src/test/resources/jta/cachejta.xml
+++ b/gemfire-core/src/test/resources/jta/cachejta.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?> 
+<!--
+  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.
+-->
+
 <!DOCTYPE cache PUBLIC
   "-//GemStone Systems, Inc.//GemFire Declarative Caching 6.0//EN"
   "http://www.gemstone.com/dtd/cache6_0.dtd">

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/spring/spring-gemfire-context.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/spring/spring-gemfire-context.xml b/gemfire-core/src/test/resources/spring/spring-gemfire-context.xml
index bb40d1e..a728493 100644
--- a/gemfire-core/src/test/resources/spring/spring-gemfire-context.xml
+++ b/gemfire-core/src/test/resources/spring/spring-gemfire-context.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="utf-8"?>
+<!--
+  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.
+-->
+
 <beans xmlns="http://www.springframework.org/schema/beans"
        xmlns:gfe="http://www.springframework.org/schema/gemfire"
        xmlns:util="http://www.springframework.org/schema/util"

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-lucene/build.gradle
----------------------------------------------------------------------
diff --git a/gemfire-lucene/build.gradle b/gemfire-lucene/build.gradle
index ff47748..6218e55 100644
--- a/gemfire-lucene/build.gradle
+++ b/gemfire-lucene/build.gradle
@@ -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.
+ */
+
 dependencies {
     compile project(':gemfire-core')
     compile project(':gemfire-common')

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlParserIntegrationJUnitTest.createIndex.cache.xml
----------------------------------------------------------------------
diff --git a/gemfire-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlParserIntegrationJUnitTest.createIndex.cache.xml b/gemfire-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlParserIntegrationJUnitTest.createIndex.cache.xml
index 42e4e84..47f3250 100644
--- a/gemfire-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlParserIntegrationJUnitTest.createIndex.cache.xml
+++ b/gemfire-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlParserIntegrationJUnitTest.createIndex.cache.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <cache
     xmlns="http://schema.pivotal.io/gemfire/cache"
     xmlns:lucene="http://geode.apache.org/schema/lucene"

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlParserIntegrationJUnitTest.parseIndex.cache.xml
----------------------------------------------------------------------
diff --git a/gemfire-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlParserIntegrationJUnitTest.parseIndex.cache.xml b/gemfire-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlParserIntegrationJUnitTest.parseIndex.cache.xml
index 42e4e84..47f3250 100644
--- a/gemfire-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlParserIntegrationJUnitTest.parseIndex.cache.xml
+++ b/gemfire-lucene/src/test/resources/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexXmlParserIntegrationJUnitTest.parseIndex.cache.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <cache
     xmlns="http://schema.pivotal.io/gemfire/cache"
     xmlns:lucene="http://geode.apache.org/schema/lucene"

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-rebalancer/build.gradle
----------------------------------------------------------------------
diff --git a/gemfire-rebalancer/build.gradle b/gemfire-rebalancer/build.gradle
index cbb6803..1f9bff8 100644
--- a/gemfire-rebalancer/build.gradle
+++ b/gemfire-rebalancer/build.gradle
@@ -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.
+ */
+
 dependencies {
     provided project(':gemfire-common')
     provided project(':gemfire-core')

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-spark-connector/gemfire-spark-connector/src/it/resources/test-regions.xml
----------------------------------------------------------------------
diff --git a/gemfire-spark-connector/gemfire-spark-connector/src/it/resources/test-regions.xml b/gemfire-spark-connector/gemfire-spark-connector/src/it/resources/test-regions.xml
index 9c59d7d..79893d6 100644
--- a/gemfire-spark-connector/gemfire-spark-connector/src/it/resources/test-regions.xml
+++ b/gemfire-spark-connector/gemfire-spark-connector/src/it/resources/test-regions.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <!DOCTYPE cache PUBLIC
   "-//GemStone Systems, Inc.//GemFire Declarative Caching 6.5//EN"
   "http://www.gemstone.com/dtd/cache6_5.dtd" >

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-spark-connector/gemfire-spark-connector/src/it/resources/test-retrieve-regions.xml
----------------------------------------------------------------------
diff --git a/gemfire-spark-connector/gemfire-spark-connector/src/it/resources/test-retrieve-regions.xml b/gemfire-spark-connector/gemfire-spark-connector/src/it/resources/test-retrieve-regions.xml
index 1608751..3023959 100644
--- a/gemfire-spark-connector/gemfire-spark-connector/src/it/resources/test-retrieve-regions.xml
+++ b/gemfire-spark-connector/gemfire-spark-connector/src/it/resources/test-retrieve-regions.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <!DOCTYPE cache PUBLIC
   "-//GemStone Systems, Inc.//GemFire Declarative Caching 6.5//EN"
   "http://www.gemstone.com/dtd/cache6_5.dtd" >

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-web-api/build.gradle
----------------------------------------------------------------------
diff --git a/gemfire-web-api/build.gradle b/gemfire-web-api/build.gradle
index 476872f..7c35ecd 100755
--- a/gemfire-web-api/build.gradle
+++ b/gemfire-web-api/build.gradle
@@ -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.
+ */
+
 apply plugin: 'war'
 
 dependencies {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-web/build.gradle
----------------------------------------------------------------------
diff --git a/gemfire-web/build.gradle b/gemfire-web/build.gradle
index c438de4..1c926c0 100755
--- a/gemfire-web/build.gradle
+++ b/gemfire-web/build.gradle
@@ -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.
+ */
+
 apply plugin: 'war'
 
 dependencies {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gradle.properties
----------------------------------------------------------------------
diff --git a/gradle.properties b/gradle.properties
index 03a07d8..c6ca51f 100755
--- a/gradle.properties
+++ b/gradle.properties
@@ -1,3 +1,17 @@
+# 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.
 versionNumber = 1.0.0-incubating
 releaseType = SNAPSHOT
 



[04/48] incubator-geode git commit: GEODE-18: Added missing source headers

Posted by kl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gradle/dependency-versions.properties
----------------------------------------------------------------------
diff --git a/gradle/dependency-versions.properties b/gradle/dependency-versions.properties
index 3e6b6a5..d02110c 100644
--- a/gradle/dependency-versions.properties
+++ b/gradle/dependency-versions.properties
@@ -1,3 +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.
+
 # Buildscript Dependencies
 gradle-maven-publish-auth.version = 2.0.1
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gradle/rat.gradle
----------------------------------------------------------------------
diff --git a/gradle/rat.gradle b/gradle/rat.gradle
index ab2da4c..361aba9 100644
--- a/gradle/rat.gradle
+++ b/gradle/rat.gradle
@@ -59,6 +59,9 @@ rat {
     'gemfire-spark-connector/project/build.properties',
     '**/log4j2*.xml',
     'gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/domain/CacheElementJUnitTest.xml',
+
+    // TODO - go through all the gemfire-site files!!
+    '**/gemfire-site/**',
  
     // ANTLR generated files
     'gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/parse/OQLLexer.java',

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/settings.gradle
----------------------------------------------------------------------
diff --git a/settings.gradle b/settings.gradle
index 50b7e40..c57c8df 100644
--- a/settings.gradle
+++ b/settings.gradle
@@ -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.
+ */
 rootProject.name = 'gemfire'
 
 include 'gemfire-common'


[20/48] incubator-geode git commit: added null check in checkAddressesForUUIDs method

Posted by kl...@apache.org.
added null check in checkAddressesForUUIDs method


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

Branch: refs/heads/feature/GEODE-217
Commit: 949507ffad3b8a6237474f7ab302a118f06f6218
Parents: 7dfce7c
Author: Hitesh Khamesra <hi...@yahoo.com>
Authored: Thu Dec 10 13:58:23 2015 -0800
Committer: Hitesh Khamesra <hi...@yahoo.com>
Committed: Thu Dec 10 15:31:15 2015 -0800

----------------------------------------------------------------------
 .../membership/gms/mgr/GMSMembershipManager.java   | 17 ++++++++++-------
 1 file changed, 10 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/949507ff/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 93c14e2..e5835c4 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
@@ -1995,13 +1995,16 @@ public class GMSMembershipManager implements MembershipManager, Manager
   
   void checkAddressesForUUIDs(InternalDistributedMember[] addresses) {
     for (int i=0; i<addresses.length; i++) {
-      GMSMember id = (GMSMember)addresses[i].getNetMember();
-      if (!id.hasUUID()) {
-        latestViewLock.readLock().lock();
-        try {
-          addresses[i] = latestView.getCanonicalID(addresses[i]);
-        } finally {
-          latestViewLock.readLock().unlock();
+      InternalDistributedMember m = addresses[i];
+      if(m != null) {
+        GMSMember id = (GMSMember)m.getNetMember();
+        if (!id.hasUUID()) {
+          latestViewLock.readLock().lock();
+          try {
+            addresses[i] = latestView.getCanonicalID(addresses[i]);
+          } finally {
+            latestViewLock.readLock().unlock();
+          }
         }
       }
     }


[02/48] incubator-geode git commit: GEODE-608: Expand the exclude directives

Posted by kl...@apache.org.
GEODE-608: Expand the exclude directives

GEODE-18 has a RAT excludes file attached. This change adds the
excludes in that file to the gradle build.


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

Branch: refs/heads/feature/GEODE-217
Commit: fddd33f3cbced1369edd95caedd7f7b98ac509cc
Parents: f133ff1
Author: Anthony Baker <ab...@pivotal.io>
Authored: Sat Nov 28 11:33:42 2015 -0800
Committer: Anthony Baker <ab...@pivotal.io>
Committed: Thu Dec 10 09:44:56 2015 -0800

----------------------------------------------------------------------
 build.gradle | 136 +++++++++++++++++++++++++++++++++++++++++++++++++++++-
 1 file changed, 134 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fddd33f3/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index 6f5c2a3..4d0216b 100755
--- a/build.gradle
+++ b/build.gradle
@@ -15,11 +15,22 @@ apply plugin: "org.nosphere.apache.rat"
 
 rat {
   excludes = [
+    // git
     '.git/**',
     '**/.gitignore',
+    
+    // gradle
     '**/.gradle/**',
-    '.gradle',
+    '**/build.gradle',
+    'gradlew',
+    'gradlew.bat',
+    'gradle.properties',
+    'settings.gradle',
+    'gradle/dependency-versions.properties',
+    'gradle/wrapper/gradle-wrapper.properties',
     '**/build/**',
+    
+    // IDE
     '**/.project',
     '**/.classpath',
     '**/.settings/**',
@@ -27,7 +38,128 @@ rat {
     '*.iml',
     '.idea/**',
 
-    '**/doc-files/*.fig'
+    // text files
+    '**/*.fig',
+    '**/*.txt',
+    '**/*.md',
+    '**/*.json',
+    '**/*.tx0',
+    '**/*.txo',
+    
+    // binary files
+    '**/*.cer',
+    '**/*.gfs',
+    '**/keystore',
+    '**/*.ser',
+    '**/*.xls',
+    
+    // other text files
+    'gemfire-assembly/src/main/dist/bin/gfsh',
+    'gemfire-assembly/src/main/dist/bin/gfsh-completion.bash',
+    'gemfire-assembly/src/main/dist/bin/gfsh.bat',
+    'gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/parse/fixantlr.sh',
+    'gemfire-spark-connector/project/plugins.sbt',
+    'gemfire-spark-connector/project/build.properties',
+    'gemfire-web-api/src/main/webapp/docs/o2c.html',
+    'gemfire-web-api/src/main/webapp/docs/index.html',
+    
+    // XML
+    'etc/eclipseFormatterProfile.xml',
+    'etc/intellijIdeaCodeStyle.xml',
+    'log4j2*.xml',
+    '**/example*cache.xml',
+    'gemfire-core/src/test/resources/**/*JUnitTest*.xml',
+    'gemfire-core/src/test/resources/**/IndexCreation.xml',
+    'gemfire-core/src/test/resources/**/PRIndexCreation.xml',
+    'gemfire-core/src/test/resources/**/PartitionRegionCacheExample*.xml',
+    'gemfire-core/src/test/resources/**/attributesUnordered.xml',
+    'gemfire-core/src/test/resources/**/bad*.xml',
+    'gemfire-core/src/test/resources/**/bug44710.xml',
+    'gemfire-core/src/test/resources/**/cachejta.xml',
+    'gemfire-core/src/test/resources/**/cachequeryindex*.xml',
+    'gemfire-core/src/test/resources/**/callback*.xml',
+    'gemfire-core/src/test/resources/**/coLocation*.xml',
+    'gemfire-core/src/test/resources/**/ewtest.xml',
+    'gemfire-core/src/test/resources/**/incorrect*.xml',
+    'gemfire-core/src/test/resources/**/index-creation-*.xml',
+    'gemfire-core/src/test/resources/**/index-recovery-overflow.xml',
+    'gemfire-core/src/test/resources/**/loaderNotLoader.xml',
+    'gemfire-core/src/test/resources/**/malformed.xml',
+    'gemfire-core/src/test/resources/**/mixed_diskstore_disk*.xml',
+    'gemfire-core/src/test/resources/**/namedAttributes.xml',
+    'gemfire-core/src/test/resources/**/partitioned*.xml',
+    'gemfire-core/src/test/resources/**/same*.xml',
+    'gemfire-core/src/test/resources/**/spring-gemfire-context.xml',
+    'gemfire-core/src/test/resources/**/test*.xml',
+    'gemfire-core/src/test/resources/**/unknownNamedAttributes.xml',
+    
+    // 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',
+    'gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/parse/oql.g',
+    
+    // Service META-INF
+    '**/META-INF/services/org.xml.sax.ext.EntityResolver2',
+    '**/META-INF/services/com.gemstone.gemfire.internal.cache.CacheService',
+    '**/META-INF/services/com.gemstone.gemfire.internal.cache.xmlcache.XmlParser',
+    '**/META-INF/services/org.springframework.shell.core.CommandMarker',
+
+    // --- Other Licenses ---
+    
+    // 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',
+
+    // Public Domain http://meyerweb.com/eric/tools/css/reset/
+    'gemfire-web-api/src/main/webapp/docs/css/reset.css',
+
+    // Public Domain - http://creativecommons.org/licenses/publicdomain
+    'SynchronousQueueNoSpin.java',
+
+    // JSON License - permissive, used for Good, not Evil
+    'gemfire-json/src/main/java/org/json/CDL.java',
+    'gemfire-json/src/main/java/org/json/Cookie.java',
+    'gemfire-json/src/main/java/org/json/CookieList.java',
+    'gemfire-json/src/main/java/org/json/CDL.java',
+    'gemfire-json/src/main/java/org/json/Cookie.java',
+    'gemfire-json/src/main/java/org/json/CookieList.java',
+    'gemfire-json/src/main/java/org/json/HTTP.java',
+    'gemfire-json/src/main/java/org/json/HTTPTokener.java',
+    'gemfire-json/src/main/java/org/json/JSONArray.java',
+    'gemfire-json/src/main/java/org/json/JSONException.java',
+    'gemfire-json/src/main/java/org/json/JSONML.java',
+    'gemfire-json/src/main/java/org/json/JSONObject.java',
+    'gemfire-json/src/main/java/org/json/JSONString.java',
+    'gemfire-json/src/main/java/org/json/JSONStringer.java',
+    'gemfire-json/src/main/java/org/json/JSONTokener.java',
+    'gemfire-json/src/main/java/org/json/JSONWriter.java',
+    'gemfire-json/src/main/java/org/json/XML.java',
+    'gemfire-json/src/main/java/org/json/XMLTokener.java',
+
+    // MIT License
+    'gemfire-web-api/src/main/webapp/docs/lib/backbone-min.js',
+    'gemfire-web-api/src/main/webapp/docs/lib/jquery-1.8.0.min.js',
+    'gemfire-web-api/src/main/webapp/docs/lib/jquery.ba-bbq.min.js',
+    'gemfire-web-api/src/main/webapp/docs/lib/jquery.slideto.min.js',
+    'gemfire-web-api/src/main/webapp/docs/lib/jquery.wiggle.min.js',
+    'gemfire-web-api/src/main/webapp/docs/lib/underscore-min.js',
+    'gemfire-site/src/jbake/**',
+
+    // MIT or ISC
+    'gemfire-web-api/src/main/webapp/docs/lib/shred.bundle.js',
+    'gemfire-web-api/src/main/webapp/docs/lib/shred/content.js',
+
+    // BSD License
+    'gemfire-web-api/src/main/webapp/docs/lib/highlight.7.3.pack.js',
+
+    // Apache License
+    'gemfire-web-api/src/main/webapp/docs/lib/swagger-oauth.js',
+    'gemfire-web-api/src/main/webapp/docs/lib/swagger.js',
+    'gemfire-web-api/src/main/webapp/docs/css/screen.css',
+    'gemfire-web-api/src/main/webapp/docs/swagger-ui.js',
+    'gemfire-web-api/src/main/webapp/docs/swagger-ui.min.js'
   ]
 }
 


[41/48] 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 kl...@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



[25/48] incubator-geode git commit: Revert "GEODE-647: Adding some test debugging output"

Posted by kl...@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-217
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


[45/48] 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 kl...@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>


[06/48] incubator-geode git commit: GEODE-18: Added missing source headers

Posted by kl...@apache.org.
GEODE-18: Added missing source headers


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

Branch: refs/heads/feature/GEODE-217
Commit: 35394efd5221dff5c99dd7021ce1dfa0083ae828
Parents: c9f677b
Author: Anthony Baker <ab...@pivotal.io>
Authored: Wed Dec 9 14:59:29 2015 -0800
Committer: Anthony Baker <ab...@pivotal.io>
Committed: Thu Dec 10 09:44:57 2015 -0800

----------------------------------------------------------------------
 build.gradle                                    | 17 ++++++++++++++
 gemfire-assembly/build.gradle                   | 17 ++++++++++++++
 gemfire-assembly/src/main/dist/bin/gfsh         | 14 ++++++++++++
 .../src/main/dist/bin/gfsh-completion.bash      | 15 ++++++++++++
 gemfire-assembly/src/main/dist/bin/gfsh.bat     | 14 ++++++++++++
 .../SharedConfigurationEndToEndDUnitTest.java   | 21 ++++++++++++-----
 gemfire-common/build.gradle                     | 17 ++++++++++++++
 gemfire-core/build.gradle                       | 17 ++++++++++++++
 gemfire-core/src/jca/ra.xml                     | 17 +++++++++++++-
 .../client/doc-files/example-client-cache.xml   | 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 +++++++++++++
 .../cache/query/internal/parse/fixantlr.sh      | 15 ++++++++++++
 .../gemfire/cache/query/internal/parse/oql.g    | 17 ++++++++++++++
 .../internal/direct/DirectChannelListener.java  | 16 +++++++++++++
 .../tools/gfsh/app/windowsbindings.properties   | 15 ++++++++++++
 .../internal/membership/NetViewJUnitTest.java   | 16 +++++++++++++
 .../membership/gms/GMSMemberJUnitTest.java      | 16 +++++++++++++
 .../InstantiatorPropagationDUnitTest.java       | 16 +++++++++++++
 .../cache/wan/AsyncEventQueueTestBase.java      | 20 +++++++++++-----
 .../asyncqueue/AsyncEventListenerDUnitTest.java | 21 ++++++++++++-----
 .../AsyncEventListenerOffHeapDUnitTest.java     | 16 +++++++++++++
 .../AsyncEventQueueStatsDUnitTest.java          | 21 ++++++++++++-----
 .../ConcurrentAsyncEventQueueDUnitTest.java     | 24 ++++++++++++--------
 ...ncurrentAsyncEventQueueOffHeapDUnitTest.java | 16 +++++++++++++
 .../CommonParallelAsyncEventQueueDUnitTest.java | 24 ++++++++++++--------
 ...ParallelAsyncEventQueueOffHeapDUnitTest.java | 16 +++++++++++++
 .../ClientCacheFactoryJUnitTest_single_pool.xml | 17 ++++++++++++++
 .../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 ++++++++++++++
 .../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   | 16 +++++++++++++
 ...testDTDFallbackWithNonEnglishLocal.cache.xml | 17 ++++++++++++++
 .../gemstone/gemfire/internal/jta/cachejta.xml  | 17 ++++++++++++++
 ...dNewNodeJUnitTest.testAddNewNodeNewNamed.xml | 17 ++++++++++++++
 ...ewNodeJUnitTest.testAddNewNodeNewUnnamed.xml | 17 ++++++++++++++
 ...itTest.testAddNewNodeNewUnnamedExtension.xml | 17 ++++++++++++++
 ...NodeJUnitTest.testAddNewNodeReplaceNamed.xml | 17 ++++++++++++++
 ...deJUnitTest.testAddNewNodeReplaceUnnamed.xml | 17 ++++++++++++++
 ...st.testAddNewNodeReplaceUnnamedExtension.xml | 17 ++++++++++++++
 ...sAddNewNodeJUnitTest.testDeleteNodeNamed.xml | 17 ++++++++++++++
 ...ddNewNodeJUnitTest.testDeleteNodeUnnamed.xml | 17 ++++++++++++++
 ...JUnitTest.testDeleteNodeUnnamedExtension.xml | 17 ++++++++++++++
 .../utils/XmlUtilsAddNewNodeJUnitTest.xml       | 17 ++++++++++++++
 ...Test.testBuildSchemaLocationMapAttribute.xml | 17 ++++++++++++++
 ...testBuildSchemaLocationMapEmptyAttribute.xml | 17 ++++++++++++++
 ...ationMapMapOfStringListOfStringAttribute.xml | 17 ++++++++++++++
 ....testBuildSchemaLocationMapNullAttribute.xml | 17 ++++++++++++++
 ...XmlUtilsJUnitTest.testQuerySingleElement.xml | 17 ++++++++++++++
 .../src/test/resources/jta/cachejta.xml         | 17 ++++++++++++++
 .../resources/spring/spring-gemfire-context.xml | 17 ++++++++++++++
 gemfire-lucene/build.gradle                     | 17 ++++++++++++++
 ...erIntegrationJUnitTest.createIndex.cache.xml | 17 ++++++++++++++
 ...serIntegrationJUnitTest.parseIndex.cache.xml | 17 ++++++++++++++
 gemfire-rebalancer/build.gradle                 | 17 ++++++++++++++
 .../src/it/resources/test-regions.xml           | 17 ++++++++++++++
 .../src/it/resources/test-retrieve-regions.xml  | 17 ++++++++++++++
 gemfire-web-api/build.gradle                    | 17 ++++++++++++++
 gemfire-web/build.gradle                        | 17 ++++++++++++++
 gradle.properties                               | 14 ++++++++++++
 gradle/dependency-versions.properties           | 15 ++++++++++++
 gradle/rat.gradle                               |  3 +++
 settings.gradle                                 | 16 +++++++++++++
 103 files changed, 1691 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index 9042976..9c8ac44 100755
--- a/build.gradle
+++ b/build.gradle
@@ -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.
+ */
+
 buildscript {
   repositories {
     maven {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-assembly/build.gradle
----------------------------------------------------------------------
diff --git a/gemfire-assembly/build.gradle b/gemfire-assembly/build.gradle
index 514b4a1..373d77a 100755
--- a/gemfire-assembly/build.gradle
+++ b/gemfire-assembly/build.gradle
@@ -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.
+ */
+
 buildscript {
     repositories {
         mavenCentral()

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-assembly/src/main/dist/bin/gfsh
----------------------------------------------------------------------
diff --git a/gemfire-assembly/src/main/dist/bin/gfsh b/gemfire-assembly/src/main/dist/bin/gfsh
index bcb5a3d..d08e0d4 100755
--- a/gemfire-assembly/src/main/dist/bin/gfsh
+++ b/gemfire-assembly/src/main/dist/bin/gfsh
@@ -1,4 +1,18 @@
 #!/bin/bash
+# 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.
 
 #
 # Environment variables:

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-assembly/src/main/dist/bin/gfsh-completion.bash
----------------------------------------------------------------------
diff --git a/gemfire-assembly/src/main/dist/bin/gfsh-completion.bash b/gemfire-assembly/src/main/dist/bin/gfsh-completion.bash
index 6db7ef1..d81e5f3 100755
--- a/gemfire-assembly/src/main/dist/bin/gfsh-completion.bash
+++ b/gemfire-assembly/src/main/dist/bin/gfsh-completion.bash
@@ -1,4 +1,19 @@
 #!/usr/bin/env bash
+# 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.
+
 #
 # Auto completion script for GemFire's gfsh script
 #

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-assembly/src/main/dist/bin/gfsh.bat
----------------------------------------------------------------------
diff --git a/gemfire-assembly/src/main/dist/bin/gfsh.bat b/gemfire-assembly/src/main/dist/bin/gfsh.bat
index 10c2726..770e0b9 100755
--- a/gemfire-assembly/src/main/dist/bin/gfsh.bat
+++ b/gemfire-assembly/src/main/dist/bin/gfsh.bat
@@ -1,4 +1,18 @@
 @echo off
+REM Licensed to the Apache Software Foundation (ASF) under one or more
+REM contributor license agreements.  See the NOTICE file distributed with
+REM this work for additional information regarding copyright ownership.
+REM The ASF licenses this file to You under the Apache License, Version 2.0
+REM (the "License"); you may not use this file except in compliance with
+REM the License.  You may obtain a copy of the License at
+REM
+REM      http://www.apache.org/licenses/LICENSE-2.0
+REM
+REM Unless required by applicable law or agreed to in writing, software
+REM distributed under the License is distributed on an "AS IS" BASIS,
+REM WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+REM See the License for the specific language governing permissions and
+REM limitations under the License.
 
 REM
 REM Environment variables:

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-assembly/src/test/java/com/gemstone/gemfire/management/internal/configuration/SharedConfigurationEndToEndDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-assembly/src/test/java/com/gemstone/gemfire/management/internal/configuration/SharedConfigurationEndToEndDUnitTest.java b/gemfire-assembly/src/test/java/com/gemstone/gemfire/management/internal/configuration/SharedConfigurationEndToEndDUnitTest.java
index 383012e..26ada92 100644
--- a/gemfire-assembly/src/test/java/com/gemstone/gemfire/management/internal/configuration/SharedConfigurationEndToEndDUnitTest.java
+++ b/gemfire-assembly/src/test/java/com/gemstone/gemfire/management/internal/configuration/SharedConfigurationEndToEndDUnitTest.java
@@ -1,9 +1,18 @@
-/*=========================================================================
- * 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.
- *=========================================================================
+/*
+ * 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.management.internal.configuration;
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-common/build.gradle
----------------------------------------------------------------------
diff --git a/gemfire-common/build.gradle b/gemfire-common/build.gradle
index aa8adcb..e28ba44 100755
--- a/gemfire-common/build.gradle
+++ b/gemfire-common/build.gradle
@@ -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.
+ */
+
 dependencies {
   provided project(path: ':gemfire-junit', configuration: 'testOutput')
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/build.gradle
----------------------------------------------------------------------
diff --git a/gemfire-core/build.gradle b/gemfire-core/build.gradle
index dd3b765..19aeee1 100755
--- a/gemfire-core/build.gradle
+++ b/gemfire-core/build.gradle
@@ -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.
+ */
+
 sourceSets {
   jca {
     compileClasspath += configurations.compile

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/jca/ra.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/jca/ra.xml b/gemfire-core/src/jca/ra.xml
index 0638ed0..78b4579 100644
--- a/gemfire-core/src/jca/ra.xml
+++ b/gemfire-core/src/jca/ra.xml
@@ -1,5 +1,20 @@
-
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
 
 <!DOCTYPE connector PUBLIC '-//Sun Microsystems, Inc.//DTD Connector 1.0//EN' 'http://java.sun.com/j2ee/dtds/connector_1_0.dtd'>
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/doc-files/example-client-cache.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/doc-files/example-client-cache.xml b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/doc-files/example-client-cache.xml
index edff479..bddeb9f 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/doc-files/example-client-cache.xml
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/doc-files/example-client-cache.xml
@@ -1,4 +1,20 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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 sample client GemFire declarative caching XML File -->
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/main/java/com/gemstone/gemfire/cache/doc-files/example-cache.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/doc-files/example-cache.xml b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/doc-files/example-cache.xml
index 6f99a58..0061b5c 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/doc-files/example-cache.xml
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/doc-files/example-cache.xml
@@ -1,4 +1,20 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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 sample GemFire declarative caching XML File -->
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/main/java/com/gemstone/gemfire/cache/doc-files/example2-cache.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/doc-files/example2-cache.xml b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/doc-files/example2-cache.xml
index 79d98b0..b65c285 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/doc-files/example2-cache.xml
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/doc-files/example2-cache.xml
@@ -1,4 +1,21 @@
 <?xml version="1.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.
+-->
+
 <cache
     xmlns="http://schema.pivotal.io/gemfire/cache"
     xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/main/java/com/gemstone/gemfire/cache/doc-files/example3-cache.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/doc-files/example3-cache.xml b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/doc-files/example3-cache.xml
index 4d3a97a..803e72e 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/doc-files/example3-cache.xml
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/doc-files/example3-cache.xml
@@ -1,4 +1,20 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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 sample GemFire declarative caching XML File that demonstrates
      the "named region attributes" feature.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/parse/fixantlr.sh
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/parse/fixantlr.sh b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/parse/fixantlr.sh
index b514b5f..533f5fa 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/parse/fixantlr.sh
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/parse/fixantlr.sh
@@ -1,3 +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.
+
 set -e
 sed -e 's/(char)//' OQLLexer.java >x.java
 mv x.java OQLLexer.java

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/parse/oql.g
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/parse/oql.g b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/parse/oql.g
index e18e480..caf0ec0 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/parse/oql.g
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/parse/oql.g
@@ -1,4 +1,21 @@
 /*
+ * 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.
+ */
+
+/*
 **  oql.g
 **
 ** Built with Antlr 2.7.4

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannelListener.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannelListener.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannelListener.java
index 28f481b..a97676d 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannelListener.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannelListener.java
@@ -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.
+ */
 package com.gemstone.gemfire.distributed.internal.direct;
 
 import com.gemstone.gemfire.distributed.internal.DistributionManager;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/main/resources/com/gemstone/gemfire/internal/tools/gfsh/app/windowsbindings.properties
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/resources/com/gemstone/gemfire/internal/tools/gfsh/app/windowsbindings.properties b/gemfire-core/src/main/resources/com/gemstone/gemfire/internal/tools/gfsh/app/windowsbindings.properties
index 1fc6136..4291d0b 100644
--- a/gemfire-core/src/main/resources/com/gemstone/gemfire/internal/tools/gfsh/app/windowsbindings.properties
+++ b/gemfire-core/src/main/resources/com/gemstone/gemfire/internal/tools/gfsh/app/windowsbindings.properties
@@ -1,3 +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.
+
 # This is a file that comes with jline distro with key bindings for 'ESC' key 
 # commented. For associated bug see: 
 # http://sourceforge.net/tracker/index.php?func=detail&aid=1982039&group_id=64033&atid=506056

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/NetViewJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/NetViewJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/NetViewJUnitTest.java
index 9e39d0f..f21ac4c 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/NetViewJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/NetViewJUnitTest.java
@@ -30,6 +30,22 @@ import com.gemstone.gemfire.distributed.internal.membership.NetView;
 import com.gemstone.gemfire.internal.SocketCreator;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
+/*
+ * 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.
+ */
 @Category(UnitTest.class)
 public class NetViewJUnitTest {
   List<InternalDistributedMember> members;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMemberJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMemberJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMemberJUnitTest.java
index 0b75d3d..606ae1a 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMemberJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMemberJUnitTest.java
@@ -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.
+ */
 package com.gemstone.gemfire.distributed.internal.membership.gms;
 
 import static org.mockito.Mockito.mock;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InstantiatorPropagationDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InstantiatorPropagationDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InstantiatorPropagationDUnitTest.java
index 063112a..7c5cf53 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InstantiatorPropagationDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InstantiatorPropagationDUnitTest.java
@@ -41,6 +41,22 @@ import dunit.DistributedTestCase;
 import dunit.Host;
 import dunit.VM;
 
+/*
+ * 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.
+ */
 public class InstantiatorPropagationDUnitTest extends DistributedTestCase {
   private static Cache cache = null;
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/AsyncEventQueueTestBase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/AsyncEventQueueTestBase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/AsyncEventQueueTestBase.java
index a800118..ec38649 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/AsyncEventQueueTestBase.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/AsyncEventQueueTestBase.java
@@ -1,10 +1,18 @@
 /*
- * =========================================================================
- * 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.
- * =========================================================================
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 package com.gemstone.gemfire.internal.cache.wan;
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventListenerDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventListenerDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventListenerDUnitTest.java
index 1eafbb0..e696248 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventListenerDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventListenerDUnitTest.java
@@ -1,9 +1,18 @@
-/*=========================================================================
- * 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.
- *=========================================================================
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 package com.gemstone.gemfire.internal.cache.wan.asyncqueue;
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventListenerOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventListenerOffHeapDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventListenerOffHeapDUnitTest.java
index b050ef5..7feec9a 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventListenerOffHeapDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventListenerOffHeapDUnitTest.java
@@ -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.
+ */
 package com.gemstone.gemfire.internal.cache.wan.asyncqueue;
 
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventQueueStatsDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventQueueStatsDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventQueueStatsDUnitTest.java
index cf4a184..30123a3 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventQueueStatsDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventQueueStatsDUnitTest.java
@@ -1,9 +1,18 @@
-/*=========================================================================
- * 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.
- *=========================================================================
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 package com.gemstone.gemfire.internal.cache.wan.asyncqueue;
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/concurrent/ConcurrentAsyncEventQueueDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/concurrent/ConcurrentAsyncEventQueueDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/concurrent/ConcurrentAsyncEventQueueDUnitTest.java
index 2fb7496..fdcc6f6 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/concurrent/ConcurrentAsyncEventQueueDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/concurrent/ConcurrentAsyncEventQueueDUnitTest.java
@@ -1,12 +1,18 @@
-/*=========================================================================
- * 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.
- *=========================================================================
- */
-/**
- * 
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 package com.gemstone.gemfire.internal.cache.wan.concurrent;
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/concurrent/ConcurrentAsyncEventQueueOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/concurrent/ConcurrentAsyncEventQueueOffHeapDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/concurrent/ConcurrentAsyncEventQueueOffHeapDUnitTest.java
index 41eb22d..ac6472e 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/concurrent/ConcurrentAsyncEventQueueOffHeapDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/concurrent/ConcurrentAsyncEventQueueOffHeapDUnitTest.java
@@ -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.
+ */
 package com.gemstone.gemfire.internal.cache.wan.concurrent;
 
 @SuppressWarnings("serial")

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/misc/CommonParallelAsyncEventQueueDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/misc/CommonParallelAsyncEventQueueDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/misc/CommonParallelAsyncEventQueueDUnitTest.java
index 425d1a6..5b9d3bd 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/misc/CommonParallelAsyncEventQueueDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/misc/CommonParallelAsyncEventQueueDUnitTest.java
@@ -1,12 +1,18 @@
-/*=========================================================================
- * 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.
- *=========================================================================
- */
-/**
- * 
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 package com.gemstone.gemfire.internal.cache.wan.misc;
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/misc/CommonParallelAsyncEventQueueOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/misc/CommonParallelAsyncEventQueueOffHeapDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/misc/CommonParallelAsyncEventQueueOffHeapDUnitTest.java
index 8ab77b9..bceae58 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/misc/CommonParallelAsyncEventQueueOffHeapDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/misc/CommonParallelAsyncEventQueueOffHeapDUnitTest.java
@@ -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.
+ */
 package com.gemstone.gemfire.internal.cache.wan.misc;
 
 @SuppressWarnings("serial")

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/ClientCacheFactoryJUnitTest_single_pool.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/ClientCacheFactoryJUnitTest_single_pool.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/ClientCacheFactoryJUnitTest_single_pool.xml
index 0bef14f..92454c4 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/ClientCacheFactoryJUnitTest_single_pool.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/client/ClientCacheFactoryJUnitTest_single_pool.xml
@@ -1,4 +1,21 @@
 <?xml version="1.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.
+-->
+
 <!-- MultiuserSecurityClient.xml
      Configures a region as a client region in a Security cache. The 
      region's pool connects to a server listening on port 40404.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/query/dunit/IndexCreation.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/query/dunit/IndexCreation.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/query/dunit/IndexCreation.xml
index 894a762..aa46ce8 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/query/dunit/IndexCreation.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/query/dunit/IndexCreation.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <!DOCTYPE cache PUBLIC
   "-//GemStone Systems, Inc.//GemFire Declarative Cache 7.0//EN" "http://www.gemstone.com/dtd/cache7_0.dtd">
   <cache>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/query/functional/index-creation-with-eviction.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/query/functional/index-creation-with-eviction.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/query/functional/index-creation-with-eviction.xml
index 4160b8d..861bb23 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/query/functional/index-creation-with-eviction.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/query/functional/index-creation-with-eviction.xml
@@ -1,4 +1,21 @@
 <?xml version="1.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.
+-->
+
 <!DOCTYPE cache PUBLIC
     "-//GemStone Systems, Inc.//GemFire Declarative Caching 6.6//EN"
     "http://www.gemstone.com/dtd/cache6_6.dtd">

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/query/functional/index-creation-without-eviction.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/query/functional/index-creation-without-eviction.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/query/functional/index-creation-without-eviction.xml
index cc4edbe..f8bdc78 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/query/functional/index-creation-without-eviction.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/query/functional/index-creation-without-eviction.xml
@@ -1,4 +1,21 @@
 <?xml version="1.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.
+-->
+
 <!DOCTYPE cache PUBLIC
     "-//GemStone Systems, Inc.//GemFire Declarative Caching 6.6//EN"
     "http://www.gemstone.com/dtd/cache6_6.dtd">

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/query/functional/index-recovery-overflow.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/query/functional/index-recovery-overflow.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/query/functional/index-recovery-overflow.xml
index 4476e9b..4ca5e2f 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/query/functional/index-recovery-overflow.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/query/functional/index-recovery-overflow.xml
@@ -1,4 +1,21 @@
 <?xml version="1.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.
+-->
+
 <!DOCTYPE cache PUBLIC
     "-//GemStone Systems, Inc.//GemFire Declarative Caching 6.6//EN"
     "http://www.gemstone.com/dtd/cache6_6.dtd">

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/query/internal/index/cachequeryindex.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/query/internal/index/cachequeryindex.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/query/internal/index/cachequeryindex.xml
index 95067a9..4a97ba1 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/query/internal/index/cachequeryindex.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/query/internal/index/cachequeryindex.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <!DOCTYPE cache PUBLIC "-//GemStone Systems, Inc.//GemFire Declarative Caching 6.6//EN" "http://www.gemstone.com/dtd/cache6_6.dtd">
 <cache lock-lease="120" lock-timeout="60" search-timeout="300">
   <vm-root-region name="root">

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/query/internal/index/cachequeryindexwitherror.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/query/internal/index/cachequeryindexwitherror.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/query/internal/index/cachequeryindexwitherror.xml
index c2f20fc..f39c330 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/query/internal/index/cachequeryindexwitherror.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/query/internal/index/cachequeryindexwitherror.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <!DOCTYPE cache PUBLIC "-//GemStone Systems, Inc.//GemFire Declarative Caching 6.6//EN" "http://www.gemstone.com/dtd/cache6_6.dtd">
 <cache lock-lease="120" lock-timeout="60" search-timeout="300">
   <vm-root-region name="root">

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/query/partitioned/PRIndexCreation.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/query/partitioned/PRIndexCreation.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/query/partitioned/PRIndexCreation.xml
index 313158d..5d0ba46 100755
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/query/partitioned/PRIndexCreation.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache/query/partitioned/PRIndexCreation.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <!DOCTYPE cache PUBLIC "-//GemStone Systems, Inc.//GemFire Declarative Caching 5.1//EN" "http://www.gemstone.com/dtd/cache5_1.dtd">
 <cache lock-lease="120" lock-timeout="60" search-timeout="300" is-server="false" copy-on-read="false">
   <cache-transaction-manager/>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/attributesUnordered.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/attributesUnordered.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/attributesUnordered.xml
index f3c39af..aa8c3f4 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/attributesUnordered.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/attributesUnordered.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <!DOCTYPE cache PUBLIC
     "-//GemStone Systems, Inc.//GemFire Declarative Caching 4.0//EN"
     "http://www.gemstone.com/dtd/cache4_0.dtd">

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/badFloat.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/badFloat.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/badFloat.xml
index c0f3957..883216c 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/badFloat.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/badFloat.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <!DOCTYPE cache PUBLIC
     "-//GemStone Systems, Inc.//GemFire Declarative Caching 4.0//EN"
     "http://www.gemstone.com/dtd/cache4_0.dtd">

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/badInt.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/badInt.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/badInt.xml
index 3544290..17f9aa1 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/badInt.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/badInt.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <!DOCTYPE cache PUBLIC
     "-//GemStone Systems, Inc.//GemFire Declarative Caching 4.0//EN"
     "http://www.gemstone.com/dtd/cache4_0.dtd">

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/badKeyConstraintClass.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/badKeyConstraintClass.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/badKeyConstraintClass.xml
index 91e2bc4..9aa8cb3 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/badKeyConstraintClass.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/badKeyConstraintClass.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <!DOCTYPE cache PUBLIC
     "-//GemStone Systems, Inc.//GemFire Declarative Caching 4.0//EN"
     "http://www.gemstone.com/dtd/cache4_0.dtd">

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/badScope.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/badScope.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/badScope.xml
index ec683e5..0744a72 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/badScope.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/badScope.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <!DOCTYPE cache PUBLIC
     "-//GemStone Systems, Inc.//GemFire Declarative Caching 4.0//EN"
     "http://www.gemstone.com/dtd/cache4_0.dtd">

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/bug44710.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/bug44710.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/bug44710.xml
index ed6ea67..fa94d5d 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/bug44710.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/bug44710.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <!DOCTYPE cache PUBLIC "-//GemStone Systems, Inc.//GemFire Declarative Cache 7.0//EN" "http://www.gemstone.com/dtd/cache7_0.dtd">
 <cache>
   <region name="r1">

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/callbackNotDeclarable.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/callbackNotDeclarable.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/callbackNotDeclarable.xml
index 7ecaa3f..eccaf7c 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/callbackNotDeclarable.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/callbackNotDeclarable.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <!DOCTYPE cache PUBLIC
     "-//GemStone Systems, Inc.//GemFire Declarative Caching 4.0//EN"
     "http://www.gemstone.com/dtd/cache4_0.dtd">

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/callbackWithException.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/callbackWithException.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/callbackWithException.xml
index 1a41012..964a497 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/callbackWithException.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/callbackWithException.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <!DOCTYPE cache PUBLIC
     "-//GemStone Systems, Inc.//GemFire Declarative Caching 4.0//EN"
     "http://www.gemstone.com/dtd/cache4_0.dtd">

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/coLocation.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/coLocation.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/coLocation.xml
index f57e6ad..6c23d5f 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/coLocation.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/coLocation.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <!DOCTYPE cache PUBLIC "-//GemStone Systems, Inc.//GemFire Declarative Cache 5.8//EN" "http://www.gemstone.com/dtd/cache5_8.dtd">
 <cache>
   <region name="Customer">

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/coLocation3.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/coLocation3.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/coLocation3.xml
index ed42012..20fae74 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/coLocation3.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/coLocation3.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
 <!DOCTYPE cache PUBLIC "-//GemStone Systems, Inc.//GemFire Declarative Cache 7.0//EN" "http://www.gemstone.com/dtd/cache7_0.dtd">
 <cache>
   <region-attributes id="ORDER_ATTS" data-policy="partition">

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/ewtest.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/ewtest.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/ewtest.xml
index 7b53fae..1f40036 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/ewtest.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/ewtest.xml
@@ -1,4 +1,21 @@
 <?xml version="1.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.
+-->
+
 <!DOCTYPE cache PUBLIC "-//GemStone Systems, Inc.//GemFire Declarative Caching 6.5//EN" "http://www.gemstone.com/dtd/cache6_5.dtd">
 <cache lock-lease="120" lock-timeout="60" search-timeout="300">
 	<!-- Disk store declarations -->

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/examples_3_0/example-cache.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/examples_3_0/example-cache.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/examples_3_0/example-cache.xml
index aabdc94..f7bdc8e 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/examples_3_0/example-cache.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/examples_3_0/example-cache.xml
@@ -1,4 +1,20 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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 sample GemFire declarative caching XML File -->
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35394efd/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/examples_4_0/example-cache.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/examples_4_0/example-cache.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/examples_4_0/example-cache.xml
index 82cecf7..4b21e8f 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/examples_4_0/example-cache.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/cache30/examples_4_0/example-cache.xml
@@ -1,4 +1,20 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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 sample GemFire declarative caching XML File -->
 



[21/48] incubator-geode git commit: GEODE-608: Fix RAT build failures

Posted by kl...@apache.org.
GEODE-608: Fix RAT build failures

Add a license header to DistributedMulticastRegionDUnitTest.  Remove
license headers from XmlUtils*JUnitTest.xml. The first element in
these files is expected to be <cache/> element. Add an exclude to
igore these.


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

Branch: refs/heads/feature/GEODE-217
Commit: a84780e32d18057d004457dfa85b1fab9d212a60
Parents: 949507f
Author: Anthony Baker <ab...@pivotal.io>
Authored: Fri Dec 11 07:08:31 2015 -0800
Committer: Anthony Baker <ab...@pivotal.io>
Committed: Fri Dec 11 07:08:31 2015 -0800

----------------------------------------------------------------------
 .../DistributedMulticastRegionDUnitTest.java       | 16 ++++++++++++++++
 ...sAddNewNodeJUnitTest.testAddNewNodeNewNamed.xml | 17 -----------------
 ...ddNewNodeJUnitTest.testAddNewNodeNewUnnamed.xml | 17 -----------------
 ...JUnitTest.testAddNewNodeNewUnnamedExtension.xml | 17 -----------------
 ...NewNodeJUnitTest.testAddNewNodeReplaceNamed.xml | 17 -----------------
 ...wNodeJUnitTest.testAddNewNodeReplaceUnnamed.xml | 17 -----------------
 ...tTest.testAddNewNodeReplaceUnnamedExtension.xml | 17 -----------------
 ...tilsAddNewNodeJUnitTest.testDeleteNodeNamed.xml | 17 -----------------
 ...lsAddNewNodeJUnitTest.testDeleteNodeUnnamed.xml | 17 -----------------
 ...odeJUnitTest.testDeleteNodeUnnamedExtension.xml | 17 -----------------
 .../utils/XmlUtilsAddNewNodeJUnitTest.xml          | 17 -----------------
 ...nitTest.testBuildSchemaLocationMapAttribute.xml | 17 -----------------
 ...st.testBuildSchemaLocationMapEmptyAttribute.xml | 17 -----------------
 ...LocationMapMapOfStringListOfStringAttribute.xml | 17 -----------------
 ...est.testBuildSchemaLocationMapNullAttribute.xml | 17 -----------------
 .../XmlUtilsJUnitTest.testQuerySingleElement.xml   | 17 -----------------
 gradle/rat.gradle                                  |  3 +++
 17 files changed, 19 insertions(+), 255 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a84780e3/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/DistributedMulticastRegionDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/DistributedMulticastRegionDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/DistributedMulticastRegionDUnitTest.java
index f8296cc..517aa17 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/DistributedMulticastRegionDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/DistributedMulticastRegionDUnitTest.java
@@ -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.
+ */
 package com.gemstone.gemfire.cache30;
 
 import java.io.File;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a84780e3/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeNewNamed.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeNewNamed.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeNewNamed.xml
index e8e8d72..3055f31 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeNewNamed.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeNewNamed.xml
@@ -1,21 +1,4 @@
 <?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-
 <cache
     xmlns="http://schema.pivotal.io/gemfire/cache"
     xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a84780e3/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeNewUnnamed.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeNewUnnamed.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeNewUnnamed.xml
index b27039d..fb50e6b 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeNewUnnamed.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeNewUnnamed.xml
@@ -1,21 +1,4 @@
 <?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-
 <cache
     xmlns="http://schema.pivotal.io/gemfire/cache"
     xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a84780e3/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeNewUnnamedExtension.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeNewUnnamedExtension.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeNewUnnamedExtension.xml
index 32cf0c2..38f791d 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeNewUnnamedExtension.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeNewUnnamedExtension.xml
@@ -1,21 +1,4 @@
 <?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-
 <cache
     xmlns="http://schema.pivotal.io/gemfire/cache"
     xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a84780e3/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeReplaceNamed.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeReplaceNamed.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeReplaceNamed.xml
index 69e919b..4f5b1a1 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeReplaceNamed.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeReplaceNamed.xml
@@ -1,21 +1,4 @@
 <?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-
 <cache
     xmlns="http://schema.pivotal.io/gemfire/cache"
     xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a84780e3/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeReplaceUnnamed.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeReplaceUnnamed.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeReplaceUnnamed.xml
index ed88234..4db3799 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeReplaceUnnamed.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeReplaceUnnamed.xml
@@ -1,21 +1,4 @@
 <?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-
 <cache
     xmlns="http://schema.pivotal.io/gemfire/cache"
     xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a84780e3/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeReplaceUnnamedExtension.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeReplaceUnnamedExtension.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeReplaceUnnamedExtension.xml
index a2cecf0..a3238d8 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeReplaceUnnamedExtension.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testAddNewNodeReplaceUnnamedExtension.xml
@@ -1,21 +1,4 @@
 <?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-
 <cache
     xmlns="http://schema.pivotal.io/gemfire/cache"
     xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a84780e3/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testDeleteNodeNamed.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testDeleteNodeNamed.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testDeleteNodeNamed.xml
index 3f74bcb..f9d548b 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testDeleteNodeNamed.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testDeleteNodeNamed.xml
@@ -1,21 +1,4 @@
 <?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-
 <cache
     xmlns="http://schema.pivotal.io/gemfire/cache"
     xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a84780e3/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testDeleteNodeUnnamed.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testDeleteNodeUnnamed.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testDeleteNodeUnnamed.xml
index 85d6acc..b325fbf 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testDeleteNodeUnnamed.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testDeleteNodeUnnamed.xml
@@ -1,21 +1,4 @@
 <?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-
 <cache
     xmlns="http://schema.pivotal.io/gemfire/cache"
     xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a84780e3/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testDeleteNodeUnnamedExtension.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testDeleteNodeUnnamedExtension.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testDeleteNodeUnnamedExtension.xml
index 07afbce..53d8d6d 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testDeleteNodeUnnamedExtension.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.testDeleteNodeUnnamedExtension.xml
@@ -1,21 +1,4 @@
 <?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-
 <cache
     xmlns="http://schema.pivotal.io/gemfire/cache"
     xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a84780e3/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.xml
index bdf18e4..bdefa30 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsAddNewNodeJUnitTest.xml
@@ -1,21 +1,4 @@
 <?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-
 <cache
     xmlns="http://schema.pivotal.io/gemfire/cache"
     xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a84780e3/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapAttribute.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapAttribute.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapAttribute.xml
index 257eb8e..832d756 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapAttribute.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapAttribute.xml
@@ -1,21 +1,4 @@
 <?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-
 <cache
     xmlns="http://schema.pivotal.io/gemfire/cache"
     xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a84780e3/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapEmptyAttribute.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapEmptyAttribute.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapEmptyAttribute.xml
index 968ca6a..05843bf 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapEmptyAttribute.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapEmptyAttribute.xml
@@ -1,21 +1,4 @@
 <?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-
 <cache
     xmlns="http://schema.pivotal.io/gemfire/cache"
     xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a84780e3/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapMapOfStringListOfStringAttribute.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapMapOfStringListOfStringAttribute.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapMapOfStringListOfStringAttribute.xml
index bf2b2d8..87b3cac 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapMapOfStringListOfStringAttribute.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapMapOfStringListOfStringAttribute.xml
@@ -1,21 +1,4 @@
 <?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-
 <cache
     xmlns="http://schema.pivotal.io/gemfire/cache"
     xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a84780e3/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapNullAttribute.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapNullAttribute.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapNullAttribute.xml
index d32cb2c..95d11b3 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapNullAttribute.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testBuildSchemaLocationMapNullAttribute.xml
@@ -1,21 +1,4 @@
 <?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-
 <cache
     xmlns="http://schema.pivotal.io/gemfire/cache"
     xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a84780e3/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testQuerySingleElement.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testQuerySingleElement.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testQuerySingleElement.xml
index d5f41d6..76b851e 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testQuerySingleElement.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/XmlUtilsJUnitTest.testQuerySingleElement.xml
@@ -1,21 +1,4 @@
 <?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-
 <cache
     xmlns="http://schema.pivotal.io/gemfire/cache"
     xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a84780e3/gradle/rat.gradle
----------------------------------------------------------------------
diff --git a/gradle/rat.gradle b/gradle/rat.gradle
index 361aba9..c0f40ec 100644
--- a/gradle/rat.gradle
+++ b/gradle/rat.gradle
@@ -58,7 +58,10 @@ rat {
     'gemfire-spark-connector/project/plugins.sbt',
     'gemfire-spark-connector/project/build.properties',
     '**/log4j2*.xml',
+
+    // these are test files that don't expect the first element to be a comment
     '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/**',


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

Posted by kl...@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-217
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


[17/48] incubator-geode git commit: adding a unit test for initiation of suspect processing when shared/unordered

Posted by kl...@apache.org.
adding a unit test for initiation of suspect processing when shared/unordered

As part of this work I changed the internal membership listener interfaces to
propagate the reason for initiating suspect processing.  The test gathers
these reasons and asserts that at least one is initiated by TCPConduit.


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

Branch: refs/heads/feature/GEODE-217
Commit: 1c423796bc6dd9b0ce0f5557b6f4b31e1466f23d
Parents: 24c170a
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Thu Dec 10 14:52:36 2015 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Thu Dec 10 14:53:50 2015 -0800

----------------------------------------------------------------------
 .../DistributedSystemHealthEvaluator.java       |  2 +-
 .../internal/DistributionAdvisor.java           |  2 +-
 .../internal/DistributionManager.java           | 24 +++++++++++++-------
 .../internal/MembershipListener.java            |  3 ++-
 .../distributed/internal/ProductUseLog.java     |  2 +-
 .../distributed/internal/ReplyProcessor21.java  |  2 +-
 .../internal/locks/DLockGrantor.java            |  2 +-
 .../DistributedMembershipListener.java          |  3 ++-
 .../internal/membership/gms/Services.java       | 12 +++++-----
 .../internal/membership/gms/SuspectMember.java  |  9 ++++++--
 .../membership/gms/auth/GMSAuthenticator.java   |  2 +-
 .../membership/gms/fd/GMSHealthMonitor.java     |  4 ++--
 .../membership/gms/interfaces/Service.java      |  3 ++-
 .../membership/gms/membership/GMSJoinLeave.java |  2 +-
 .../gms/messenger/JGroupsMessenger.java         |  2 +-
 .../gms/mgr/GMSMembershipManager.java           |  6 ++---
 .../admin/remote/RemoteGfManagerAgent.java      |  2 +-
 .../internal/cache/DistributedRegion.java       |  2 +-
 .../internal/cache/InitialImageFlowControl.java |  2 +-
 .../internal/cache/PRHARedundancyProvider.java  |  4 ++--
 .../internal/cache/PartitionedRegion.java       |  2 +-
 .../internal/cache/PartitionedRegionHelper.java |  2 +-
 .../cache/SearchLoadAndWriteProcessor.java      |  2 +-
 .../gemfire/internal/cache/TXCommitMessage.java |  2 +-
 .../internal/cache/TXFarSideCMTracker.java      |  2 +-
 .../gemfire/internal/cache/TXManagerImpl.java   |  2 +-
 .../PartitionedRegionRebalanceOp.java           |  2 +-
 .../cache/persistence/BackupManager.java        |  2 +-
 .../persistence/PersistenceAdvisorImpl.java     |  2 +-
 .../persistence/PersistentMemberManager.java    |  2 +-
 .../cache/versions/RegionVersionVector.java     |  2 +-
 .../gemfire/internal/tcp/Connection.java        |  6 +++--
 .../management/internal/FederatingManager.java  |  5 ++--
 .../internal/ManagementMembershipListener.java  |  4 ++--
 .../internal/SystemManagementService.java       |  4 ++--
 .../internal/beans/MBeanAggregator.java         |  2 +-
 .../gemfire/distributed/LocatorDUnitTest.java   |  9 ++++++--
 .../gms/mgr/GMSMembershipManagerJUnitTest.java  |  8 +++----
 38 files changed, 87 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/gemfire-core/src/main/java/com/gemstone/gemfire/admin/internal/DistributedSystemHealthEvaluator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/admin/internal/DistributedSystemHealthEvaluator.java b/gemfire-core/src/main/java/com/gemstone/gemfire/admin/internal/DistributedSystemHealthEvaluator.java
index 534f559..511edba 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/admin/internal/DistributedSystemHealthEvaluator.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/admin/internal/DistributedSystemHealthEvaluator.java
@@ -166,7 +166,7 @@ class DistributedSystemHealthEvaluator
   }
 
   public void memberSuspect(InternalDistributedMember id,
-      InternalDistributedMember whoSuspected) {
+      InternalDistributedMember whoSuspected, String reason) {
   }
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionAdvisor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionAdvisor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionAdvisor.java
index dc76d6b..538fa96 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionAdvisor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionAdvisor.java
@@ -244,7 +244,7 @@ public class DistributionAdvisor  {
       }
 
       public void memberSuspect(InternalDistributedMember id,
-          InternalDistributedMember whoSuspected) {
+          InternalDistributedMember whoSuspected, String reason) {
       }
       
    };

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/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 7a9f7c0..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
@@ -3333,9 +3333,10 @@ public class DistributionManager
   }
 
   /**
+   * @param reason TODO
    */
   public void handleManagerSuspect(InternalDistributedMember suspect, 
-      InternalDistributedMember whoSuspected) {
+      InternalDistributedMember whoSuspected, String reason) {
     if (!isCurrentMember(suspect)) {
       return; // fault tolerance
     }
@@ -3345,7 +3346,7 @@ public class DistributionManager
       return;
     }
 
-    addMemberEvent(new MemberSuspectEvent(suspect, whoSuspected));
+    addMemberEvent(new MemberSuspectEvent(suspect, whoSuspected, reason));
   }
   
   public void handleViewInstalled(NetView view) {
@@ -3756,7 +3757,7 @@ public class DistributionManager
                   }
                 }
                 public void memberSuspect(InternalDistributedMember id,
-                    InternalDistributedMember whoSuspected) {
+                    InternalDistributedMember whoSuspected, String reason) {
                 }
                 public void viewInstalled(NetView view) {
                 }
@@ -4424,8 +4425,8 @@ public class DistributionManager
       dm.handleManagerDeparture(theId, crashed, reason);
     }
     
-    public void memberSuspect(InternalDistributedMember suspect, InternalDistributedMember whoSuspected) {
-      dm.handleManagerSuspect(suspect, whoSuspected);
+    public void memberSuspect(InternalDistributedMember suspect, InternalDistributedMember whoSuspected, String reason) {
+      dm.handleManagerSuspect(suspect, whoSuspected, reason);
     }
     
     public void viewInstalled(NetView view) {
@@ -4579,20 +4580,27 @@ public class DistributionManager
    */
   private static final class MemberSuspectEvent extends MemberEvent {
     InternalDistributedMember whoSuspected;
-    MemberSuspectEvent(InternalDistributedMember suspect, InternalDistributedMember whoSuspected) {
+    String reason;
+    MemberSuspectEvent(InternalDistributedMember suspect, InternalDistributedMember whoSuspected, String reason) {
       super(suspect);
       this.whoSuspected = whoSuspected;
+      this.reason = reason;
     }
     public InternalDistributedMember whoSuspected() {
       return this.whoSuspected;
     }
+    
+    public String getReason() {
+      return this.reason;
+    }
+    
     @Override
     public String toString() {
-      return "member " + getId() + " suspected by: " + this.whoSuspected;
+      return "member " + getId() + " suspected by: " + this.whoSuspected + " reason: " + reason;
     }
     @Override
     protected void handleEvent(MembershipListener listener) {
-      listener.memberSuspect(getId(), whoSuspected());	
+      listener.memberSuspect(getId(), whoSuspected(), reason);	
     }
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/MembershipListener.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/MembershipListener.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/MembershipListener.java
index 78ed51b..b545653 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/MembershipListener.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/MembershipListener.java
@@ -60,9 +60,10 @@ public interface MembershipListener {
    * removed from the membership view
    * @param id the suspected member
    * @param whoSuspected the member that initiated suspect processing
+   * @param reason TODO
    */
   public void memberSuspect(InternalDistributedMember id,
-      InternalDistributedMember whoSuspected);
+      InternalDistributedMember whoSuspected, String reason);
   
   /**
    * This is notification that more than 50% of member weight has been

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/ProductUseLog.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/ProductUseLog.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/ProductUseLog.java
index 823b3d0..95d538c 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/ProductUseLog.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/ProductUseLog.java
@@ -136,7 +136,7 @@ public final class ProductUseLog implements MembershipListener {
 
   @Override
   public void memberSuspect(InternalDistributedMember id,
-      InternalDistributedMember whoSuspected) {
+      InternalDistributedMember whoSuspected, String reason) {
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/ReplyProcessor21.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/ReplyProcessor21.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/ReplyProcessor21.java
index aa5f66c..b219630 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/ReplyProcessor21.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/ReplyProcessor21.java
@@ -508,7 +508,7 @@ public class ReplyProcessor21
   }
 
   public void memberSuspect(InternalDistributedMember id,
-      InternalDistributedMember whoSuspected) {
+      InternalDistributedMember whoSuspected, String reason) {
     if (isSevereAlertProcessingEnabled()) {
       // if we're waiting for the member that initiated suspicion, we don't
       // want to be hasty about kicking it out of the distributed system

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/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 92cea62..b4e646c 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
@@ -3731,7 +3731,7 @@ public class DLockGrantor {
     public void quorumLost(Set<InternalDistributedMember> failures, List<InternalDistributedMember> remaining) {
     }
     public void memberSuspect(InternalDistributedMember id,
-        InternalDistributedMember whoSuspected) {
+        InternalDistributedMember whoSuspected, String reason) {
     }
     public void memberDeparted(final InternalDistributedMember id, final boolean crashed) {
       final DLockGrantor me = DLockGrantor.this;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/DistributedMembershipListener.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/DistributedMembershipListener.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/DistributedMembershipListener.java
index acfd6ba..d970b6b 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/DistributedMembershipListener.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/DistributedMembershipListener.java
@@ -49,8 +49,9 @@ public interface DistributedMembershipListener extends DirectChannelListener {
   /**
    * Event indicating that a member is suspected of having departed but
    * is still in the membership view
+   * @param reason TODO
    */
-  public void memberSuspect(InternalDistributedMember suspect, InternalDistributedMember whoSuspected);
+  public void memberSuspect(InternalDistributedMember suspect, InternalDistributedMember whoSuspected, String reason);
 
   /**
    * Event indicating a message has been delivered that we need to process.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/Services.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/Services.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/Services.java
index 799f95d..4484c00 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/Services.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/Services.java
@@ -277,20 +277,20 @@ public class Services {
     manager.installView(v);
   }
   
-  public void memberSuspected(InternalDistributedMember initiator, InternalDistributedMember suspect) {
+  public void memberSuspected(InternalDistributedMember initiator, InternalDistributedMember suspect, String reason) {
     try {
-      joinLeave.memberSuspected(initiator, suspect);
+      joinLeave.memberSuspected(initiator, suspect, reason);
     } finally {
       try {
-        healthMon.memberSuspected(initiator, suspect);
+        healthMon.memberSuspected(initiator, suspect, reason);
       } finally {
         try {
-          auth.memberSuspected(initiator, suspect);
+          auth.memberSuspected(initiator, suspect, reason);
         } finally {
           try {
-            messenger.memberSuspected(initiator, suspect);
+            messenger.memberSuspected(initiator, suspect, reason);
           } finally {
-            manager.memberSuspected(initiator, suspect);
+            manager.memberSuspected(initiator, suspect, reason);
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/SuspectMember.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/SuspectMember.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/SuspectMember.java
index a03ee92..ca27698 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/SuspectMember.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/SuspectMember.java
@@ -27,10 +27,15 @@ public class SuspectMember
   /** suspected member */
   public InternalDistributedMember suspectedMember;
   
-  /** create a new SuspectMember */
-  public SuspectMember(InternalDistributedMember whoSuspected, InternalDistributedMember suspectedMember) {
+  /** the reason */
+  public String reason;
+  
+  /** create a new SuspectMember 
+   * @param reason TODO*/
+  public SuspectMember(InternalDistributedMember whoSuspected, InternalDistributedMember suspectedMember, String reason) {
     this.whoSuspected = whoSuspected;
     this.suspectedMember = suspectedMember;
+    this.reason = reason;
   }
   
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/auth/GMSAuthenticator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/auth/GMSAuthenticator.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/auth/GMSAuthenticator.java
index 982942d..ba35e46 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/auth/GMSAuthenticator.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/auth/GMSAuthenticator.java
@@ -90,7 +90,7 @@ public class GMSAuthenticator implements Authenticator {
   }
   
   @Override
-  public void memberSuspected(InternalDistributedMember initiator, InternalDistributedMember suspect) {
+  public void memberSuspected(InternalDistributedMember initiator, InternalDistributedMember suspect, String reason) {
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/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 7709114..8c38ba6 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
@@ -940,7 +940,7 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
   }
 
   @Override
-  public void memberSuspected(InternalDistributedMember initiator, InternalDistributedMember suspect) {
+  public void memberSuspected(InternalDistributedMember initiator, InternalDistributedMember suspect, String reason) {
   }
 
   @Override
@@ -1177,7 +1177,7 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
           @Override
           public void run() {
             try {
-              services.memberSuspected(initiator, mbr);
+              services.memberSuspected(initiator, mbr, reason);
               long startTime = System.currentTimeMillis();
               // for some reason we used to update the timestamp for the member
               // with the startTime, but we don't want to do that because it looks

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/interfaces/Service.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/interfaces/Service.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/interfaces/Service.java
index 2f4a728..8465067 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/interfaces/Service.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/interfaces/Service.java
@@ -75,8 +75,9 @@ public interface Service {
   
   /**
    * a member is suspected of having crashed
+   * @param reason TODO
    */
-  void memberSuspected(InternalDistributedMember initiator, InternalDistributedMember suspect);
+  void memberSuspected(InternalDistributedMember initiator, InternalDistributedMember suspect, String reason);
 
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/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 ccc9d8c..e1821db 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
@@ -1301,7 +1301,7 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
   }
 
   @Override
-  public void memberSuspected(InternalDistributedMember initiator, InternalDistributedMember suspect) {
+  public void memberSuspected(InternalDistributedMember initiator, InternalDistributedMember suspect, String reason) {
     prepareProcessor.memberSuspected(initiator, suspect);
     viewProcessor.memberSuspected(initiator, suspect);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/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 326491a..36a6200 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
@@ -350,7 +350,7 @@ public class JGroupsMessenger implements Messenger {
   }
 
   @Override
-  public void memberSuspected(InternalDistributedMember initiator, InternalDistributedMember suspect) {
+  public void memberSuspected(InternalDistributedMember initiator, InternalDistributedMember suspect, String reason) {
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/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 bbe7ab3..93c14e2 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
@@ -1209,8 +1209,8 @@ public class GMSMembershipManager implements MembershipManager, Manager
   }
   
   @Override
-  public void memberSuspected(InternalDistributedMember initiator, InternalDistributedMember suspect) {
-    SuspectMember s = new SuspectMember(initiator, suspect);
+  public void memberSuspected(InternalDistributedMember initiator, InternalDistributedMember suspect, String reason) {
+    SuspectMember s = new SuspectMember(initiator, suspect, reason);
     handleOrDeferSuspect(s);
   }
 
@@ -1230,7 +1230,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
       InternalDistributedMember who = suspectInfo.whoSuspected;
       this.suspectedMembers.put(suspect, Long.valueOf(System.currentTimeMillis()));
       try {
-        listener.memberSuspect(suspect, who);
+        listener.memberSuspect(suspect, who, suspectInfo.reason);
       }
       catch (DistributedSystemDisconnectedException se) {
         // let's not get huffy about it

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/RemoteGfManagerAgent.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/RemoteGfManagerAgent.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/RemoteGfManagerAgent.java
index eb62468..6a31b50 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/RemoteGfManagerAgent.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/RemoteGfManagerAgent.java
@@ -1428,7 +1428,7 @@ public
     }
     
     public void memberSuspect(InternalDistributedMember id,
-        InternalDistributedMember whoSuspected) {
+        InternalDistributedMember whoSuspected, String reason) {
     }
     
     public void quorumLost(Set<InternalDistributedMember> failures, List<InternalDistributedMember> remaining) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegion.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegion.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegion.java
index 5d263a6..92b585a 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegion.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegion.java
@@ -3805,7 +3805,7 @@ public class DistributedRegion extends LocalRegion implements
     }
 
     public void memberSuspect(InternalDistributedMember id,
-        InternalDistributedMember whoSuspected) {
+        InternalDistributedMember whoSuspected, String reason) {
     }
     
     /** called when membership listeners are added after region creation */

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/InitialImageFlowControl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/InitialImageFlowControl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/InitialImageFlowControl.java
index a37a39b..7777a33 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/InitialImageFlowControl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/InitialImageFlowControl.java
@@ -186,7 +186,7 @@ public class InitialImageFlowControl implements MembershipListener {
   }
 
   public void memberSuspect(InternalDistributedMember id,
-      InternalDistributedMember whoSuspected) {
+      InternalDistributedMember whoSuspected, String reason) {
     //Do nothing
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PRHARedundancyProvider.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PRHARedundancyProvider.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PRHARedundancyProvider.java
index 3e21113..3abb0d6 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PRHARedundancyProvider.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PRHARedundancyProvider.java
@@ -2206,7 +2206,7 @@ public class PRHARedundancyProvider
     }
 
     public void memberSuspect(InternalDistributedMember id,
-        InternalDistributedMember whoSuspected) {
+        InternalDistributedMember whoSuspected, String reason) {
     }
     
     public void memberDeparted(InternalDistributedMember id, boolean crashed) {
@@ -2339,7 +2339,7 @@ public class PRHARedundancyProvider
     }
     
     public void memberSuspect(InternalDistributedMember id,
-        InternalDistributedMember whoSuspected) {
+        InternalDistributedMember whoSuspected, String reason) {
     }
     
     public void memberJoined(InternalDistributedMember id)

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegion.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegion.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegion.java
index 48f4787..a36d719 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegion.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegion.java
@@ -9968,7 +9968,7 @@ public class PartitionedRegion extends LocalRegion implements
     }
 
     public void memberSuspect(InternalDistributedMember id,
-        InternalDistributedMember whoSuspected) {
+        InternalDistributedMember whoSuspected, String reason) {
     }
     
     public synchronized void memberDeparted(InternalDistributedMember id,

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegionHelper.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegionHelper.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegionHelper.java
index f4a20da..ef89c80 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegionHelper.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegionHelper.java
@@ -1086,7 +1086,7 @@ public class PartitionedRegionHelper
     }
 
     public void memberSuspect(InternalDistributedMember id,
-        InternalDistributedMember whoSuspected) {
+        InternalDistributedMember whoSuspected, String reason) {
     }
     
     public void quorumLost(Set<InternalDistributedMember> failures, List<InternalDistributedMember> remaining) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/SearchLoadAndWriteProcessor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/SearchLoadAndWriteProcessor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/SearchLoadAndWriteProcessor.java
index d7caf66..e96d99d 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/SearchLoadAndWriteProcessor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/SearchLoadAndWriteProcessor.java
@@ -237,7 +237,7 @@ public class SearchLoadAndWriteProcessor implements MembershipListener {
   }
 
   public void memberSuspect(InternalDistributedMember id,
-      InternalDistributedMember whoSuspected) {
+      InternalDistributedMember whoSuspected, String reason) {
   }
   
   public void quorumLost(Set<InternalDistributedMember> failures, List<InternalDistributedMember> remaining) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TXCommitMessage.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TXCommitMessage.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TXCommitMessage.java
index 9648503..e712943 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TXCommitMessage.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TXCommitMessage.java
@@ -2027,7 +2027,7 @@ public class TXCommitMessage extends PooledDistributionMessage implements Member
   }
   
   public void memberSuspect(InternalDistributedMember id,
-      InternalDistributedMember whoSuspected) {
+      InternalDistributedMember whoSuspected, String reason) {
   }
   
   public void quorumLost(Set<InternalDistributedMember> failures, List<InternalDistributedMember> remaining) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TXFarSideCMTracker.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TXFarSideCMTracker.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TXFarSideCMTracker.java
index c99d8ae..94e3059 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TXFarSideCMTracker.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TXFarSideCMTracker.java
@@ -211,7 +211,7 @@ public class TXFarSideCMTracker
         public void memberJoined(InternalDistributedMember id) {
         }
         public void memberSuspect(InternalDistributedMember id,
-            InternalDistributedMember whoSuspected) {
+            InternalDistributedMember whoSuspected, String reason) {
         }
         public void memberDeparted(InternalDistributedMember id, boolean crashed) {
           if (memberId.equals(id)) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TXManagerImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TXManagerImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TXManagerImpl.java
index 994990b..f4216ac 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TXManagerImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TXManagerImpl.java
@@ -925,7 +925,7 @@ public final class TXManagerImpl implements CacheTransactionManager,
   }
 
   public void memberSuspect(InternalDistributedMember id,
-      InternalDistributedMember whoSuspected) {
+      InternalDistributedMember whoSuspected, String reason) {
   }
   
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PartitionedRegionRebalanceOp.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PartitionedRegionRebalanceOp.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PartitionedRegionRebalanceOp.java
index 2b21982..a8d1350 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PartitionedRegionRebalanceOp.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PartitionedRegionRebalanceOp.java
@@ -644,7 +644,7 @@ public class PartitionedRegionRebalanceOp {
     }
 
     public void memberSuspect(InternalDistributedMember id,
-        InternalDistributedMember whoSuspected) {
+        InternalDistributedMember whoSuspected, String reason) {
       // do nothing.
     }
     

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/persistence/BackupManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/persistence/BackupManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/persistence/BackupManager.java
index 8f27736..1a5c765 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/persistence/BackupManager.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/persistence/BackupManager.java
@@ -352,7 +352,7 @@ public class BackupManager implements MembershipListener {
   }
 
   public void memberSuspect(InternalDistributedMember id,
-      InternalDistributedMember whoSuspected) {
+      InternalDistributedMember whoSuspected, String reason) {
   }
 
   public void waitForBackup() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/persistence/PersistenceAdvisorImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/persistence/PersistenceAdvisorImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/persistence/PersistenceAdvisorImpl.java
index 21af700..168fbfc 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/persistence/PersistenceAdvisorImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/persistence/PersistenceAdvisorImpl.java
@@ -1186,7 +1186,7 @@ public class PersistenceAdvisorImpl implements PersistenceAdvisor {
     }
 
     public void memberSuspect(InternalDistributedMember id,
-        InternalDistributedMember whoSuspected) {
+        InternalDistributedMember whoSuspected, String reason) {
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/persistence/PersistentMemberManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/persistence/PersistentMemberManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/persistence/PersistentMemberManager.java
index 87e956b..a6f6d64 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/persistence/PersistentMemberManager.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/persistence/PersistentMemberManager.java
@@ -255,7 +255,7 @@ public class PersistentMemberManager {
 
     @Override
     public void memberSuspect(InternalDistributedMember id,
-        InternalDistributedMember whoSuspected) {
+        InternalDistributedMember whoSuspected, String reason) {
     }
     
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/versions/RegionVersionVector.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/versions/RegionVersionVector.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/versions/RegionVersionVector.java
index 61423d1..27f3747 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/versions/RegionVersionVector.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/versions/RegionVersionVector.java
@@ -1364,7 +1364,7 @@ public abstract class RegionVersionVector<T extends VersionSource<?>> implements
 
 
   public void memberJoined(InternalDistributedMember id) { }
-  public void memberSuspect(InternalDistributedMember id, InternalDistributedMember whoSuspected) {  }
+  public void memberSuspect(InternalDistributedMember id, InternalDistributedMember whoSuspected, String reason) {  }
   public void quorumLost(Set<InternalDistributedMember> failures, List<InternalDistributedMember> remaining) {  }
 
   /* 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/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 30962e7..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
@@ -123,6 +123,9 @@ public class Connection implements Runnable {
   /** counter to give connections a unique id */
   private static AtomicLong idCounter = new AtomicLong(1);
 
+  /** string used as the reason for initiating suspect processing */
+  public static final String INITIATING_SUSPECT_PROCESSING = "member unexpectedly shut down shared, unordered connection";
+
   /** the table holding this connection */
   final ConnectionTable owner;
   
@@ -1954,9 +1957,8 @@ public class Connection implements Runnable {
   private void initiateSuspicionIfSharedUnordered() {
     if (this.isReceiver && this.handshakeRead && !this.preserveOrder && this.sharedResource) {
       if (this.owner.getConduit().getCancelCriterion().cancelInProgress() == null) {
-        String reason = "member unexpectedly shut down shared, unordered connection";
         this.owner.getDM().getMembershipManager().suspectMember(this.getRemoteAddress(),
-            reason);
+            INITIATING_SUSPECT_PROCESSING);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/FederatingManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/FederatingManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/FederatingManager.java
index d49589c..91e7b22 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/FederatingManager.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/FederatingManager.java
@@ -276,9 +276,10 @@ public class FederatingManager extends Manager {
    * block the membership listener
    * 
    * @param member
+   * @param reason TODO
    */
-  public void suspectMember(DistributedMember member, InternalDistributedMember whoSuspected) {
-    service.memberSuspect((InternalDistributedMember) member, whoSuspected);
+  public void suspectMember(DistributedMember member, InternalDistributedMember whoSuspected, String reason) {
+    service.memberSuspect((InternalDistributedMember) member, whoSuspected, reason);
 
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/ManagementMembershipListener.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/ManagementMembershipListener.java b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/ManagementMembershipListener.java
index 561981c..9a7c657 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/ManagementMembershipListener.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/ManagementMembershipListener.java
@@ -76,7 +76,7 @@ public class ManagementMembershipListener implements MembershipListener {
   }
 
   @Override
-  public void memberSuspect(InternalDistributedMember id, InternalDistributedMember whoSuspected) {
+  public void memberSuspect(InternalDistributedMember id, InternalDistributedMember whoSuspected, String reason) {
 
     if (logger.isDebugEnabled()) {
       logger.debug("ManagementMembershipListener member suspected .. {}", id.getId());
@@ -85,7 +85,7 @@ public class ManagementMembershipListener implements MembershipListener {
       if (logger.isDebugEnabled()) {
         logger.debug("Suspecting member {}", id.getId());
       }
-      service.getFederatingManager().suspectMember(id, whoSuspected);
+      service.getFederatingManager().suspectMember(id, whoSuspected, reason);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/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 7ac85b8..68209f2 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
@@ -729,9 +729,9 @@ public final class SystemManagementService extends BaseManagementService {
     }
   }
 
-  public void memberSuspect(InternalDistributedMember id, InternalDistributedMember whoSuspected) {
+  public void memberSuspect(InternalDistributedMember id, InternalDistributedMember whoSuspected, String reason) {
     for (ProxyListener listener : proxyListeners) {
-      listener.memberSuspect(id, whoSuspected);
+      listener.memberSuspect(id, whoSuspected, reason);
     }
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/beans/MBeanAggregator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/beans/MBeanAggregator.java b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/beans/MBeanAggregator.java
index 8c096a4..a64e7f7 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/beans/MBeanAggregator.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/beans/MBeanAggregator.java
@@ -423,7 +423,7 @@ public class MBeanAggregator implements ProxyListener {
 
   @Override
   public void memberSuspect(InternalDistributedMember id,
-      InternalDistributedMember whoSuspected) {
+      InternalDistributedMember whoSuspected, String reason) {
     distributedSystemBridge.memberSuspect(id, whoSuspected);
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/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 5b12563..7d34ab1 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
@@ -20,6 +20,7 @@ import java.io.File;
 import java.io.FileReader;
 import java.io.IOException;
 import java.io.LineNumberReader;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Properties;
 import java.util.Set;
@@ -48,6 +49,7 @@ import com.gemstone.gemfire.internal.AvailablePort;
 import com.gemstone.gemfire.internal.AvailablePortHelper;
 import com.gemstone.gemfire.internal.logging.InternalLogWriter;
 import com.gemstone.gemfire.internal.logging.LocalLogWriter;
+import com.gemstone.gemfire.internal.tcp.Connection;
 
 import dunit.AsyncInvocation;
 import dunit.DistributedTestCase;
@@ -60,7 +62,6 @@ import dunit.VM;
  * Tests the ability of the {@link Locator} API to start and stop
  * locators running in remote VMs.
  *
- * @author David Whitlock
  * @since 4.0
  */
 public class LocatorDUnitTest extends DistributedTestCase {
@@ -581,6 +582,7 @@ public class LocatorDUnitTest extends DistributedTestCase {
       }
       // quorumLost should be invoked if we get a ForcedDisconnect in this situation
       assertTrue("expected quorumLost to be invoked", listener.quorumLostInvoked);
+      assertTrue("expected suspect processing initiated by TCPConduit", listener.suspectReasons.contains(Connection.INITIATING_SUSPECT_PROCESSING));
     }
     finally {
       if (locator != null) {
@@ -1882,11 +1884,14 @@ public class LocatorDUnitTest extends DistributedTestCase {
   }
   class MyMembershipListener implements MembershipListener {
     boolean quorumLostInvoked;
+    List<String> suspectReasons = new ArrayList<>(50);
     
     public void memberJoined(InternalDistributedMember id) {  }
     public void memberDeparted(InternalDistributedMember id, boolean crashed) { }
     public void memberSuspect(InternalDistributedMember id,
-        InternalDistributedMember whoSuspected) { }
+        InternalDistributedMember whoSuspected, String reason) {
+      suspectReasons.add(reason);
+    }
     public void quorumLost(Set<InternalDistributedMember> failures,
         List<InternalDistributedMember> remaining) {
       quorumLostInvoked = true;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c423796/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 e133625..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
@@ -233,10 +233,10 @@ public class GMSMembershipManagerJUnitTest {
 
     // suspect a member
     InternalDistributedMember suspectMember = mockMembers[1];
-    manager.handleOrDeferSuspect(new SuspectMember(mockMembers[0], suspectMember));
+    manager.handleOrDeferSuspect(new SuspectMember(mockMembers[0], suspectMember, "testing"));
     // suspect messages aren't queued - they're ignored before joining the system
     assertEquals(2, manager.getStartupEvents().size());
-    verify(listener, never()).memberSuspect(suspectMember, mockMembers[0]);
+    verify(listener, never()).memberSuspect(suspectMember, mockMembers[0], "testing");
 
     HighPriorityAckedMessage m = new HighPriorityAckedMessage();
     mockMembers[0].setVmViewId(1);
@@ -293,8 +293,8 @@ public class GMSMembershipManagerJUnitTest {
     // process a suspect now - it will be passed to the listener
     reset(listener);
     suspectMember = mockMembers[1];
-    manager.handleOrDeferSuspect(new SuspectMember(mockMembers[0], suspectMember));
-    verify(listener).memberSuspect(suspectMember, mockMembers[0]);
+    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);


[13/48] incubator-geode git commit: Revision 56b3757129fb20ab047d8c164fa965eff50099c4 closed #50

Posted by kl...@apache.org.
Revision 56b3757129fb20ab047d8c164fa965eff50099c4 closed #50


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

Branch: refs/heads/feature/GEODE-217
Commit: aa27c6a77734d417e4f369a32dd709978ecff63d
Parents: 56b3757
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Thu Dec 10 11:28:24 2015 -0800
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Thu Dec 10 11:28:24 2015 -0800

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

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



[36/48] 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 kl...@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'
   ]
 }
-


[29/48] incubator-geode git commit: Removing TCPConduit's Stub ID class

Posted by kl...@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-217
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);
-  }
-}


[37/48] 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 kl...@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'


[14/48] incubator-geode git commit: Revision 68dfcab10d68b1babb2035bc4c87c93acf52077c closed #48

Posted by kl...@apache.org.
Revision 68dfcab10d68b1babb2035bc4c87c93acf52077c closed #48


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

Branch: refs/heads/feature/GEODE-217
Commit: 34eb0fe441b0759895240f605f5e33c67a657ead
Parents: aa27c6a
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Thu Dec 10 11:42:19 2015 -0800
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Thu Dec 10 11:42:19 2015 -0800

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

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



[18/48] incubator-geode git commit: added test for region multicast enabled

Posted by kl...@apache.org.
added test for region multicast enabled

updated multicast test


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

Branch: refs/heads/feature/GEODE-217
Commit: 81c472f714967da158a1039e5efda4fc478b56b4
Parents: 1c42379
Author: Hitesh Khamesra <hi...@yahoo.com>
Authored: Wed Dec 9 11:15:34 2015 -0800
Committer: Hitesh Khamesra <hi...@yahoo.com>
Committed: Thu Dec 10 15:31:12 2015 -0800

----------------------------------------------------------------------
 .../gemfire/distributed/internal/DMStats.java   |   1 +
 .../distributed/internal/DistributionStats.java |   3 +
 .../internal/LonerDistributionManager.java      |   2 +
 .../DistributedMulticastRegionDUnitTest.java    | 197 +++++++++++++++++++
 4 files changed, 203 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/81c472f7/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DMStats.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DMStats.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DMStats.java
index e7b155a..e79a40b 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DMStats.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DMStats.java
@@ -205,6 +205,7 @@ public interface DMStats {
    * returns the current value of the mcastWrites statistic
    */
   public int getMcastWrites();
+  public int getMcastReads();
 
   public long startSerialization();
   public void endSerialization(long start, int bytes);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/81c472f7/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionStats.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionStats.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionStats.java
index 39b4986..804b507 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionStats.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionStats.java
@@ -1094,6 +1094,9 @@ public class DistributionStats implements DMStats {
   public int getMcastWrites() {
     return stats.getInt(mcastWritesId);
   }
+  public int getMcastReads() {
+    return stats.getInt(mcastReadsId);
+  }
   public void incMcastReadBytes(int amount) {
     stats.incInt(mcastReadsId, 1);
     stats.incLong(mcastReadBytesId, amount);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/81c472f7/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/LonerDistributionManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/LonerDistributionManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/LonerDistributionManager.java
index 1fc9f5b..60158d1 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/LonerDistributionManager.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/LonerDistributionManager.java
@@ -451,6 +451,8 @@ public class LonerDistributionManager implements DM {
     @Override
     public int getMcastWrites() { return 0; }
     @Override
+    public int getMcastReads() { return 0; }
+    @Override
     public void incUcastReadBytes(int amount) {}
     @Override
     public void incMcastReadBytes(int amount) {}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/81c472f7/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/DistributedMulticastRegionDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/DistributedMulticastRegionDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/DistributedMulticastRegionDUnitTest.java
new file mode 100755
index 0000000..f8296cc
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/DistributedMulticastRegionDUnitTest.java
@@ -0,0 +1,197 @@
+package com.gemstone.gemfire.cache30;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import com.gemstone.gemfire.cache.AttributesFactory;
+import com.gemstone.gemfire.cache.CacheException;
+import com.gemstone.gemfire.cache.DataPolicy;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionAttributes;
+import com.gemstone.gemfire.cache.Scope;
+import com.gemstone.gemfire.distributed.Locator;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.internal.InternalLocator;
+import com.gemstone.gemfire.distributed.internal.SharedConfiguration;
+import com.gemstone.gemfire.internal.AvailablePortHelper;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.internal.cache.OffHeapTestUtil;
+
+import dunit.DistributedTestCase;
+import dunit.Host;
+import dunit.SerializableCallable;
+import dunit.SerializableRunnable;
+import dunit.VM;
+
+public class DistributedMulticastRegionDUnitTest extends CacheTestCase {
+
+  static int locatorVM = 3;
+  static String mcastport = "42786";
+  static String mcastttl = "0";
+  
+  private int locatorPort;
+
+  public DistributedMulticastRegionDUnitTest(String name) {
+    super(name);
+  }
+  
+  @Override
+  public void setUp() throws Exception {
+    clean();
+    super.setUp();    
+  }
+  
+  @Override
+  public void tearDown2() throws Exception {
+    clean();
+    super.tearDown2();
+  }
+  
+  private void clean(){
+    SerializableRunnable cleanVM =
+        new CacheSerializableRunnable("clean VM") {
+            public void run2() throws CacheException {
+              disconnectFromDS();
+            }
+        };
+    invokeInEveryVM(cleanVM);    
+  }
+  
+  public void testMulticastEnabled() {
+    final String name = "mcastRegion";
+    SerializableRunnable create =
+      new CacheSerializableRunnable("Create Region") {
+          public void run2() throws CacheException {
+            createRegion(name, getRegionAttributes());
+          }
+        };
+
+    locatorPort = startLocator();
+    Host host = Host.getHost(0);
+    final VM vm0 = host.getVM(0);
+    final VM vm1 = host.getVM(1);
+    //1. start locator with mcast port
+    vm0.invoke(create);
+    vm1.invoke(create);
+    
+    SerializableRunnable validateMulticastBeforeRegionOps =
+        new CacheSerializableRunnable("validateMulticast before region ops") {
+            public void run2() throws CacheException {
+              validateMulticastOpsBeforeRegionOps();
+            }
+        };
+      
+    vm0.invoke(validateMulticastBeforeRegionOps);
+    vm1.invoke(validateMulticastBeforeRegionOps);
+    
+    SerializableRunnable doPuts =
+      new CacheSerializableRunnable("do put") {
+          public void run2() throws CacheException {
+            final Region region =
+                getRootRegion().getSubregion(name);
+            for(int i =0 ; i < 5; i++) {
+              region.put(i, i);
+            }
+          }
+      };
+      
+    vm0.invoke(doPuts);
+    
+    SerializableRunnable validateMulticastAfterRegionOps =
+      new CacheSerializableRunnable("validateMulticast after region ops") {
+          public void run2() throws CacheException {
+            validateMulticastOpsAfterRegionOps();
+          }
+      };
+    
+      vm0.invoke(validateMulticastAfterRegionOps);
+      vm1.invoke(validateMulticastAfterRegionOps);
+   
+      closeLocator();      
+  }
+  
+  protected RegionAttributes getRegionAttributes() {
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.DISTRIBUTED_ACK);
+    factory.setDataPolicy(DataPolicy.PRELOADED);
+    factory.setEarlyAck(false);
+    factory.setConcurrencyChecksEnabled(false);
+    factory.setMulticastEnabled(true);
+    return factory.create();
+  }
+  
+  public Properties getDistributedSystemProperties() {
+    Properties p = new Properties();
+    p.put(DistributionConfig.STATISTIC_SAMPLING_ENABLED_NAME, "true");
+    p.put(DistributionConfig.STATISTIC_ARCHIVE_FILE_NAME, "multicast");
+    p.put(DistributionConfig.MCAST_PORT_NAME, mcastport);
+    p.put(DistributionConfig.MCAST_TTL_NAME, mcastttl);
+    p.put(DistributionConfig.LOCATORS_NAME, "localhost[" + locatorPort +"]");
+    p.put(DistributionConfig.LOG_LEVEL_NAME, "info");
+    return p;
+  } 
+  
+  private void validateMulticastOpsAfterRegionOps() {
+    int writes = getGemfireCache().getDistributionManager().getStats().getMcastWrites();
+    int reads = getGemfireCache().getDistributionManager().getStats().getMcastReads();
+    assertTrue("Should have multicast writes or reads. Writes=  " + writes +  " ,read= " + reads, 
+        writes > 0 || reads > 0);
+  }
+  
+  private void validateMulticastOpsBeforeRegionOps() {
+    int writes = getGemfireCache().getDistributionManager().getStats().getMcastWrites();
+    int reads = getGemfireCache().getDistributionManager().getStats().getMcastReads();
+    int total = writes + reads;
+    assertTrue("Should not have any multicast writes or reads before region ops. Writes=  " + writes +  " ,read= " + reads, 
+        total == 0);
+  }
+  
+  private int startLocator() {
+  final int [] ports = AvailablePortHelper.getRandomAvailableTCPPorts(3);
+  final int locatorPort = ports[0];
+  
+  VM locator1Vm = Host.getHost(0).getVM(locatorVM);;
+    locator1Vm.invoke(new SerializableCallable() {
+      @Override
+      public Object call() {
+        final File locatorLogFile = new File(testName + "-locator-" + locatorPort + ".log");
+        final Properties locatorProps = new Properties();
+        locatorProps.setProperty(DistributionConfig.NAME_NAME, "LocatorWithMcast");
+        locatorProps.setProperty(DistributionConfig.MCAST_PORT_NAME, mcastport);
+        locatorProps.setProperty(DistributionConfig.MCAST_TTL_NAME, mcastttl);
+        locatorProps.setProperty(DistributionConfig.LOG_LEVEL_NAME, "info");
+        //locatorProps.setProperty(DistributionConfig.ENABLE_CLUSTER_CONFIGURATION_NAME, "true");
+        try {
+          final InternalLocator locator = (InternalLocator) Locator.startLocatorAndDS(locatorPort, null, null,
+              locatorProps);
+          System.out.println("test Locator started " + locatorPort);
+           } catch (IOException ioex) {
+          fail("Unable to create a locator with a shared configuration");
+        }
+  
+        return null;
+      }
+    });
+    return locatorPort;
+  }
+  
+  private void closeLocator() {
+    VM locator1Vm = Host.getHost(0).getVM(locatorVM);;
+    SerializableRunnable locatorCleanup = new SerializableRunnable() {
+      @Override
+      public void run() {
+        System.out.println("test Locator closing " + locatorPort);;
+        InternalLocator locator = InternalLocator.getLocator();
+        if (locator != null ) {
+          locator.stop();
+          System.out.println("test Locator closed " + locatorPort);;
+        }
+      }
+    };
+    locator1Vm.invoke(locatorCleanup);
+  }
+  
+}


[15/48] incubator-geode git commit: GEODE-503: Addresses config passwords written to logs

Posted by kl...@apache.org.
GEODE-503: Addresses config passwords written to logs

Prevents configuration passwords from being written to log files
for keystores used by SSL or any config parameter with the
keyword password in its name.

Adds unit test to validate AbstractConfigJUnitTest


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

Branch: refs/heads/feature/GEODE-217
Commit: 11c62f232014d4c93cf3c625b31b1a3139613818
Parents: 34eb0fe
Author: Vince Ford <vf...@apache.org>
Authored: Thu Dec 10 11:01:13 2015 -0800
Committer: Vince Ford <vf...@apache.org>
Committed: Thu Dec 10 11:46:46 2015 -0800

----------------------------------------------------------------------
 .../gemfire/internal/AbstractConfig.java        |   4 -
 .../internal/AbstractConfigJUnitTest.java       | 114 +++++++++++++++++++
 2 files changed, 114 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/11c62f23/gemfire-core/src/main/java/com/gemstone/gemfire/internal/AbstractConfig.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/AbstractConfig.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/AbstractConfig.java
index 2f2f0f7..ddf2970 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/AbstractConfig.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/AbstractConfig.java
@@ -204,10 +204,6 @@ public abstract class AbstractConfig implements Config {
   }
   
   private boolean okToDisplayPropertyValue(String attName) {
-    if (AbstractDistributionConfig.isWellKnownAttribute(attName)) {
-      // it is always ok to display the well know attributes
-      return true;
-    }
     if (attName.startsWith(DistributionConfig.SECURITY_PREFIX_NAME)) {
       return false;
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/11c62f23/gemfire-core/src/test/java/com/gemstone/gemfire/internal/AbstractConfigJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/AbstractConfigJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/AbstractConfigJUnitTest.java
new file mode 100644
index 0000000..80c92e6
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/AbstractConfigJUnitTest.java
@@ -0,0 +1,114 @@
+/*
+ * 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 static org.junit.Assert.*;
+
+import org.apache.logging.log4j.Logger;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import com.gemstone.gemfire.distributed.internal.DistributionConfigImpl;
+
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.Map;
+import java.util.Properties;
+import junit.framework.TestCase;
+import org.junit.experimental.categories.Category;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class AbstractConfigJUnitTest extends TestCase  {
+
+	@Test
+	public void testDisplayPropertyValue(){
+		AbstractConfigTestClass actc=new AbstractConfigTestClass();
+        Method method=null;
+        try{
+            method=actc.getClass().getSuperclass().getDeclaredMethod("okToDisplayPropertyValue",String.class);
+            method.setAccessible(true);
+            assertFalse((Boolean) method.invoke(actc, "password"));
+            assertFalse((Boolean)method.invoke(actc,"cluster-ssl-truststore-password"));
+            assertTrue((Boolean) method.invoke(actc, "cluster-ssl-enabled"));
+            assertFalse((Boolean)method.invoke(actc,"gateway-ssl-truststore-password"));
+            assertFalse((Boolean)method.invoke(actc,"server-ssl-keystore-password"));
+            assertTrue((Boolean) method.invoke(actc, "ssl-enabled"));
+            assertTrue((Boolean)method.invoke(actc,"conserve-sockets"));
+            assertFalse((Boolean)method.invoke(actc,"javax.net.ssl.keyStorePassword"));
+            assertFalse((Boolean)method.invoke(actc,"javax.net.ssl.keyStoreType"));
+            assertFalse((Boolean)method.invoke(actc,"sysprop-value"));
+        } catch (NoSuchMethodException e) {
+            e.printStackTrace();
+        } catch (InvocationTargetException e) {
+            e.printStackTrace();
+        } catch (IllegalAccessException e) {
+            e.printStackTrace();
+        } catch(Exception e){
+            e.printStackTrace();
+        }
+    }
+
+}
+
+class AbstractConfigTestClass extends AbstractConfig{
+
+
+	@Override
+	protected Map getAttDescMap() {
+		return null;
+	}
+
+	@Override
+	protected Map<String, ConfigSource> getAttSourceMap() {
+		return null;
+	}
+
+	@Override
+	public Object getAttributeObject(String attName) {
+		return null;
+	}
+
+	@Override
+	public void setAttributeObject(String attName, Object attValue, ConfigSource source) {
+
+	}
+
+	@Override
+	public boolean isAttributeModifiable(String attName) {
+		return false;
+	}
+
+	@Override
+	public Class getAttributeType(String attName) {
+		return null;
+	}
+
+	@Override
+	public String[] getAttributeNames() {
+		return new String[0];
+	}
+
+	@Override
+	public String[] getSpecificAttributeNames() {
+		return new String[0];
+	}
+}
+


[10/48] incubator-geode git commit: GEODE-608: Add rat task as a depedency for the check task

Posted by kl...@apache.org.
GEODE-608: Add rat task as a depedency for the check task

The rat constraints will now be applied whenever the check task
is performed (typically via `gradle build`).


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

Branch: refs/heads/feature/GEODE-217
Commit: e45539a831fb57c6fdcaafab52463790ee491f71
Parents: a31c8fb
Author: Anthony Baker <ab...@pivotal.io>
Authored: Wed Dec 2 08:45:11 2015 -0800
Committer: Anthony Baker <ab...@pivotal.io>
Committed: Thu Dec 10 09:44:57 2015 -0800

----------------------------------------------------------------------
 build.gradle | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e45539a8/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index babdb8b..6045b72 100755
--- a/build.gradle
+++ b/build.gradle
@@ -567,7 +567,7 @@ subprojects {
     description 'Run this task before checking in code to validate changes. This task combines the following tasks: build, integrationTest, and distributedTest'
   }
 
-  check.dependsOn checkMissedTests
+  check.dependsOn checkMissedTests, rat
   combineReports.mustRunAfter check, test, integrationTest, distributedTest, checkMissedTests
   build.finalizedBy combineReports
   check.finalizedBy combineReports


[44/48] 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 kl...@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;
-}


[09/48] incubator-geode git commit: GEODE-608: Removed exclusions from licensing check

Posted by kl...@apache.org.
GEODE-608: Removed exclusions from licensing check

XML, script, and build files are now checked for source license
headers.


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

Branch: refs/heads/feature/GEODE-217
Commit: a31c8fb9ecfc7ac960fc4f1f83e06a0fb3e1cd9b
Parents: fddd33f
Author: Anthony Baker <ab...@pivotal.io>
Authored: Tue Dec 1 21:09:20 2015 -0800
Committer: Anthony Baker <ab...@pivotal.io>
Committed: Thu Dec 10 09:44:57 2015 -0800

----------------------------------------------------------------------
 build.gradle | 48 ++++--------------------------------------------
 1 file changed, 4 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a31c8fb9/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index 4d0216b..babdb8b 100755
--- a/build.gradle
+++ b/build.gradle
@@ -21,16 +21,14 @@ rat {
     
     // gradle
     '**/.gradle/**',
-    '**/build.gradle',
     'gradlew',
     'gradlew.bat',
-    'gradle.properties',
-    'settings.gradle',
-    'gradle/dependency-versions.properties',
     'gradle/wrapper/gradle-wrapper.properties',
     '**/build/**',
     
     // IDE
+    'etc/eclipseFormatterProfile.xml',
+    'etc/intellijIdeaCodeStyle.xml',
     '**/.project',
     '**/.classpath',
     '**/.settings/**',
@@ -54,51 +52,14 @@ rat {
     '**/*.xls',
     
     // other text files
-    'gemfire-assembly/src/main/dist/bin/gfsh',
-    'gemfire-assembly/src/main/dist/bin/gfsh-completion.bash',
-    'gemfire-assembly/src/main/dist/bin/gfsh.bat',
-    'gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/parse/fixantlr.sh',
     'gemfire-spark-connector/project/plugins.sbt',
     'gemfire-spark-connector/project/build.properties',
-    'gemfire-web-api/src/main/webapp/docs/o2c.html',
-    'gemfire-web-api/src/main/webapp/docs/index.html',
-    
-    // XML
-    'etc/eclipseFormatterProfile.xml',
-    'etc/intellijIdeaCodeStyle.xml',
-    'log4j2*.xml',
-    '**/example*cache.xml',
-    'gemfire-core/src/test/resources/**/*JUnitTest*.xml',
-    'gemfire-core/src/test/resources/**/IndexCreation.xml',
-    'gemfire-core/src/test/resources/**/PRIndexCreation.xml',
-    'gemfire-core/src/test/resources/**/PartitionRegionCacheExample*.xml',
-    'gemfire-core/src/test/resources/**/attributesUnordered.xml',
-    'gemfire-core/src/test/resources/**/bad*.xml',
-    'gemfire-core/src/test/resources/**/bug44710.xml',
-    'gemfire-core/src/test/resources/**/cachejta.xml',
-    'gemfire-core/src/test/resources/**/cachequeryindex*.xml',
-    'gemfire-core/src/test/resources/**/callback*.xml',
-    'gemfire-core/src/test/resources/**/coLocation*.xml',
-    'gemfire-core/src/test/resources/**/ewtest.xml',
-    'gemfire-core/src/test/resources/**/incorrect*.xml',
-    'gemfire-core/src/test/resources/**/index-creation-*.xml',
-    'gemfire-core/src/test/resources/**/index-recovery-overflow.xml',
-    'gemfire-core/src/test/resources/**/loaderNotLoader.xml',
-    'gemfire-core/src/test/resources/**/malformed.xml',
-    'gemfire-core/src/test/resources/**/mixed_diskstore_disk*.xml',
-    'gemfire-core/src/test/resources/**/namedAttributes.xml',
-    'gemfire-core/src/test/resources/**/partitioned*.xml',
-    'gemfire-core/src/test/resources/**/same*.xml',
-    'gemfire-core/src/test/resources/**/spring-gemfire-context.xml',
-    'gemfire-core/src/test/resources/**/test*.xml',
-    'gemfire-core/src/test/resources/**/unknownNamedAttributes.xml',
     
     // 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',
-    'gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/parse/oql.g',
     
     // Service META-INF
     '**/META-INF/services/org.xml.sax.ext.EntityResolver2',
@@ -115,9 +76,6 @@ rat {
     // Public Domain http://meyerweb.com/eric/tools/css/reset/
     'gemfire-web-api/src/main/webapp/docs/css/reset.css',
 
-    // Public Domain - http://creativecommons.org/licenses/publicdomain
-    'SynchronousQueueNoSpin.java',
-
     // JSON License - permissive, used for Good, not Evil
     'gemfire-json/src/main/java/org/json/CDL.java',
     'gemfire-json/src/main/java/org/json/Cookie.java',
@@ -155,6 +113,8 @@ rat {
     'gemfire-web-api/src/main/webapp/docs/lib/highlight.7.3.pack.js',
 
     // Apache License
+    'gemfire-web-api/src/main/webapp/docs/o2c.html',
+    'gemfire-web-api/src/main/webapp/docs/index.html',
     'gemfire-web-api/src/main/webapp/docs/lib/swagger-oauth.js',
     'gemfire-web-api/src/main/webapp/docs/lib/swagger.js',
     'gemfire-web-api/src/main/webapp/docs/css/screen.css',


[40/48] 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 kl...@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);


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

Posted by kl...@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


[11/48] incubator-geode git commit: Add unit tests for GMSHealthMonitor tcp check

Posted by kl...@apache.org.
Add unit tests for GMSHealthMonitor tcp check


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

Branch: refs/heads/feature/GEODE-217
Commit: 7cbb5db091107267da87243259c09d9d047ffbbf
Parents: 35394ef
Author: Jason Huynh <hu...@gmail.com>
Authored: Thu Dec 10 10:02:24 2015 -0800
Committer: Jason Huynh <hu...@gmail.com>
Committed: Thu Dec 10 10:17:00 2015 -0800

----------------------------------------------------------------------
 .../membership/gms/fd/GMSHealthMonitor.java     | 34 ++++++++----
 .../gms/fd/GMSHealthMonitorJUnitTest.java       | 57 ++++++++++++++++++--
 2 files changed, 76 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cbb5db0/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 cc64f9b..7709114 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
@@ -472,6 +472,7 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
     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.
@@ -486,6 +487,26 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
       logger.debug("Checking member {} with TCP socket connection {}:{}.", suspectMember, suspectMember.getInetAddress(), port);
       clientSocket = SocketCreator.getDefaultInstance().connect(suspectMember.getInetAddress(), port,
           (int)memberTimeout, new ConnectTimeoutTask(services.getTimer(), memberTimeout), false, -1, false);
+      return doTCPCheckMember(suspectMember, clientSocket);
+    }
+    catch (IOException e) {
+      logger.debug("Unexpected exception", e);
+    } 
+    finally {
+      try {
+        if (clientSocket != null) {
+          clientSocket.close();
+        }
+      } catch (IOException e) {
+        logger.trace("Unexpected exception", e);
+      }
+    }
+    return false;
+  }
+
+  //Package protected for testing purposes
+  boolean doTCPCheckMember(InternalDistributedMember suspectMember, Socket clientSocket) {
+    try {
       if (clientSocket.isConnected()) {
         clientSocket.setSoTimeout((int) services.getConfig().getMemberTimeout());
         InputStream in = clientSocket.getInputStream();
@@ -513,17 +534,8 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
       logger.debug("tcp/ip connection timed out");
       return false;
     } catch (IOException e) {
-      logger.debug("Unexpected exception", e);
-    } finally {
-      try {
-        if (clientSocket != null) {
-          clientSocket.close();
-        }
-      } catch (IOException e) {
-        logger.trace("Unexpected exception", e);
-      }
-    }
-
+      logger.trace("Unexpected exception", e);
+    } 
     return false;
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cbb5db0/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 c4ac3a6..eb17ca8 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
@@ -79,7 +79,7 @@ public class GMSHealthMonitorJUnitTest {
 
   @Before
   public void initMocks() throws UnknownHostException {
-    System.setProperty("gemfire.bind-address", "localhost");
+    //System.setProperty("gemfire.bind-address", "localhost");
     mockDistConfig = mock(DistributionConfig.class);
     mockConfig = mock(ServiceConfig.class);
     messenger = mock(Messenger.class);
@@ -121,7 +121,7 @@ public class GMSHealthMonitorJUnitTest {
   @After
   public void tearDown() {
     gmsHealthMonitor.stop();
-    System.getProperties().remove("gemfire.bind-address");
+    //System.getProperties().remove("gemfire.bind-address");
   }
 
   @Test
@@ -562,12 +562,61 @@ public class GMSHealthMonitorJUnitTest {
     when(smm.getDSFID()).thenCallRealMethod();
     gmsHealthMonitor.processMessage(smm);
   }
-
-  private GMSMember createGMSMember(short version, int viewId, long msb, long lsb) {
+  
+  @Test
+  public void testDoTCPCheckMemberWithOkStatus() throws Exception {
+    executeTestDoTCPCheck(GMSHealthMonitor.OK, true);
+  }
+  
+  @Test
+  public void testDoTCPCheckMemberWithErrorStatus() throws Exception {
+    executeTestDoTCPCheck(GMSHealthMonitor.ERROR, false);
+  }
+  
+  @Test
+  public void testDoTCPCheckMemberWithUnkownStatus() throws Exception {
+    executeTestDoTCPCheck(GMSHealthMonitor.ERROR + 100, false);
+  }
+  
+  private void executeTestDoTCPCheck(int receivedStatus, boolean expectedResult) throws Exception {
+    InternalDistributedMember otherMember = createInternalDistributedMember(Version.CURRENT_ORDINAL, 0, 1, 1);
+    InternalDistributedMember gmsMember = createInternalDistributedMember(Version.CURRENT_ORDINAL, 0, 1, 1);
+    
+    //Set up the incoming/received bytes.  We just wrap output streams and write out the gms member information
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    baos.write(receivedStatus);
+    
+    byte[] receivedBytes = baos.toByteArray();
+    InputStream mockInputStream = new ByteArrayInputStream(receivedBytes);
+    
+    Socket fakeSocket = mock(Socket.class);
+    ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+    when(fakeSocket.getInputStream()).thenReturn(mockInputStream);
+    when(fakeSocket.getOutputStream()).thenReturn(outputStream);
+    when(fakeSocket.isConnected()).thenReturn(true);
+    
+    Assert.assertEquals(expectedResult, gmsHealthMonitor.doTCPCheckMember(otherMember, fakeSocket));
+    
+    //we can check to see if the gms member information was written out by the tcp check
+    byte[] bytesWritten = outputStream.toByteArray();
+    Assert.assertArrayEquals(writeMemberToBytes((GMSMember)gmsMember.getNetMember()), bytesWritten);
+  }
+  
+  private InternalDistributedMember createInternalDistributedMember(short version, int viewId, long msb, long lsb) throws UnknownHostException{
+    GMSMember gmsMember = createGMSMember(version, viewId, msb, lsb);
+    InternalDistributedMember idm = new InternalDistributedMember("localhost", 9000, Version.CURRENT, gmsMember);
+    //We set to our expected test viewId in the IDM as well as reseting the gms member
+    idm.setVmViewId(viewId);
+    gmsMember.setBirthViewId(viewId);
+    return idm;
+  }
+  
+  private GMSMember createGMSMember(short version, int viewId, long msb, long lsb) throws UnknownHostException{
     GMSMember gmsMember = new GMSMember();
     gmsMember.setVersionOrdinal(version);
     gmsMember.setBirthViewId(viewId);
     gmsMember.setUUID(new UUID(msb, lsb));
+    gmsMember.setInetAddr(InetAddress.getLocalHost());
     return gmsMember;
   }
   


[03/48] incubator-geode git commit: GEODE 608: Initial cut at adding RAT to build

Posted by kl...@apache.org.
GEODE 608: Initial cut at adding RAT to build

Add the RAT plugin to check for license headers in source files.  This
is a first pass that excludess generated files, typical IDE files, etc.
Create the report using `gradle rat`.  As GEODE-18 continues we can
update the excludes list appropriately.  Eventually this task should
pass.


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

Branch: refs/heads/feature/GEODE-217
Commit: f133ff1c86be467bd4ecda5f8d24b7fe969531b7
Parents: a5906e5
Author: Anthony Baker <ab...@pivotal.io>
Authored: Sat Nov 28 08:07:31 2015 -0800
Committer: Anthony Baker <ab...@pivotal.io>
Committed: Thu Dec 10 09:44:56 2015 -0800

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


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f133ff1c/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index 0c94573..6f5c2a3 100755
--- a/build.gradle
+++ b/build.gradle
@@ -5,11 +5,31 @@ buildscript {
     }
   }
   dependencies {
+    classpath "gradle.plugin.org.nosphere.apache:creadur-rat-gradle:0.2.0"
     classpath "org.ajoberstar:gradle-git:1.3.2"
   }
 }
 
 apply plugin: 'wrapper'
+apply plugin: "org.nosphere.apache.rat"
+
+rat {
+  excludes = [
+    '.git/**',
+    '**/.gitignore',
+    '**/.gradle/**',
+    '.gradle',
+    '**/build/**',
+    '**/.project',
+    '**/.classpath',
+    '**/.settings/**',
+    '**/build-eclipse/**',
+    '*.iml',
+    '.idea/**',
+
+    '**/doc-files/*.fig'
+  ]
+}
 
 // Load all properties in dependency-version.properties as project properties, so all projects can read them
 Properties dependencyVersions = new Properties()


[38/48] 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 kl...@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">


[34/48] incubator-geode git commit: Revert "Removing TCPConduit's Stub ID class"

Posted by kl...@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-217
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);
+  }
+}


[32/48] incubator-geode git commit: GEODE-654: Changes LIFO queue to remove touched entry

Posted by kl...@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-217
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;
+  } 
+}
+


[22/48] incubator-geode git commit: GEODE-647: Adding some test debugging output

Posted by kl...@apache.org.
GEODE-647: Adding some test debugging output


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

Branch: refs/heads/feature/GEODE-217
Commit: 9d047b4df4af8d60161cab95fc9a1af5e6d956ae
Parents: a84780e
Author: Jens Deppe <jd...@pivotal.io>
Authored: Fri Dec 11 08:31:51 2015 -0800
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Fri Dec 11 08:32:41 2015 -0800

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


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9d047b4d/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..c536147 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,8 +150,11 @@ 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());
@@ -167,6 +170,16 @@ 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


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

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


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

Branch: refs/heads/feature/GEODE-217
Commit: afed665770187d554f24ca84670cbc6a02c9d35e
Parents: 0e62bc0 ec9d16a
Author: Kirk Lund <kl...@pivotal.io>
Authored: Mon Dec 14 10:05:31 2015 -0800
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Mon Dec 14 10:05:31 2015 -0800

----------------------------------------------------------------------
 build.gradle                                    |   21 +-
 gemfire-assembly/build.gradle                   |   17 +
 gemfire-assembly/src/main/dist/bin/gfsh         |   14 +
 .../src/main/dist/bin/gfsh-completion.bash      |   15 +
 gemfire-assembly/src/main/dist/bin/gfsh.bat     |   14 +
 .../LocatorLauncherAssemblyJUnitTest.java       |  157 ++
 .../SharedConfigurationEndToEndDUnitTest.java   |   21 +-
 gemfire-common/build.gradle                     |   17 +
 gemfire-core/build.gradle                       |   17 +
 gemfire-core/src/jca/ra.xml                     |   17 +-
 .../DistributedSystemHealthEvaluator.java       |    2 +-
 .../client/doc-files/example-client-cache.xml   |   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 +
 .../cache/query/internal/parse/fixantlr.sh      |   15 +
 .../gemfire/cache/query/internal/parse/oql.g    |   17 +
 .../gemfire/distributed/AbstractLauncher.java   |    2 +-
 .../gemfire/distributed/DistributedSystem.java  |  208 ++-
 .../gemfire/distributed/internal/DMStats.java   |    1 +
 .../internal/DistributionAdvisor.java           |    2 +-
 .../internal/DistributionManager.java           |   24 +-
 .../distributed/internal/DistributionStats.java |    3 +
 .../internal/InternalDistributedSystem.java     |   19 +-
 .../internal/LonerDistributionManager.java      |    2 +
 .../internal/MembershipListener.java            |    3 +-
 .../distributed/internal/ProductUseLog.java     |    2 +-
 .../distributed/internal/ReplyProcessor21.java  |    4 +-
 .../internal/direct/DirectChannel.java          |    2 +-
 .../internal/direct/DirectChannelListener.java  |   16 +
 .../internal/locks/DLockGrantor.java            |    2 +-
 .../DistributedMembershipListener.java          |    3 +-
 .../internal/membership/MembershipManager.java  |   18 +-
 .../internal/membership/NetView.java            |   19 +
 .../internal/membership/gms/GMSMember.java      |    8 +
 .../internal/membership/gms/ServiceConfig.java  |   21 +-
 .../internal/membership/gms/Services.java       |   12 +-
 .../internal/membership/gms/SuspectMember.java  |    9 +-
 .../membership/gms/auth/GMSAuthenticator.java   |    2 +-
 .../membership/gms/fd/GMSHealthMonitor.java     |   38 +-
 .../membership/gms/interfaces/Messenger.java    |   22 +
 .../membership/gms/interfaces/Service.java      |    3 +-
 .../gms/locator/FindCoordinatorRequest.java     |   33 +
 .../membership/gms/membership/GMSJoinLeave.java |  165 +-
 .../gms/messenger/JGroupsMessenger.java         |   71 +-
 .../gms/mgr/GMSMembershipManager.java           |   45 +-
 .../gemfire/internal/AbstractConfig.java        |    4 -
 .../admin/remote/RemoteGfManagerAgent.java      |    2 +-
 .../internal/cache/DistributedRegion.java       |    2 +-
 .../internal/cache/InitialImageFlowControl.java |    2 +-
 .../internal/cache/PRHARedundancyProvider.java  |    4 +-
 .../internal/cache/PartitionedRegion.java       |    2 +-
 .../internal/cache/PartitionedRegionHelper.java |    2 +-
 .../cache/SearchLoadAndWriteProcessor.java      |    2 +-
 .../internal/cache/StateFlushOperation.java     |    6 +-
 .../gemfire/internal/cache/TXCommitMessage.java |    2 +-
 .../internal/cache/TXFarSideCMTracker.java      |    2 +-
 .../gemfire/internal/cache/TXManagerImpl.java   |    2 +-
 .../internal/cache/lru/NewLIFOClockHand.java    |   78 +-
 .../internal/cache/lru/NewLRUClockHand.java     |   14 +-
 .../PartitionedRegionRebalanceOp.java           |    2 +-
 .../cache/persistence/BackupManager.java        |    2 +-
 .../persistence/PersistenceAdvisorImpl.java     |    2 +-
 .../persistence/PersistentMemberManager.java    |    2 +-
 .../cache/versions/RegionVersionVector.java     |    2 +-
 .../gemfire/internal/offheap/DataAsAddress.java |   14 +-
 .../offheap/OffHeapRegionEntryHelper.java       |   51 +-
 .../internal/offheap/RefCountChangeInfo.java    |   77 +-
 .../internal/offheap/ReferenceCountHelper.java  |   14 +-
 .../gemfire/internal/tcp/Connection.java        |    6 +-
 .../gemfire/internal/tcp/ConnectionTable.java   |    2 +-
 .../gemfire/internal/tcp/TCPConduit.java        |    2 +-
 .../management/internal/FederatingManager.java  |    5 +-
 .../internal/ManagementMembershipListener.java  |    4 +-
 .../internal/SystemManagementService.java       |    7 +-
 .../internal/beans/MBeanAggregator.java         |    2 +-
 .../tools/gfsh/app/windowsbindings.properties   |   15 +
 .../DistributedMulticastRegionDUnitTest.java    |  213 +++
 .../AbstractLauncherIntegrationJUnitTest.java   |   71 +
 .../distributed/AbstractLauncherJUnitTest.java  |   39 +-
 .../distributed/CommonLauncherTestSuite.java    |   65 -
 .../DistributedSystemIntegrationJUnitTest.java  |   91 +
 .../distributed/DistributedSystemJUnitTest.java |   78 +
 .../gemfire/distributed/LocatorDUnitTest.java   |    9 +-
 .../LocatorLauncherIntegrationJUnitTest.java    |  248 +++
 .../distributed/LocatorLauncherJUnitTest.java   |  154 +-
 .../ServerLauncherIntegrationJUnitTest.java     |  312 ++++
 .../distributed/ServerLauncherJUnitTest.java    |  183 +-
 .../internal/DistributionManagerDUnitTest.java  |   43 +-
 .../membership/MembershipJUnitTest.java         |  130 +-
 .../internal/membership/NetViewJUnitTest.java   |   82 +-
 .../membership/gms/GMSMemberJUnitTest.java      |   16 +
 .../gms/fd/GMSHealthMonitorJUnitTest.java       |  111 +-
 .../gms/membership/GMSJoinLeaveJUnitTest.java   |  150 +-
 .../messenger/JGroupsMessengerJUnitTest.java    |   64 +-
 .../gms/mgr/GMSMembershipManagerJUnitTest.java  |   82 +-
 .../internal/AbstractConfigJUnitTest.java       |  114 ++
 .../internal/DataSerializableJUnitTest.java     |   35 +-
 ...victionAlgoMemoryEnabledRegionJUnitTest.java |  119 ++
 .../InstantiatorPropagationDUnitTest.java       |   16 +
 .../cache/wan/AsyncEventQueueTestBase.java      |   20 +-
 .../asyncqueue/AsyncEventListenerDUnitTest.java |   21 +-
 .../AsyncEventListenerOffHeapDUnitTest.java     |   16 +
 .../AsyncEventQueueStatsDUnitTest.java          |   21 +-
 .../ConcurrentAsyncEventQueueDUnitTest.java     |   24 +-
 ...ncurrentAsyncEventQueueOffHeapDUnitTest.java |   16 +
 .../CommonParallelAsyncEventQueueDUnitTest.java |   24 +-
 ...ParallelAsyncEventQueueOffHeapDUnitTest.java |   16 +
 .../offheap/AbstractStoredObjectTestBase.java   |  203 +++
 .../offheap/DataAsAddressJUnitTest.java         |  368 +++++
 .../offheap/LifecycleListenerJUnitTest.java     |  222 +++
 .../offheap/RefCountChangeInfoJUnitTest.java    |  140 +-
 ...moryAllocatorLifecycleListenerJUnitTest.java |  147 --
 .../internal/cli/GfshParserJUnitTest.java       |   50 +-
 .../internal/cli/HeadlessGfshJUnitTest.java     |    7 +-
 .../cli/commands/MemberCommandsDUnitTest.java   |    1 +
 .../test/process/ProcessOutputReader.java       |  101 +-
 .../test/process/ProcessStreamReader.java       |   42 +-
 .../gemfire/test/process/ProcessWrapper.java    |   10 +-
 .../ClientCacheFactoryJUnitTest_single_pool.xml |   17 +
 .../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 +
 .../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   |   16 +
 ...testDTDFallbackWithNonEnglishLocal.cache.xml |   17 +
 .../gemstone/gemfire/internal/jta/cachejta.xml  |   17 +
 .../src/test/resources/jta/cachejta.xml         |   17 +
 .../resources/spring/spring-gemfire-context.xml |   17 +
 gemfire-lucene/build.gradle                     |   17 +
 ...erIntegrationJUnitTest.createIndex.cache.xml |   17 +
 ...serIntegrationJUnitTest.parseIndex.cache.xml |   17 +
 gemfire-rebalancer/build.gradle                 |   17 +
 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 ----
 .../src/it/resources/test-regions.xml           |   17 +
 .../src/it/resources/test-retrieve-regions.xml  |   17 +
 gemfire-web-api/build.gradle                    |   17 +
 gemfire-web/build.gradle                        |   17 +
 gradle.properties                               |   14 +
 gradle/dependency-versions.properties           |   21 +-
 gradle/rat.gradle                               |  152 ++
 settings.gradle                                 |   16 +
 227 files changed, 5518 insertions(+), 9810 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/afed6657/gradle/dependency-versions.properties
----------------------------------------------------------------------
diff --cc gradle/dependency-versions.properties
index 160ff5b,8ea38c3..3833f54
--- a/gradle/dependency-versions.properties
+++ b/gradle/dependency-versions.properties
@@@ -6,8 -21,8 +21,8 @@@ activation.version = 1.1.
  annotations.version = 3.0.0
  antlr.version = 2.7.7
  asm.version = 5.0.3
- assertj-core.version = 3.1.0
- awaitility.version = 1.6.5
+ 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
@@@ -64,4 -79,4 +79,4 @@@ stephenc-findbugs.version = 1.3.9-
  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


[28/48] incubator-geode git commit: Removing TCPConduit's Stub ID class

Posted by kl...@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


[23/48] incubator-geode git commit: GEODE-624: add unit test for LifeCycleListener

Posted by kl...@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-217
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));
-    }
-  }
-}


[47/48] incubator-geode git commit: GEODE-650: Fix DiskStoreCOmmandsDUnitTest - previous tests not cleaning up properly

Posted by kl...@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-217
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;
   }
 


[42/48] 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 kl...@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