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 2016/02/08 22:58:46 UTC

[19/50] [abbrv] incubator-geode git commit: GEODE-14: Integration of GemFire Session Replication and Hibernate modules

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/48552465/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/DeltaSession.java
----------------------------------------------------------------------
diff --git a/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/DeltaSession.java b/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/DeltaSession.java
new file mode 100644
index 0000000..c81a232
--- /dev/null
+++ b/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/DeltaSession.java
@@ -0,0 +1,597 @@
+/*
+* 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.modules.session.catalina;
+
+import com.gemstone.gemfire.DataSerializable;
+import com.gemstone.gemfire.DataSerializer;
+import com.gemstone.gemfire.Delta;
+import com.gemstone.gemfire.InvalidDeltaException;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.internal.cache.lru.Sizeable;
+import com.gemstone.gemfire.internal.util.BlobHelper;
+import com.gemstone.gemfire.modules.gatewaydelta.GatewayDelta;
+import com.gemstone.gemfire.modules.gatewaydelta.GatewayDeltaEvent;
+import com.gemstone.gemfire.modules.session.catalina.internal.DeltaSessionAttributeEvent;
+import com.gemstone.gemfire.modules.session.catalina.internal.DeltaSessionAttributeEventBatch;
+import com.gemstone.gemfire.modules.session.catalina.internal.DeltaSessionDestroyAttributeEvent;
+import com.gemstone.gemfire.modules.session.catalina.internal.DeltaSessionUpdateAttributeEvent;
+import org.apache.catalina.Manager;
+import org.apache.catalina.ha.session.SerializablePrincipal;
+import org.apache.catalina.realm.GenericPrincipal;
+import org.apache.catalina.security.SecurityUtil;
+import org.apache.catalina.session.StandardSession;
+import org.apache.juli.logging.Log;
+import org.apache.juli.logging.LogFactory;
+
+import javax.servlet.http.HttpSession;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.InputStream;
+import java.security.AccessController;
+import java.security.Principal;
+import java.security.PrivilegedAction;
+import java.util.ArrayList;
+import java.util.Enumeration;
+import java.util.Hashtable;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+@SuppressWarnings("serial")
+public class DeltaSession extends StandardSession implements DataSerializable, Delta, GatewayDelta, Sizeable {
+
+  private transient Region<String, HttpSession> operatingRegion;
+
+  private String sessionRegionName;
+
+  private String contextName;
+
+  private boolean hasDelta;
+
+  private boolean applyRemotely;
+
+  private boolean enableGatewayDeltaReplication;
+
+  private transient final Object changeLock = new Object();
+
+  private final List<DeltaSessionAttributeEvent> eventQueue = new ArrayList<DeltaSessionAttributeEvent>();
+
+  private transient GatewayDeltaEvent currentGatewayDeltaEvent;
+
+  private transient boolean expired = false;
+
+  private transient boolean preferDeserializedForm = true;
+
+  private byte[] serializedPrincipal;
+
+  private final Log LOG = LogFactory.getLog(DeltaSession.class.getName());
+
+  /**
+   * The string manager for this package.
+   */
+//  protected static StringManager STRING_MANAGER =
+//    StringManager.getManager("com.gemstone.gemfire.modules.session.catalina");
+
+  /**
+   * Construct a new <code>Session</code> associated with no <code>Manager</code>. The <code>Manager</code> will be
+   * assigned later using {@link #setOwner(Object)}.
+   */
+  public DeltaSession() {
+    super(null);
+  }
+
+  /**
+   * Construct a new Session associated with the specified Manager.
+   *
+   * @param manager The manager with which this Session is associated
+   */
+  public DeltaSession(Manager manager) {
+    super(manager);
+    setOwner(manager);
+  }
+
+  /**
+   * Return the <code>HttpSession</code> for which this object is the facade.
+   */
+  @SuppressWarnings("unchecked")
+  public HttpSession getSession() {
+    if (facade == null) {
+      if (SecurityUtil.isPackageProtectionEnabled()) {
+        final DeltaSession fsession = this;
+        facade = (DeltaSessionFacade) AccessController.doPrivileged(new PrivilegedAction() {
+          public Object run() {
+            return new DeltaSessionFacade(fsession);
+          }
+        });
+      } else {
+        facade = new DeltaSessionFacade(this);
+      }
+    }
+    return (facade);
+  }
+
+  public Principal getPrincipal() {
+    if (this.principal == null && this.serializedPrincipal != null) {
+      SerializablePrincipal sp = null;
+      try {
+        sp = (SerializablePrincipal) BlobHelper.deserializeBlob(this.serializedPrincipal);
+      } catch (Exception e) {
+        StringBuilder builder = new StringBuilder();
+        builder.append(this)
+            .append(
+                ": Serialized principal contains a byte[] that cannot be deserialized due to the following exception");
+        ((DeltaSessionManager) getManager()).getLogger().warn(builder.toString(), e);
+        return null;
+      }
+      this.principal = sp.getPrincipal(this.manager.getContainer().getRealm());
+      if (getManager() != null) {
+        DeltaSessionManager mgr = (DeltaSessionManager) getManager();
+        if (mgr.getLogger().isDebugEnabled()) {
+          mgr.getLogger().debug(this + ": Deserialized principal: " + this.principal);
+          //mgr.logCurrentStack();
+        }
+      }
+    }
+    return this.principal;
+  }
+
+  public void setPrincipal(Principal principal) {
+    super.setPrincipal(principal);
+
+    // Put the session into the region to serialize the principal
+    if (getManager() != null) {
+      // TODO convert this to a delta
+      getManager().add(this);
+      DeltaSessionManager mgr = (DeltaSessionManager) getManager();
+      if (mgr.getLogger().isDebugEnabled()) {
+        mgr.getLogger().debug(this + ": Cached principal: " + principal);
+        //mgr.logCurrentStack();
+      }
+    }
+  }
+
+  private byte[] getSerializedPrincipal() {
+    if (this.serializedPrincipal == null) {
+      if (this.principal != null && this.principal instanceof GenericPrincipal) {
+        GenericPrincipal gp = (GenericPrincipal) this.principal;
+        SerializablePrincipal sp = SerializablePrincipal.createPrincipal(gp);
+        this.serializedPrincipal = serialize(sp);
+        if (manager != null) {
+          DeltaSessionManager mgr = (DeltaSessionManager) getManager();
+          if (mgr.getLogger().isDebugEnabled()) {
+            mgr.getLogger().debug(this + ": Serialized principal: " + sp);
+            //mgr.logCurrentStack();
+          }
+        }
+      }
+    }
+    return this.serializedPrincipal;
+  }
+
+  protected Region<String, HttpSession> getOperatingRegion() {
+    // This region shouldn't be null when it is needed.
+    // It should have been set by the setOwner method.
+    return this.operatingRegion;
+  }
+
+  public boolean isCommitEnabled() {
+    DeltaSessionManager mgr = (DeltaSessionManager) getManager();
+    return mgr.isCommitValveEnabled();
+  }
+
+  public GatewayDeltaEvent getCurrentGatewayDeltaEvent() {
+    return this.currentGatewayDeltaEvent;
+  }
+
+  public void setCurrentGatewayDeltaEvent(GatewayDeltaEvent currentGatewayDeltaEvent) {
+    this.currentGatewayDeltaEvent = currentGatewayDeltaEvent;
+  }
+
+  @SuppressWarnings("unchecked")
+  public void setOwner(Object manager) {
+    if (manager instanceof DeltaSessionManager) {
+      DeltaSessionManager sessionManager = (DeltaSessionManager) manager;
+      this.manager = sessionManager;
+      initializeRegion(sessionManager);
+      this.hasDelta = false;
+      this.applyRemotely = false;
+      this.enableGatewayDeltaReplication = sessionManager.getEnableGatewayDeltaReplication();
+      this.preferDeserializedForm = sessionManager.getPreferDeserializedForm();
+
+      // Initialize transient variables
+      if (this.listeners == null) {
+        this.listeners = new ArrayList();
+      }
+
+      if (this.notes == null) {
+        this.notes = new Hashtable();
+      }
+
+      contextName = ((DeltaSessionManager) manager).getContainer().getName();
+    } else {
+      throw new IllegalArgumentException(this + ": The Manager must be an AbstractManager");
+    }
+  }
+
+  private void checkBackingCacheAvailable() {
+    if (!((SessionManager) getManager()).isBackingCacheAvailable()) {
+      throw new IllegalStateException("No backing cache server is available.");
+    }
+  }
+
+  public void setAttribute(String name, Object value, boolean notify) {
+    checkBackingCacheAvailable();
+    synchronized (this.changeLock) {
+      // Serialize the value
+      byte[] serializedValue = serialize(value);
+
+      // Store the attribute locally
+      if (this.preferDeserializedForm) {
+        super.setAttribute(name, value, true);
+      } else {
+        super.setAttribute(name, serializedValue, true);
+      }
+
+      if (serializedValue == null) {
+        return;
+      }
+
+      // Create the update attribute message
+      DeltaSessionAttributeEvent event = new DeltaSessionUpdateAttributeEvent(name, serializedValue);
+      queueAttributeEvent(event, true);
+
+      // Distribute the update
+      if (!isCommitEnabled()) {
+        putInRegion(getOperatingRegion(), true, null);
+      }
+    }
+  }
+
+  public void removeAttribute(String name, boolean notify) {
+    checkBackingCacheAvailable();
+    synchronized (this.changeLock) {
+      // Remove the attribute locally
+      super.removeAttribute(name, true);
+
+      // Create the destroy attribute message
+      DeltaSessionAttributeEvent event = new DeltaSessionDestroyAttributeEvent(name);
+      queueAttributeEvent(event, true);
+
+      // Distribute the update
+      if (!isCommitEnabled()) {
+        putInRegion(getOperatingRegion(), true, null);
+      }
+    }
+  }
+
+  public Object getAttribute(String name) {
+    checkBackingCacheAvailable();
+    Object value = super.getAttribute(name);
+
+    // If the attribute is a byte[] (meaning it came from the server),
+    // deserialize it and add it to attributes map before returning it.
+    if (value instanceof byte[]) {
+      try {
+        value = BlobHelper.deserializeBlob((byte[]) value);
+      } catch (Exception e) {
+        StringBuilder builder = new StringBuilder();
+        builder.append(this)
+            .append(": Attribute named ")
+            .append(name)
+            .append(" contains a byte[] that cannot be deserialized due to the following exception");
+        ((DeltaSessionManager) getManager()).getLogger().warn(builder.toString(), e);
+      }
+      if (this.preferDeserializedForm) {
+        localUpdateAttribute(name, value);
+      }
+    }
+
+    // Touch the session region if necessary. This is an asynchronous operation
+    // that prevents the session region from prematurely expiring a session that
+    // is only getting attributes.
+    ((DeltaSessionManager) getManager()).addSessionToTouch(getId());
+
+    return value;
+  }
+
+  public void invalidate() {
+    super.invalidate();
+    //getOperatingRegion().destroy(this.id, true); // already done in super (remove)
+    ((DeltaSessionManager) getManager()).getStatistics().incSessionsInvalidated();
+  }
+
+  public void processExpired() {
+    if (((DeltaSessionManager) getManager()).getLogger().isDebugEnabled()) {
+      ((DeltaSessionManager) getManager()).getLogger().debug(this + ": Expired");
+    }
+    // Set expired (so region.destroy is not called again)
+    setExpired(true);
+
+    // Do expire processing
+    expire();
+
+    // Update statistics
+    DeltaSessionManager manager = (DeltaSessionManager) getManager();
+    if (manager != null) {
+      manager.getStatistics().incSessionsExpired();
+    }
+  }
+
+  public void setMaxInactiveInterval(int interval) {
+    super.setMaxInactiveInterval(interval);
+  }
+
+  public void localUpdateAttribute(String name, Object value) {
+    super.setAttribute(name, value, false); // don't do notification since this is a replication
+  }
+
+  public void localDestroyAttribute(String name) {
+    super.removeAttribute(name, false); // don't do notification since this is a replication
+  }
+
+  public void applyAttributeEvents(Region<String, DeltaSession> region, List<DeltaSessionAttributeEvent> events) {
+    for (DeltaSessionAttributeEvent event : events) {
+      event.apply(this);
+      queueAttributeEvent(event, false);
+    }
+
+    putInRegion(region, false, true);
+  }
+
+  private void initializeRegion(DeltaSessionManager sessionManager) {
+    // Get the session region name
+    this.sessionRegionName = sessionManager.getRegionName();
+
+    // Get the operating region.
+    // If a P2P manager is used, then this will be a local region fronting the
+    // session region if local cache is enabled; otherwise, it will be the
+    // session region itself.
+    // If a CS manager is used, it will be the session proxy region.
+    this.operatingRegion = sessionManager.getSessionCache().getOperatingRegion();
+    if (sessionManager.getLogger().isDebugEnabled()) {
+      sessionManager.getLogger().debug(this + ": Set operating region: " + this.operatingRegion);
+    }
+  }
+
+  private void queueAttributeEvent(DeltaSessionAttributeEvent event, boolean checkAddToCurrentGatewayDelta) {
+    // Add to current gateway delta if necessary
+    if (checkAddToCurrentGatewayDelta) {
+      // If the manager has enabled gateway delta replication and is a P2P
+      // manager, the GatewayDeltaForwardCacheListener will be invoked in this
+      // VM. Add the event to the currentDelta.
+      DeltaSessionManager mgr = (DeltaSessionManager) this.manager;
+      if (this.enableGatewayDeltaReplication && mgr.isPeerToPeer()) {
+        // If commit is not enabled, add the event to the current batch; else,
+        // the current batch will be initialized to the events in the queue will
+        // be added at commit time.
+        if (!isCommitEnabled()) {
+          List<DeltaSessionAttributeEvent> events = new ArrayList<DeltaSessionAttributeEvent>();
+          events.add(event);
+          this.currentGatewayDeltaEvent = new DeltaSessionAttributeEventBatch(this.sessionRegionName, this.id, events);
+        }
+      }
+    }
+    this.eventQueue.add(event);
+  }
+
+  @SuppressWarnings("unchecked")
+  private void putInRegion(Region region, boolean applyRemotely, Object callbackArgument) {
+    this.hasDelta = true;
+    this.applyRemotely = applyRemotely;
+    region.put(this.id, this, callbackArgument);
+    this.eventQueue.clear();
+  }
+
+  public void commit() {
+    if (!isValidInternal()) throw new IllegalStateException("commit: Session " + getId() +
+        " already invalidated");
+//          (STRING_MANAGER.getString("deltaSession.commit.ise", getId()));
+
+    synchronized (this.changeLock) {
+      // Jens - there used to be a check to only perform this if the queue is
+      // empty, but we want this to always run so that the lastAccessedTime
+      // will be updated even when no attributes have been changed.
+      DeltaSessionManager mgr = (DeltaSessionManager) this.manager;
+      if (this.enableGatewayDeltaReplication && mgr.isPeerToPeer()) {
+        setCurrentGatewayDeltaEvent(
+            new DeltaSessionAttributeEventBatch(this.sessionRegionName, this.id, this.eventQueue));
+      }
+      this.hasDelta = true;
+      this.applyRemotely = true;
+      putInRegion(getOperatingRegion(), true, null);
+      this.eventQueue.clear();
+    }
+  }
+
+  public void abort() {
+    synchronized (this.changeLock) {
+      this.eventQueue.clear();
+    }
+  }
+
+  private void setExpired(boolean expired) {
+    this.expired = expired;
+  }
+
+  protected boolean getExpired() {
+    return this.expired;
+  }
+
+  public String getContextName() {
+    return contextName;
+  }
+
+  public boolean hasDelta() {
+    return this.hasDelta;
+  }
+
+  public void toDelta(DataOutput out) throws IOException {
+    // Write whether to apply the changes to another DS if necessary
+    out.writeBoolean(this.applyRemotely);
+
+    // Write the events
+    DataSerializer.writeArrayList((ArrayList) this.eventQueue, out);
+
+    out.writeLong(this.lastAccessedTime);
+    out.writeInt(this.maxInactiveInterval);
+  }
+
+  public void fromDelta(DataInput in) throws IOException, InvalidDeltaException {
+    // Read whether to apply the changes to another DS if necessary
+    this.applyRemotely = in.readBoolean();
+
+    // Read the events
+    List<DeltaSessionAttributeEvent> events = null;
+    try {
+      events = DataSerializer.readArrayList(in);
+    } catch (ClassNotFoundException e) {
+      throw new InvalidDeltaException(e);
+    }
+
+    // This allows for backwards compatibility with 2.1 clients
+    if (((InputStream) in).available() > 0) {
+      this.lastAccessedTime = in.readLong();
+      this.maxInactiveInterval = in.readInt();
+    }
+
+    // Iterate and apply the events
+    for (DeltaSessionAttributeEvent event : events) {
+      event.apply(this);
+    }
+
+    // Add the events to the gateway delta region if necessary
+    if (this.enableGatewayDeltaReplication && this.applyRemotely) {
+      setCurrentGatewayDeltaEvent(new DeltaSessionAttributeEventBatch(this.sessionRegionName, this.id, events));
+    }
+
+    // Access it to set the last accessed time. End access it to set not new.
+    access();
+    endAccess();
+  }
+
+  @Override
+  public void toData(DataOutput out) throws IOException {
+    // Write the StandardSession state
+    DataSerializer.writeString(this.id, out);
+    out.writeLong(this.creationTime);
+    out.writeLong(this.lastAccessedTime);
+    out.writeLong(this.thisAccessedTime);
+    out.writeInt(this.maxInactiveInterval);
+    out.writeBoolean(this.isNew);
+    out.writeBoolean(this.isValid);
+    DataSerializer.writeObject(getSerializedAttributes(), out);
+    DataSerializer.writeByteArray(getSerializedPrincipal(), out);
+
+    // Write the DeltaSession state
+    out.writeBoolean(this.enableGatewayDeltaReplication);
+    DataSerializer.writeString(this.sessionRegionName, out);
+
+    DataSerializer.writeString(this.contextName, out);
+  }
+
+  @Override
+  public void fromData(DataInput in) throws IOException, ClassNotFoundException {
+    // Read the StandardSession state
+    this.id = DataSerializer.readString(in);
+    this.creationTime = in.readLong();
+    this.lastAccessedTime = in.readLong();
+    this.thisAccessedTime = in.readLong();
+    this.maxInactiveInterval = in.readInt();
+    this.isNew = in.readBoolean();
+    this.isValid = in.readBoolean();
+    this.attributes = DataSerializer.readObject(in);
+    this.serializedPrincipal = DataSerializer.readByteArray(in);
+
+    // Read the DeltaSession state
+    this.enableGatewayDeltaReplication = in.readBoolean();
+    this.sessionRegionName = DataSerializer.readString(in);
+
+    // This allows for backwards compatibility with 2.1 clients
+    if (((InputStream) in).available() > 0) {
+      this.contextName = DataSerializer.readString(in);
+    }
+
+    // Initialize the transients if necessary
+    if (this.listeners == null) {
+      this.listeners = new ArrayList();
+    }
+
+    if (this.notes == null) {
+      this.notes = new Hashtable();
+    }
+  }
+
+  @Override
+  public int getSizeInBytes() {
+    int size = 0;
+    for (Enumeration<String> e = getAttributeNames(); e.hasMoreElements(); ) {
+      // Don't use this.getAttribute() because we don't want to deserialize
+      // the value.
+      Object value = super.getAttribute(e.nextElement());
+      if (value instanceof byte[]) {
+        size += ((byte[]) value).length;
+      }
+    }
+
+    return size;
+  }
+
+  @SuppressWarnings({"unchecked", "rawtypes"})
+  private Map<String, byte[]> getSerializedAttributes() {
+    // Iterate the values and serialize them if necessary before sending them to the server. This makes the application classes unnecessary on the server.
+    Map<String, byte[]> serializedAttributes = new ConcurrentHashMap<String, byte[]>();
+    for (Iterator i = this.attributes.entrySet().iterator(); i.hasNext(); ) {
+      Map.Entry<String, Object> entry = (Map.Entry<String, Object>) i.next();
+      Object value = entry.getValue();
+      byte[] serializedValue = value instanceof byte[] ? (byte[]) value : serialize(value);
+      serializedAttributes.put(entry.getKey(), serializedValue);
+    }
+    return serializedAttributes;
+  }
+
+  private byte[] serialize(Object obj) {
+    byte[] serializedValue = null;
+    try {
+      serializedValue = BlobHelper.serializeToBlob(obj);
+    } catch (IOException e) {
+      StringBuilder builder = new StringBuilder();
+      builder.append(this)
+          .append(": Object ")
+          .append(obj)
+          .append(" cannot be serialized due to the following exception");
+      ((DeltaSessionManager) getManager()).getLogger().warn(builder.toString(), e);
+    }
+    return serializedValue;
+  }
+
+  @Override
+  public String toString() {
+    return new StringBuilder().append("DeltaSession[")
+        .append("id=")
+        .append(getId())
+        .append("; context=")
+        .append(this.contextName)
+        .append("; sessionRegionName=")
+        .append(this.sessionRegionName)
+        .append("; operatingRegionName=")
+        .append(getOperatingRegion() == null ? "unset" : getOperatingRegion().getFullPath())
+        .append("]")
+        .toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/48552465/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/DeltaSessionFacade.java
----------------------------------------------------------------------
diff --git a/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/DeltaSessionFacade.java b/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/DeltaSessionFacade.java
new file mode 100644
index 0000000..1ac4da2
--- /dev/null
+++ b/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/DeltaSessionFacade.java
@@ -0,0 +1,49 @@
+/*
+* 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.modules.session.catalina;
+
+import org.apache.catalina.session.StandardSessionFacade;
+
+import javax.servlet.http.HttpSession;
+
+public class DeltaSessionFacade extends StandardSessionFacade {
+
+  private DeltaSession session;
+
+  /**
+   * Construct a new session facade.
+   */
+  public DeltaSessionFacade(DeltaSession session) {
+    super((HttpSession) session);
+    // Store session locally since the super session is private and provides no accessor.
+    this.session = session;
+  }
+
+  // ----------- DeltaSession Methods
+
+  public void commit() {
+    this.session.commit();
+  }
+
+  public void abort() {
+    this.session.abort();
+  }
+
+  boolean isValid() {
+    return this.session.isValid();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/48552465/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/DeltaSessionManager.java
----------------------------------------------------------------------
diff --git a/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/DeltaSessionManager.java b/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/DeltaSessionManager.java
new file mode 100644
index 0000000..92d9ef6
--- /dev/null
+++ b/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/DeltaSessionManager.java
@@ -0,0 +1,992 @@
+/*
+* 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.modules.session.catalina;
+
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.query.Query;
+import com.gemstone.gemfire.cache.query.QueryService;
+import com.gemstone.gemfire.cache.query.SelectResults;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.modules.session.catalina.internal.DeltaSessionStatistics;
+import com.gemstone.gemfire.modules.util.ContextMapper;
+import com.gemstone.gemfire.modules.util.RegionConfiguration;
+import com.gemstone.gemfire.modules.util.RegionHelper;
+import org.apache.catalina.Container;
+import org.apache.catalina.Context;
+import org.apache.catalina.Lifecycle;
+import org.apache.catalina.LifecycleListener;
+import org.apache.catalina.Loader;
+import org.apache.catalina.Session;
+import org.apache.catalina.Valve;
+import org.apache.catalina.session.ManagerBase;
+import org.apache.catalina.session.StandardSession;
+import org.apache.catalina.util.CustomObjectInputStream;
+import org.apache.catalina.util.LifecycleSupport;
+import org.apache.juli.logging.Log;
+import org.apache.juli.logging.LogFactory;
+
+import java.beans.PropertyChangeEvent;
+import java.beans.PropertyChangeListener;
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+abstract public class DeltaSessionManager extends ManagerBase implements Lifecycle, PropertyChangeListener, SessionManager {
+
+  /**
+   * The <code>LifecycleSupport</code> for this component.
+   */
+  protected LifecycleSupport lifecycle = new LifecycleSupport(this);
+
+  /**
+   * The number of rejected sessions.
+   */
+  private AtomicInteger rejectedSessions;
+
+  /**
+   * The maximum number of active Sessions allowed, or -1 for no limit.
+   */
+  protected int maxActiveSessions = -1;
+
+  /**
+   * Has this <code>Manager</code> been started?
+   */
+  protected AtomicBoolean started = new AtomicBoolean(false);
+
+  /**
+   * The name of this <code>Manager</code>
+   */
+  protected String name;
+
+  protected Valve jvmRouteBinderValve;
+
+  protected Valve commitSessionValve;
+
+  protected SessionCache sessionCache;
+
+  protected static final String DEFAULT_REGION_NAME = RegionHelper.NAME + "_sessions";
+
+  protected static final boolean DEFAULT_ENABLE_GATEWAY_DELTA_REPLICATION = false;
+
+  protected static final boolean DEFAULT_ENABLE_GATEWAY_REPLICATION = false;
+
+  protected static final boolean DEFAULT_ENABLE_DEBUG_LISTENER = false;
+
+  protected static final boolean DEFAULT_ENABLE_COMMIT_VALVE = true;
+
+  protected static final boolean DEFAULT_ENABLE_COMMIT_VALVE_FAILFAST = false;
+
+  protected static final boolean DEFAULT_PREFER_DESERIALIZED_FORM = true;
+
+  /*
+   * This *MUST* only be assigned during start/startInternal otherwise it will be associated
+   * with the incorrect context class loader.
+   */
+  protected Log LOGGER;
+
+  protected String regionName = DEFAULT_REGION_NAME;
+
+  protected String regionAttributesId; // the default is different for client-server and peer-to-peer
+
+  protected Boolean enableLocalCache; // the default is different for client-server and peer-to-peer
+
+  protected boolean enableCommitValve = DEFAULT_ENABLE_COMMIT_VALVE;
+
+  protected boolean enableCommitValveFailfast = DEFAULT_ENABLE_COMMIT_VALVE_FAILFAST;
+
+  protected boolean enableGatewayDeltaReplication = DEFAULT_ENABLE_GATEWAY_DELTA_REPLICATION;
+
+  protected boolean enableGatewayReplication = DEFAULT_ENABLE_GATEWAY_REPLICATION;
+
+  protected boolean enableDebugListener = DEFAULT_ENABLE_DEBUG_LISTENER;
+
+  protected boolean preferDeserializedForm = DEFAULT_PREFER_DESERIALIZED_FORM;
+
+  private Timer timer;
+
+  private final Set<String> sessionsToTouch;
+
+  private static final long TIMER_TASK_PERIOD = Long.getLong("gemfiremodules.sessionTimerTaskPeriod", 10000);
+
+  private static final long TIMER_TASK_DELAY = Long.getLong("gemfiremodules.sessionTimerTaskDelay", 10000);
+
+  public DeltaSessionManager() {
+    // Create the set to store sessions to be touched after get attribute requests
+    this.sessionsToTouch = Collections.newSetFromMap(new ConcurrentHashMap<String, Boolean>());
+  }
+
+  @Override
+  public String getRegionName() {
+    return this.regionName;
+  }
+
+  public void setRegionName(String regionName) {
+    this.regionName = regionName;
+  }
+
+  @Override
+  public String getRegionAttributesId() {
+    // This property will be null if it hasn't been set in the context.xml file.
+    // Since its default is dependent on the session cache, get the default from
+    // the session cache.
+    if (this.regionAttributesId == null) {
+      this.regionAttributesId = getSessionCache().getDefaultRegionAttributesId();
+    }
+    return this.regionAttributesId;
+  }
+
+  public void setRegionAttributesId(String regionType) {
+    this.regionAttributesId = regionType;
+  }
+
+  @Override
+  public boolean getEnableLocalCache() {
+    // This property will be null if it hasn't been set in the context.xml file.
+    // Since its default is dependent on the session cache, get the default from
+    // the session cache.
+    if (this.enableLocalCache == null) {
+      this.enableLocalCache = getSessionCache().getDefaultEnableLocalCache();
+    }
+    return this.enableLocalCache;
+  }
+
+  public void setEnableLocalCache(boolean enableLocalCache) {
+    this.enableLocalCache = enableLocalCache;
+  }
+
+  public int getMaxActiveSessions() {
+    return this.maxActiveSessions;
+  }
+
+  public void setMaxActiveSessions(int maxActiveSessions) {
+    int oldMaxActiveSessions = this.maxActiveSessions;
+    this.maxActiveSessions = maxActiveSessions;
+    support.firePropertyChange("maxActiveSessions", new Integer(oldMaxActiveSessions),
+        new Integer(this.maxActiveSessions));
+  }
+
+  @Override
+  public boolean getEnableGatewayDeltaReplication() {
+    //return this.enableGatewayDeltaReplication;
+    return false; // disabled
+  }
+
+  public void setEnableGatewayDeltaReplication(boolean enableGatewayDeltaReplication) {
+    this.enableGatewayDeltaReplication = enableGatewayDeltaReplication;
+  }
+
+  @Override
+  public boolean getEnableGatewayReplication() {
+    return this.enableGatewayReplication;
+  }
+
+  public void setEnableGatewayReplication(boolean enableGatewayReplication) {
+    this.enableGatewayReplication = enableGatewayReplication;
+  }
+
+  @Override
+  public boolean getEnableDebugListener() {
+    return this.enableDebugListener;
+  }
+
+  public void setEnableDebugListener(boolean enableDebugListener) {
+    this.enableDebugListener = enableDebugListener;
+  }
+
+  @Override
+  public boolean isCommitValveEnabled() {
+    return this.enableCommitValve;
+  }
+
+  public void setEnableCommitValve(boolean enable) {
+    this.enableCommitValve = enable;
+  }
+
+  @Override
+  public boolean isCommitValveFailfastEnabled() {
+    return this.enableCommitValveFailfast;
+  }
+
+  public void setEnableCommitValveFailfast(boolean enable) {
+    this.enableCommitValveFailfast = enable;
+  }
+
+  @Override
+  public boolean isBackingCacheAvailable() {
+    return sessionCache.isBackingCacheAvailable();
+  }
+
+  public void setPreferDeserializedForm(boolean enable) {
+    this.preferDeserializedForm = enable;
+  }
+
+  @Override
+  public boolean getPreferDeserializedForm() {
+    return this.preferDeserializedForm;
+  }
+
+  @Override
+  public String getStatisticsName() {
+    return getContainer().getName().replace("/", "");
+  }
+
+  @Override
+  public Log getLogger() {
+    if (LOGGER == null) {
+      LOGGER = LogFactory.getLog(DeltaSessionManager.class);
+    }
+    return LOGGER;
+  }
+
+  public SessionCache getSessionCache() {
+    return this.sessionCache;
+  }
+
+  public DeltaSessionStatistics getStatistics() {
+    return getSessionCache().getStatistics();
+  }
+
+  public boolean isPeerToPeer() {
+    return getSessionCache().isPeerToPeer();
+  }
+
+  public boolean isClientServer() {
+    return getSessionCache().isClientServer();
+  }
+
+  /**
+   * This method was taken from StandardManager to set the default maxInactiveInterval based on the container (to 30
+   * minutes).
+   * <p>
+   * Set the Container with which this Manager has been associated. If it is a Context (the usual case), listen for
+   * changes to the session timeout property.
+   *
+   * @param container The associated Container
+   */
+  @Override
+  public void setContainer(Container container) {
+    // De-register from the old Container (if any)
+    if ((this.container != null) && (this.container instanceof Context)) {
+      ((Context) this.container).removePropertyChangeListener(this);
+    }
+
+    // Default processing provided by our superclass
+    super.setContainer(container);
+
+    // Register with the new Container (if any)
+    if ((this.container != null) && (this.container instanceof Context)) {
+      // Overwrite the max inactive interval with the context's session timeout.
+      setMaxInactiveInterval(((Context) this.container).getSessionTimeout() * 60);
+      ((Context) this.container).addPropertyChangeListener(this);
+    }
+  }
+
+  @Override
+  public Session findSession(String id) throws IOException {
+    if (id == null) {
+      return null;
+    }
+    if (getLogger().isDebugEnabled()) {
+      getLogger().debug(this + ": Finding session " + id + " in " + getSessionCache().getOperatingRegionName());
+    }
+    DeltaSession session = (DeltaSession) getSessionCache().getSession(id);
+    /*
+     * Check that the context name for this session is the same as this manager's.
+     * This comes into play when multiple versions of a webapp are deployed and
+     * active at the same time; the context name will contain an embedded
+     * version number; something like /test###2.
+     */
+    if (session != null &&
+        !session.getContextName().isEmpty() &&
+        !getContainer().getName().equals(session.getContextName())) {
+      getLogger().info(this + ": Session " + id +
+          " rejected as container name and context do not match: " +
+          getContainer().getName() + " != " + session.getContextName());
+      session = null;
+    }
+
+    if (session == null) {
+      if (getLogger().isDebugEnabled()) {
+        getLogger().debug(this + ": Did not find session " + id + " in " + getSessionCache().getOperatingRegionName());
+      }
+    } else {
+      if (getLogger().isDebugEnabled()) {
+        getLogger().debug(
+            this + ": Found session " + id + " in " + getSessionCache().getOperatingRegionName() + ": " + session);
+      }
+      // The session was previously stored. Set new to false.
+      session.setNew(false);
+
+      // Check the manager.
+      // If the manager is null, the session was replicated and this is a
+      // failover situation. Reset the manager and activate the session.
+      if (session.getManager() == null) {
+        DeltaSession ds = (DeltaSession) session;
+        ds.setOwner(this);
+        ds.activate();
+      }
+    }
+    return session;
+  }
+
+  protected void initializeSessionCache() {
+    // Retrieve the cache
+    GemFireCacheImpl cache = (GemFireCacheImpl) CacheFactory.getAnyInstance();
+    if (cache == null) {
+      throw new IllegalStateException(
+          "No cache exists. Please configure either a PeerToPeerCacheLifecycleListener or ClientServerCacheLifecycleListener in the server.xml file.");
+    }
+
+    // Create the appropriate session cache
+    this.sessionCache = cache.isClient() ? new ClientServerSessionCache(this, cache) : new PeerToPeerSessionCache(this,
+        cache);
+
+    // Initialize the session cache
+    this.sessionCache.initialize();
+  }
+
+  @Override
+  protected StandardSession getNewSession() {
+    return new DeltaSession(this);
+  }
+
+  @Override
+  public void remove(Session session) {
+    remove(session, false);
+  }
+
+  public void remove(Session session, boolean update) {
+    //super.remove(session);
+    // Remove the session from the region if necessary.
+    // It will have already been removed if it expired implicitly.
+    DeltaSession ds = (DeltaSession) session;
+    if (ds.getExpired()) {
+      if (getLogger().isDebugEnabled()) {
+        getLogger().debug(
+            this + ": Expired session " + session.getId() + " from " + getSessionCache().getOperatingRegionName());
+      }
+    } else {
+      if (getLogger().isDebugEnabled()) {
+        getLogger().debug(
+            this + ": Destroying session " + session.getId() + " from " + getSessionCache().getOperatingRegionName());
+      }
+      getSessionCache().destroySession(session.getId());
+      if (getLogger().isDebugEnabled()) {
+        getLogger().debug(
+            this + ": Destroyed session " + session.getId() + " from " + getSessionCache().getOperatingRegionName());
+      }
+    }
+  }
+
+  @Override
+  public void add(Session session) {
+    //super.add(session);
+    if (getLogger().isDebugEnabled()) {
+      getLogger().debug(
+          this + ": Storing session " + session.getId() + " into " + getSessionCache().getOperatingRegionName());
+    }
+    getSessionCache().putSession(session);
+    if (getLogger().isDebugEnabled()) {
+      getLogger().debug(
+          this + ": Stored session " + session.getId() + " into " + getSessionCache().getOperatingRegionName());
+    }
+    getSessionCache().getStatistics().incSessionsCreated();
+  }
+
+  @Override
+  public int getRejectedSessions() {
+    return this.rejectedSessions.get();
+  }
+
+  public void setRejectedSessions(int rejectedSessions) {
+    this.rejectedSessions.set(rejectedSessions);
+  }
+
+  private void incrementRejectedSessions() {
+    this.rejectedSessions.incrementAndGet();
+  }
+
+  /**
+   * Returns the number of active sessions
+   *
+   * @return number of sessions active
+   */
+  @Override
+  public int getActiveSessions() {
+    return getSessionCache().size();
+  }
+
+  /**
+   * For debugging: return a list of all session ids currently active
+   */
+  @Override
+  public String listSessionIds() {
+    StringBuilder builder = new StringBuilder();
+    Iterator<String> sessionIds = getSessionCache().keySet().iterator();
+    while (sessionIds.hasNext()) {
+      builder.append(sessionIds.next());
+      if (sessionIds.hasNext()) {
+        builder.append(" ");
+      }
+    }
+    return builder.toString();
+  }
+
+  /*
+   * If local caching is enabled, add the session to the set of sessions to be
+   * touched. A timer task will be periodically invoked to get the session in
+   * the session region to update its last accessed time. This prevents the
+   * session from expiring in the case where the application is only getting
+   * attributes from the session and never putting attributes into the
+   * session. If local caching is disabled. the session's last accessed time
+   * would already have been updated properly in the sessions region.
+   *
+   * Note: Due to issues in GemFire expiry, sessions are always asynchronously
+   * touched using a function regardless whether or not local caching is
+   * enabled. This prevents premature expiration.
+   */
+  protected void addSessionToTouch(String sessionId) {
+    this.sessionsToTouch.add(sessionId);
+  }
+
+  protected Set<String> getSessionsToTouch() {
+    return this.sessionsToTouch;
+  }
+
+  protected boolean removeTouchedSession(String sessionId) {
+    return this.sessionsToTouch.remove(sessionId);
+  }
+
+  protected void scheduleTimerTasks() {
+    // Create the timer
+    this.timer = new Timer("Timer for " + toString(), true);
+
+    // Schedule the task to handle sessions to be touched
+    scheduleTouchSessionsTask();
+
+    // Schedule the task to maintain the maxActive sessions
+    scheduleDetermineMaxActiveSessionsTask();
+  }
+
+  private void scheduleTouchSessionsTask() {
+    TimerTask task = new TimerTask() {
+      @Override
+      public void run() {
+        // Get the sessionIds to touch and clear the set inside synchronization
+        Set<String> sessionIds = null;
+        sessionIds = new HashSet<String>(getSessionsToTouch());
+        getSessionsToTouch().clear();
+
+        // Touch the sessions we currently have
+        if (sessionIds != null && (!sessionIds.isEmpty())) {
+          getSessionCache().touchSessions(sessionIds);
+          if (getLogger().isDebugEnabled()) {
+            getLogger().debug(DeltaSessionManager.this + ": Touched sessions: " + sessionIds);
+          }
+        }
+      }
+    };
+    this.timer.schedule(task, TIMER_TASK_DELAY, TIMER_TASK_PERIOD);
+  }
+
+  protected void cancelTimer() {
+    if (timer != null) {
+      this.timer.cancel();
+    }
+  }
+
+  private void scheduleDetermineMaxActiveSessionsTask() {
+    TimerTask task = new TimerTask() {
+      @Override
+      public void run() {
+        int currentActiveSessions = getSessionCache().size();
+        if (currentActiveSessions > getMaxActive()) {
+          setMaxActive(currentActiveSessions);
+          if (getLogger().isDebugEnabled()) {
+            getLogger().debug(DeltaSessionManager.this + ": Set max active sessions: " + currentActiveSessions);
+          }
+        }
+      }
+    };
+    this.timer.schedule(task, TIMER_TASK_DELAY, TIMER_TASK_PERIOD);
+  }
+
+  @Override
+  public void load() throws ClassNotFoundException, IOException {
+    doLoad();
+    ContextMapper.addContext(getContainer().getName(), this);
+  }
+
+  @Override
+  public void unload() throws IOException {
+    doUnload();
+    ContextMapper.removeContext(getContainer().getName());
+  }
+
+  protected void registerJvmRouteBinderValve() {
+    if (getLogger().isDebugEnabled()) {
+      getLogger().debug(this + ": Registering JVM route binder valve");
+    }
+    jvmRouteBinderValve = new JvmRouteBinderValve();
+    getContainer().getPipeline().addValve(jvmRouteBinderValve);
+  }
+
+  protected void unregisterJvmRouteBinderValve() {
+    if (getLogger().isDebugEnabled()) {
+      getLogger().debug(this + ": Unregistering JVM route binder valve");
+    }
+    if (jvmRouteBinderValve != null) {
+      getContainer().getPipeline().removeValve(jvmRouteBinderValve);
+    }
+  }
+
+  protected void registerCommitSessionValve() {
+    if (getLogger().isDebugEnabled()) {
+      getLogger().debug(this + ": Registering CommitSessionValve");
+    }
+    commitSessionValve = new CommitSessionValve();
+    getContainer().getPipeline().addValve(commitSessionValve);
+  }
+
+  protected void unregisterCommitSessionValve() {
+    if (getLogger().isDebugEnabled()) {
+      getLogger().debug(this + ": Unregistering CommitSessionValve");
+    }
+    if (commitSessionValve != null) {
+      getContainer().getPipeline().removeValve(commitSessionValve);
+    }
+  }
+
+  // ------------------------------ Lifecycle Methods
+
+  /**
+   * Add a lifecycle event listener to this component.
+   *
+   * @param listener The listener to add
+   */
+  @Override
+  public void addLifecycleListener(LifecycleListener listener) {
+    this.lifecycle.addLifecycleListener(listener);
+  }
+
+  /**
+   * Get the lifecycle listeners associated with this lifecycle. If this Lifecycle has no listeners registered, a
+   * zero-length array is returned.
+   */
+  @Override
+  public LifecycleListener[] findLifecycleListeners() {
+    return this.lifecycle.findLifecycleListeners();
+  }
+
+  /**
+   * Remove a lifecycle event listener from this component.
+   *
+   * @param listener The listener to remove
+   */
+  @Override
+  public void removeLifecycleListener(LifecycleListener listener) {
+    this.lifecycle.removeLifecycleListener(listener);
+  }
+
+  /**
+   * Process property change events from our associated Context.
+   * <p>
+   * Part of this method implementation was taken from StandardManager. The sessionTimeout can be changed in the web.xml
+   * which is processed after the context.xml. The context (and the default session timeout) would already have been set
+   * in this Manager. This is the way to get the new session timeout value specified in the web.xml.
+   * <p>
+   * The precedence order for setting the session timeout value is:
+   * <p>
+   * <ol> <li>the max inactive interval is set based on the Manager defined in the context.xml <li>the max inactive
+   * interval is then overwritten by the value of the Context's session timeout when setContainer is called <li>the max
+   * inactive interval is then overwritten by the value of the session-timeout specified in the web.xml (if any) </ol>
+   *
+   * @param event The property change event that has occurred
+   */
+  @Override
+  public void propertyChange(PropertyChangeEvent event) {
+
+    // Validate the source of this event
+    if (!(event.getSource() instanceof Context)) {
+      return;
+    }
+    Context context = (Context) event.getSource();
+
+    // Process a relevant property change
+    if (event.getPropertyName().equals("sessionTimeout")) {
+      try {
+        int interval = ((Integer) event.getNewValue()).intValue();
+        if (interval < RegionConfiguration.DEFAULT_MAX_INACTIVE_INTERVAL) {
+          getLogger().warn(
+              "The configured session timeout of " + interval + " minutes is invalid. Using the original value of " + event
+                  .getOldValue() + " minutes.");
+          interval = ((Integer) event.getOldValue()).intValue();
+          ;
+        }
+        // StandardContext.setSessionTimeout passes -1 if the configured timeout
+        // is 0; otherwise it passes the value set in web.xml. If the interval
+        // parameter equals the default, set the max inactive interval to the
+        // default (no expiration); otherwise set it in seconds.
+        setMaxInactiveInterval(
+            interval == RegionConfiguration.DEFAULT_MAX_INACTIVE_INTERVAL ? RegionConfiguration.DEFAULT_MAX_INACTIVE_INTERVAL : interval * 60);
+      } catch (NumberFormatException e) {
+        getLogger().error(sm.getString("standardManager.sessionTimeout", event.getNewValue().toString()));
+      }
+    }
+  }
+
+  /**
+   * Save any currently active sessions in the appropriate persistence mechanism, if any.  If persistence is not
+   * supported, this method returns without doing anything.
+   *
+   * @throws IOException if an input/output error occurs
+   */
+  protected void doUnload() throws IOException {
+    QueryService querySvc = sessionCache.getCache().getQueryService();
+    Context context;
+    if (getContainer() instanceof Context) {
+      context = (Context) getContainer();
+    } else {
+      getLogger().error("Unable to unload sessions - container is of type " +
+          getContainer().getClass().getName() + " instead of StandardContext");
+      return;
+    }
+    String regionName;
+    if (getRegionName().startsWith("/")) {
+      regionName = getRegionName();
+    } else {
+      regionName = "/" + getRegionName();
+    }
+    Query query = querySvc.newQuery("select s.id from " + regionName +
+        " as s where s.contextName = '" + context.getPath() + "'");
+    getLogger().debug("Query: " + query.getQueryString());
+
+    SelectResults results;
+    try {
+      results = (SelectResults) query.execute();
+    } catch (Exception ex) {
+      getLogger().error("Unable to perform query during doUnload", ex);
+      return;
+    }
+
+    if (results.isEmpty()) {
+      getLogger().debug("No sessions to unload for context " + context.getPath());
+      return; // nothing to do
+    }
+
+    // Open an output stream to the specified pathname, if any
+    File store = sessionStore(context.getPath());
+    if (store == null) {
+      return;
+    }
+    if (getLogger().isDebugEnabled()) {
+      getLogger().debug("Unloading sessions to " + store.getAbsolutePath());
+    }
+    FileOutputStream fos = null;
+    BufferedOutputStream bos = null;
+    ObjectOutputStream oos = null;
+    boolean error = false;
+    try {
+      fos = new FileOutputStream(store.getAbsolutePath());
+      bos = new BufferedOutputStream(fos);
+      oos = new ObjectOutputStream(bos);
+    } catch (IOException e) {
+      error = true;
+      getLogger().error("Exception unloading sessions", e);
+      throw e;
+    } finally {
+      if (error) {
+        if (oos != null) {
+          try {
+            oos.close();
+          } catch (IOException ioe) {
+            // Ignore
+          }
+        }
+        if (bos != null) {
+          try {
+            bos.close();
+          } catch (IOException ioe) {
+            // Ignore
+          }
+        }
+        if (fos != null) {
+          try {
+            fos.close();
+          } catch (IOException ioe) {
+            // Ignore
+          }
+        }
+      }
+    }
+
+    ArrayList<StandardSession> list = new ArrayList<StandardSession>();
+    Iterator<String> elements = results.iterator();
+    while (elements.hasNext()) {
+      String id = elements.next();
+      DeltaSession session = (DeltaSession) findSession(id);
+      if (session != null) {
+        list.add(session);
+      }
+    }
+
+    // Write the number of active sessions, followed by the details
+    if (getLogger().isDebugEnabled()) getLogger().debug("Unloading " + list.size() + " sessions");
+    try {
+      oos.writeObject(new Integer(list.size()));
+      for (StandardSession session : list) {
+        session.passivate();
+        session.writeObjectData(oos);
+      }
+    } catch (IOException e) {
+      getLogger().error("Exception unloading sessions", e);
+      try {
+        oos.close();
+      } catch (IOException f) {
+        // Ignore
+      }
+      throw e;
+    }
+
+    // Flush and close the output stream
+    try {
+      oos.flush();
+    } finally {
+      try {
+        oos.close();
+      } catch (IOException f) {
+        // Ignore
+      }
+    }
+
+    // Locally destroy the sessions we just wrote
+    if (getSessionCache().isClientServer()) {
+      for (StandardSession session : list) {
+        if (getLogger().isDebugEnabled()) {
+          getLogger().debug("Locally destroying session " + session.getId());
+        }
+        getSessionCache().getOperatingRegion().localDestroy(session.getId());
+      }
+    }
+
+//    // Expire all the sessions we just wrote
+//    if (getLogger().isDebugEnabled()) {
+//      getLogger().debug("Expiring " + list.size() + " persisted sessions");
+//    }
+//    Iterator<StandardSession> expires = list.iterator();
+//    while (expires.hasNext()) {
+//      StandardSession session = expires.next();
+//      try {
+//        session.expire(false);
+//      } catch (Throwable t) {
+////        ExceptionUtils.handleThrowable(t);
+//      } finally {
+//        session.recycle();
+//      }
+//    }
+
+    if (getLogger().isDebugEnabled()) {
+      getLogger().debug("Unloading complete");
+    }
+  }
+
+  /**
+   * Load any currently active sessions that were previously unloaded to the appropriate persistence mechanism, if any.
+   * If persistence is not supported, this method returns without doing anything.
+   *
+   * @throws ClassNotFoundException if a serialized class cannot be found during the reload
+   * @throws IOException            if an input/output error occurs
+   */
+  protected void doLoad() throws ClassNotFoundException, IOException {
+    Context context;
+    if (getContainer() instanceof Context) {
+      context = (Context) getContainer();
+    } else {
+      getLogger().error("Unable to unload sessions - container is of type " +
+          getContainer().getClass().getName() + " instead of StandardContext");
+      return;
+    }
+
+    // Open an input stream to the specified pathname, if any
+    File store = sessionStore(context.getPath());
+    if (store == null) {
+      getLogger().debug("No session store file found");
+      return;
+    }
+    if (getLogger().isDebugEnabled()) {
+      getLogger().debug("Loading sessions from " + store.getAbsolutePath());
+    }
+    FileInputStream fis = null;
+    BufferedInputStream bis = null;
+    ObjectInputStream ois = null;
+    Loader loader = null;
+    ClassLoader classLoader = null;
+    try {
+      fis = new FileInputStream(store.getAbsolutePath());
+      bis = new BufferedInputStream(fis);
+      if (container != null) {
+        loader = container.getLoader();
+      }
+      if (loader != null) {
+        classLoader = loader.getClassLoader();
+      }
+      if (classLoader != null) {
+        if (getLogger().isDebugEnabled()) {
+          getLogger().debug("Creating custom object input stream for class loader");
+        }
+        ois = new CustomObjectInputStream(bis, classLoader);
+      } else {
+        if (getLogger().isDebugEnabled()) {
+          getLogger().debug("Creating standard object input stream");
+        }
+        ois = new ObjectInputStream(bis);
+      }
+    } catch (FileNotFoundException e) {
+      if (getLogger().isDebugEnabled()) {
+        getLogger().debug("No persisted data file found");
+      }
+      return;
+    } catch (IOException e) {
+      getLogger().error("Exception loading sessions", e);
+      if (fis != null) {
+        try {
+          fis.close();
+        } catch (IOException f) {
+          // Ignore
+        }
+      }
+      if (bis != null) {
+        try {
+          bis.close();
+        } catch (IOException f) {
+          // Ignore
+        }
+      }
+      throw e;
+    }
+
+    // Load the previously unloaded active sessions
+    try {
+      Integer count = (Integer) ois.readObject();
+      int n = count.intValue();
+      if (getLogger().isDebugEnabled()) {
+        getLogger().debug("Loading " + n + " persisted sessions");
+      }
+      for (int i = 0; i < n; i++) {
+        StandardSession session = getNewSession();
+        session.readObjectData(ois);
+        session.setManager(this);
+
+        Region region = getSessionCache().getOperatingRegion();
+        DeltaSession existingSession = (DeltaSession) region.get(session.getId());
+        // Check whether the existing session is newer
+        if (existingSession != null && existingSession.getLastAccessedTime() > session.getLastAccessedTime()) {
+          if (getLogger().isDebugEnabled()) {
+            getLogger().debug("Loaded session " + session.getId() + " is older than cached copy");
+          }
+          continue;
+        }
+
+        // Check whether the new session has already expired
+        if (!session.isValid()) {
+          if (getLogger().isDebugEnabled()) {
+            getLogger().debug("Loaded session " + session.getId() + " is invalid");
+          }
+          continue;
+        }
+
+        getLogger().debug("Loading session " + session.getId());
+        session.activate();
+        add(session);
+      }
+    } catch (ClassNotFoundException e) {
+      getLogger().error(e);
+      try {
+        ois.close();
+      } catch (IOException f) {
+        // Ignore
+      }
+      throw e;
+    } catch (IOException e) {
+      getLogger().error(e);
+      try {
+        ois.close();
+      } catch (IOException f) {
+        // Ignore
+      }
+      throw e;
+    } finally {
+      // Close the input stream
+      try {
+        ois.close();
+      } catch (IOException f) {
+        // ignored
+      }
+
+      // Delete the persistent storage file
+      if (store.exists()) {
+        store.delete();
+      }
+    }
+  }
+
+  /**
+   * Return a File object representing the pathname to our persistence file, if any.
+   */
+  private File sessionStore(String ctxPath) {
+    String storeDir = System.getProperty("catalina.base");
+    if (storeDir == null || storeDir.isEmpty()) {
+      storeDir = System.getProperty("java.io.tmpdir");
+    } else {
+      storeDir += System.getProperty("file.separator") + "temp";
+    }
+    File file = new File(storeDir, ctxPath.replaceAll("/", "_") + ".sessions.ser");
+
+    return (file);
+  }
+
+  @Override
+  public String toString() {
+    return new StringBuilder().append(getClass().getSimpleName())
+        .append("[")
+        .append("container=")
+        .append(getContainer())
+        .append("; regionName=")
+        .append(this.regionName)
+        .append("; regionAttributesId=")
+        .append(this.regionAttributesId)
+        .append("]")
+        .toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/48552465/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/JvmRouteBinderValve.java
----------------------------------------------------------------------
diff --git a/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/JvmRouteBinderValve.java b/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/JvmRouteBinderValve.java
new file mode 100644
index 0000000..98481c7
--- /dev/null
+++ b/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/JvmRouteBinderValve.java
@@ -0,0 +1,108 @@
+/*
+* 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.modules.session.catalina;
+
+import org.apache.catalina.Manager;
+import org.apache.catalina.Session;
+import org.apache.catalina.connector.Request;
+import org.apache.catalina.connector.Response;
+import org.apache.catalina.valves.ValveBase;
+
+import javax.servlet.ServletException;
+import java.io.IOException;
+
+public class JvmRouteBinderValve extends ValveBase {
+
+  protected static final String info = "com.gemstone.gemfire.modules.session.JvmRouteBinderValve/1.0";
+
+  @Override
+  public void invoke(Request request, Response response) throws IOException, ServletException {
+
+    // Get the Manager
+    Manager manager = request.getContext().getManager();
+
+    // If it is an AbstractManager, handle possible failover
+    if (manager instanceof DeltaSessionManager) {
+      DeltaSessionManager absMgr = (DeltaSessionManager) manager;
+      String localJvmRoute = absMgr.getJvmRoute();
+      if (localJvmRoute != null) {
+        handlePossibleFailover(request, absMgr, localJvmRoute);
+      }
+    }
+
+    // Invoke the next Valve
+    getNext().invoke(request, response);
+  }
+
+  private void handlePossibleFailover(Request request, DeltaSessionManager manager, String localJvmRoute) {
+    String sessionId = request.getRequestedSessionId();
+    if (sessionId != null) {
+      // Get request JVM route
+      String requestJvmRoute = null;
+      int index = sessionId.indexOf(".");
+      if (index > 0) {
+        requestJvmRoute = sessionId.substring(index + 1, sessionId.length());
+      }
+
+      // If the requested JVM route doesn't equal the session's JVM route, handle failover
+      if (requestJvmRoute != null && !requestJvmRoute.equals(localJvmRoute)) {
+        if (manager.getLogger().isDebugEnabled()) {
+          StringBuilder builder = new StringBuilder();
+          builder.append(this)
+              .append(": Handling failover of session ")
+              .append(sessionId)
+              .append(" from ")
+              .append(requestJvmRoute)
+              .append(" to ")
+              .append(localJvmRoute);
+          manager.getLogger().debug(builder.toString());
+        }
+        // Get the original session
+        Session session = null;
+        try {
+          session = manager.findSession(sessionId);
+        } catch (IOException e) {
+          StringBuilder builder = new StringBuilder();
+          builder.append(this)
+              .append(": Caught exception attempting to find session ")
+              .append(sessionId)
+              .append(" in ")
+              .append(manager);
+          manager.getLogger().warn(builder.toString(), e);
+        }
+
+        if (session == null) {
+          StringBuilder builder = new StringBuilder();
+          builder.append(this)
+              .append(": Did not find session ")
+              .append(sessionId)
+              .append(" to failover in ")
+              .append(manager);
+          manager.getLogger().warn(builder.toString());
+        } else {
+          // Change its session id. This removes the previous session and creates the new one.
+          String baseSessionId = sessionId.substring(0, index);
+          String newSessionId = baseSessionId + "." + localJvmRoute;
+          session.setId(newSessionId);
+
+          // Change the request's session id
+          request.changeSessionId(newSessionId);
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/48552465/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/LocalStrings.properties
----------------------------------------------------------------------
diff --git a/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/LocalStrings.properties b/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/LocalStrings.properties
new file mode 100644
index 0000000..679cad3
--- /dev/null
+++ b/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/LocalStrings.properties
@@ -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.
+
+deltaSession.commit.ise=commit: Session {0} already invalidated

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/48552465/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/PeerToPeerCacheLifecycleListener.java
----------------------------------------------------------------------
diff --git a/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/PeerToPeerCacheLifecycleListener.java b/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/PeerToPeerCacheLifecycleListener.java
new file mode 100644
index 0000000..a86486c
--- /dev/null
+++ b/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/PeerToPeerCacheLifecycleListener.java
@@ -0,0 +1,29 @@
+/*
+* 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.modules.session.catalina;
+
+import com.gemstone.gemfire.modules.session.bootstrap.PeerToPeerCache;
+
+/**
+ * This is a thin wrapper around a peer-to-peer cache.
+ */
+public class PeerToPeerCacheLifecycleListener extends AbstractCacheLifecycleListener {
+
+  public PeerToPeerCacheLifecycleListener() {
+    cache = PeerToPeerCache.getInstance();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/48552465/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/PeerToPeerSessionCache.java
----------------------------------------------------------------------
diff --git a/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/PeerToPeerSessionCache.java b/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/PeerToPeerSessionCache.java
new file mode 100644
index 0000000..75b5d41
--- /dev/null
+++ b/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/PeerToPeerSessionCache.java
@@ -0,0 +1,215 @@
+/*
+* 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.modules.session.catalina;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.GemFireCache;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionFactory;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.cache.execute.Execution;
+import com.gemstone.gemfire.cache.execute.FunctionService;
+import com.gemstone.gemfire.cache.execute.ResultCollector;
+import com.gemstone.gemfire.modules.session.catalina.callback.LocalSessionCacheLoader;
+import com.gemstone.gemfire.modules.session.catalina.callback.LocalSessionCacheWriter;
+import com.gemstone.gemfire.modules.session.catalina.callback.SessionExpirationCacheListener;
+import com.gemstone.gemfire.modules.util.RegionConfiguration;
+import com.gemstone.gemfire.modules.util.RegionHelper;
+import com.gemstone.gemfire.modules.util.SessionCustomExpiry;
+import com.gemstone.gemfire.modules.util.TouchPartitionedRegionEntriesFunction;
+import com.gemstone.gemfire.modules.util.TouchReplicatedRegionEntriesFunction;
+
+import javax.servlet.http.HttpSession;
+import java.util.Set;
+
+public class PeerToPeerSessionCache extends AbstractSessionCache {
+
+  private Cache cache;
+
+  protected static final String DEFAULT_REGION_ATTRIBUTES_ID = RegionShortcut.REPLICATE.toString();
+
+  protected static final boolean DEFAULT_ENABLE_LOCAL_CACHE = false;
+
+  public PeerToPeerSessionCache(SessionManager sessionManager, Cache cache) {
+    super(sessionManager);
+    this.cache = cache;
+  }
+
+  @Override
+  public void initialize() {
+    // Register Functions
+    registerFunctions();
+
+    // Create or retrieve the region
+    createOrRetrieveRegion();
+
+    // If local cache is enabled, create the local region fronting the session region
+    // and set it as the operating region; otherwise, use the session region directly
+    // as the operating region.
+    this.operatingRegion = getSessionManager().getEnableLocalCache() ? createOrRetrieveLocalRegion() : this.sessionRegion;
+
+    // Create or retrieve the statistics
+    createStatistics();
+  }
+
+  @Override
+  public String getDefaultRegionAttributesId() {
+    return DEFAULT_REGION_ATTRIBUTES_ID;
+  }
+
+  @Override
+  public boolean getDefaultEnableLocalCache() {
+    return DEFAULT_ENABLE_LOCAL_CACHE;
+  }
+
+  @Override
+  public void touchSessions(Set<String> sessionIds) {
+    // Get the region attributes id to determine the region type. This is
+    // problematic since the region attributes id doesn't really define the
+    // region type. This should look at the actual session region.
+    String regionAttributesID = getSessionManager().getRegionAttributesId().toLowerCase();
+
+    // Invoke the appropriate function depending on the type of region
+    ResultCollector collector = null;
+    if (regionAttributesID.startsWith("partition")) {
+      // Execute the partitioned touch function on the primary server(s)
+      Execution execution = FunctionService.onRegion(getSessionRegion()).withFilter(sessionIds);
+      collector = execution.execute(TouchPartitionedRegionEntriesFunction.ID, true, false, true);
+    } else {
+      // Execute the member touch function on all the server(s)
+      Execution execution = FunctionService.onMembers(getCache().getDistributedSystem())
+          .withArgs(new Object[]{this.sessionRegion.getFullPath(), sessionIds});
+      collector = execution.execute(TouchReplicatedRegionEntriesFunction.ID, true, false, false);
+    }
+
+    // Get the result
+    try {
+      collector.getResult();
+    } catch (Exception e) {
+      // If an exception occurs in the function, log it.
+      getSessionManager().getLogger().warn("Caught unexpected exception:", e);
+    }
+  }
+
+  @Override
+  public boolean isPeerToPeer() {
+    return true;
+  }
+
+  @Override
+  public boolean isClientServer() {
+    return false;
+  }
+
+  @Override
+  public Set<String> keySet() {
+    return getSessionRegion().keySet();
+  }
+
+  @Override
+  public int size() {
+    return getSessionRegion().size();
+  }
+
+  @Override
+  public GemFireCache getCache() {
+    return this.cache;
+  }
+
+  /**
+   * For peer-to-peer the backing cache *is* what's embedded in tomcat so it's always available
+   *
+   * @return
+   */
+  @Override
+  public boolean isBackingCacheAvailable() {
+    return true;
+  }
+
+  private void registerFunctions() {
+    // Register the touch partitioned region entries function if it is not already registered
+    if (!FunctionService.isRegistered(TouchPartitionedRegionEntriesFunction.ID)) {
+      FunctionService.registerFunction(new TouchPartitionedRegionEntriesFunction());
+    }
+
+    // Register the touch replicated region entries function if it is not already registered
+    if (!FunctionService.isRegistered(TouchReplicatedRegionEntriesFunction.ID)) {
+      FunctionService.registerFunction(new TouchReplicatedRegionEntriesFunction());
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  protected void createOrRetrieveRegion() {
+    // Create the RegionConfiguration
+    RegionConfiguration configuration = createRegionConfiguration();
+    configuration.setSessionExpirationCacheListener(true);
+
+    // Attempt to retrieve the region
+    // If it already exists, validate it
+    // If it doesn't already exist, create it
+    Region region = this.cache.getRegion(getSessionManager().getRegionName());
+    if (region == null) {
+      // Create the region
+      region = RegionHelper.createRegion((Cache) getCache(), configuration);
+      if (getSessionManager().getLogger().isDebugEnabled()) {
+        getSessionManager().getLogger().debug("Created new session region: " + region);
+      }
+    } else {
+      // Validate the existing region
+      if (getSessionManager().getLogger().isDebugEnabled()) {
+        getSessionManager().getLogger().debug("Retrieved existing session region: " + region);
+      }
+      RegionHelper.validateRegion((Cache) getCache(), configuration, region);
+    }
+
+    // Set the session region
+    this.sessionRegion = region;
+  }
+
+  private Region<String, HttpSession> createOrRetrieveLocalRegion() {
+    // Attempt to retrieve the fronting region
+    String frontingRegionName = this.sessionRegion.getName() + "_local";
+    Region<String, HttpSession> frontingRegion = this.cache.getRegion(frontingRegionName);
+    if (frontingRegion == null) {
+      // Create the region factory
+      RegionFactory<String, HttpSession> factory = this.cache.createRegionFactory(RegionShortcut.LOCAL_HEAP_LRU);
+
+      // Add the cache loader and writer
+      factory.setCacheLoader(new LocalSessionCacheLoader(this.sessionRegion));
+      factory.setCacheWriter(new LocalSessionCacheWriter(this.sessionRegion));
+
+      // Set the expiration time, action and listener if necessary
+      int maxInactiveInterval = getSessionManager().getMaxInactiveInterval();
+      if (maxInactiveInterval != RegionConfiguration.DEFAULT_MAX_INACTIVE_INTERVAL) {
+        factory.setStatisticsEnabled(true);
+        factory.setCustomEntryIdleTimeout(new SessionCustomExpiry());
+        factory.addCacheListener(new SessionExpirationCacheListener());
+      }
+
+      // Create the region
+      frontingRegion = factory.create(frontingRegionName);
+      if (getSessionManager().getLogger().isDebugEnabled()) {
+        getSessionManager().getLogger().debug("Created new local session region: " + frontingRegion);
+      }
+    } else {
+      if (getSessionManager().getLogger().isDebugEnabled()) {
+        getSessionManager().getLogger().debug("Retrieved existing local session region: " + frontingRegion);
+      }
+    }
+    return frontingRegion;
+  }
+}  

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/48552465/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/SessionCache.java
----------------------------------------------------------------------
diff --git a/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/SessionCache.java b/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/SessionCache.java
new file mode 100644
index 0000000..b1128e7
--- /dev/null
+++ b/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/SessionCache.java
@@ -0,0 +1,64 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*      http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+package com.gemstone.gemfire.modules.session.catalina;
+
+import com.gemstone.gemfire.cache.GemFireCache;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.modules.session.catalina.internal.DeltaSessionStatistics;
+import org.apache.catalina.Session;
+
+import javax.servlet.http.HttpSession;
+import java.util.Set;
+
+public interface SessionCache {
+
+  public void initialize();
+
+  public String getDefaultRegionAttributesId();
+
+  public boolean getDefaultEnableLocalCache();
+
+  public String getSessionRegionName();
+
+  public String getOperatingRegionName();
+
+  public void putSession(Session session);
+
+  public HttpSession getSession(String sessionId);
+
+  public void destroySession(String sessionId);
+
+  public void touchSessions(Set<String> sessionIds);
+
+  public DeltaSessionStatistics getStatistics();
+
+  public GemFireCache getCache();
+
+  public Region<String, HttpSession> getSessionRegion();
+
+  public Region<String, HttpSession> getOperatingRegion();
+
+  public boolean isPeerToPeer();
+
+  public boolean isClientServer();
+
+  public Set<String> keySet();
+
+  public int size();
+
+  public boolean isBackingCacheAvailable();
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/48552465/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/SessionManager.java
----------------------------------------------------------------------
diff --git a/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/SessionManager.java b/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/SessionManager.java
new file mode 100644
index 0000000..e3d6c7e
--- /dev/null
+++ b/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/SessionManager.java
@@ -0,0 +1,48 @@
+/*
+* 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.modules.session.catalina;
+
+import org.apache.juli.logging.Log;
+
+public interface SessionManager {
+
+  public String getRegionName();
+
+  public String getRegionAttributesId();
+
+  public int getMaxInactiveInterval();
+
+  public boolean getEnableGatewayReplication();
+
+  public boolean getEnableGatewayDeltaReplication();
+
+  public boolean getEnableDebugListener();
+
+  public boolean getEnableLocalCache();
+
+  public boolean isCommitValveEnabled();
+
+  public boolean isCommitValveFailfastEnabled();
+
+  public boolean isBackingCacheAvailable();
+
+  public boolean getPreferDeserializedForm();
+
+  public String getStatisticsName();
+
+  public Log getLogger();
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/48552465/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/Tomcat6DeltaSessionManager.java
----------------------------------------------------------------------
diff --git a/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/Tomcat6DeltaSessionManager.java b/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/Tomcat6DeltaSessionManager.java
new file mode 100644
index 0000000..d5b5991
--- /dev/null
+++ b/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/Tomcat6DeltaSessionManager.java
@@ -0,0 +1,98 @@
+/*
+* 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.modules.session.catalina;
+
+import org.apache.catalina.LifecycleException;
+
+public class Tomcat6DeltaSessionManager extends DeltaSessionManager {
+
+  /**
+   * Prepare for the beginning of active use of the public methods of this component.  This method should be called
+   * after <code>configure()</code>, and before any of the public methods of the component are utilized.
+   *
+   * @throws LifecycleException if this component detects a fatal error that prevents this component from being used
+   */
+  @Override
+  public void start() throws LifecycleException {
+    if (getLogger().isDebugEnabled()) {
+      getLogger().debug(this + ": Starting");
+    }
+    if (this.started.get()) {
+      return;
+    }
+    this.lifecycle.fireLifecycleEvent(START_EVENT, null);
+    try {
+      init();
+    } catch (Throwable t) {
+      getLogger().error(t.getMessage(), t);
+    }
+
+    // Register our various valves
+    registerJvmRouteBinderValve();
+
+    if (isCommitValveEnabled()) {
+      registerCommitSessionValve();
+    }
+
+    // Initialize the appropriate session cache interface
+    initializeSessionCache();
+
+    // Create the timer and schedule tasks
+    scheduleTimerTasks();
+
+    this.started.set(true);
+  }
+
+  /**
+   * Gracefully terminate the active use of the public methods of this component.  This method should be the last one
+   * called on a given instance of this component.
+   *
+   * @throws LifecycleException if this component detects a fatal error that needs to be reported
+   */
+  @Override
+  public void stop() throws LifecycleException {
+    if (getLogger().isDebugEnabled()) {
+      getLogger().debug(this + ": Stopping");
+    }
+    this.started.set(false);
+    this.lifecycle.fireLifecycleEvent(STOP_EVENT, null);
+
+    // StandardManager expires all Sessions here.
+    // All Sessions are not known by this Manager.
+
+    // Require a new random number generator if we are restarted
+    this.random = null;
+
+    // Remove from RMI registry
+    if (this.initialized) {
+      destroy();
+    }
+
+    // Clear any sessions to be touched
+    getSessionsToTouch().clear();
+
+    // Cancel the timer
+    cancelTimer();
+
+    // Unregister the JVM route valve
+    unregisterJvmRouteBinderValve();
+
+    if (isCommitValveEnabled()) {
+      unregisterCommitSessionValve();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/48552465/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/callback/LocalSessionCacheLoader.java
----------------------------------------------------------------------
diff --git a/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/callback/LocalSessionCacheLoader.java b/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/callback/LocalSessionCacheLoader.java
new file mode 100644
index 0000000..a4afad4
--- /dev/null
+++ b/extensions/gemfire-modules/src/main/java/com/gemstone/gemfire/modules/session/catalina/callback/LocalSessionCacheLoader.java
@@ -0,0 +1,45 @@
+/*
+* 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.modules.session.catalina.callback;
+
+import com.gemstone.gemfire.cache.CacheLoader;
+import com.gemstone.gemfire.cache.CacheLoaderException;
+import com.gemstone.gemfire.cache.Declarable;
+import com.gemstone.gemfire.cache.LoaderHelper;
+import com.gemstone.gemfire.cache.Region;
+
+import javax.servlet.http.HttpSession;
+import java.util.Properties;
+
+public class LocalSessionCacheLoader implements CacheLoader<String, HttpSession>, Declarable {
+
+  private final Region<String, HttpSession> backingRegion;
+
+  public LocalSessionCacheLoader(Region<String, HttpSession> backingRegion) {
+    this.backingRegion = backingRegion;
+  }
+
+  public HttpSession load(LoaderHelper<String, HttpSession> helper) throws CacheLoaderException {
+    return this.backingRegion.get(helper.getKey());
+  }
+
+  public void close() {
+  }
+
+  public void init(Properties p) {
+  }
+}