You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by br...@apache.org on 2021/04/09 23:28:14 UTC

[geode] 08/12: GEODE-8513: Remove (de)serialization of local sessions. (#5532)

This is an automated email from the ASF dual-hosted git repository.

bross pushed a commit to branch support/1.12
in repository https://gitbox.apache.org/repos/asf/geode.git

commit 7d3dc5dce8cfa35262592ce0073484be44d35f63
Author: Jacob Barrett <jb...@pivotal.io>
AuthorDate: Tue Sep 29 09:57:53 2020 -0700

    GEODE-8513: Remove (de)serialization of local sessions. (#5532)
    
    * Removes query that resulted in casting issue in antlr.
    * Clears all locally cached entries.
    * Remove dead code and tests.
    
    (cherry picked from commit d4b9ecd8569281f257e384d397422e7ef4336fb9)
---
 .../catalina/AbstractDeltaSessionManagerTest.java  | 223 +------------
 .../session/catalina/DeltaSessionManager.java      | 351 +--------------------
 2 files changed, 14 insertions(+), 560 deletions(-)

diff --git a/extensions/geode-modules-test/src/main/java/org/apache/geode/modules/session/catalina/AbstractDeltaSessionManagerTest.java b/extensions/geode-modules-test/src/main/java/org/apache/geode/modules/session/catalina/AbstractDeltaSessionManagerTest.java
index d5fea45..c28256c 100644
--- a/extensions/geode-modules-test/src/main/java/org/apache/geode/modules/session/catalina/AbstractDeltaSessionManagerTest.java
+++ b/extensions/geode-modules-test/src/main/java/org/apache/geode/modules/session/catalina/AbstractDeltaSessionManagerTest.java
@@ -17,50 +17,28 @@ package org.apache.geode.modules.session.catalina;
 
 import static org.apache.geode.modules.util.RegionConfiguration.DEFAULT_MAX_INACTIVE_INTERVAL;
 import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.Assertions.assertThatThrownBy;
-import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyInt;
 import static org.mockito.ArgumentMatchers.anyString;
-import static org.mockito.Mockito.doNothing;
 import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
-import static org.mockito.Mockito.withSettings;
 
 import java.beans.PropertyChangeEvent;
-import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
 import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
 import java.util.HashSet;
 import java.util.Set;
 
 import javax.servlet.http.HttpSession;
 
 import org.apache.catalina.Context;
-import org.apache.catalina.Loader;
 import org.apache.catalina.Session;
-import org.apache.catalina.session.StandardSession;
 import org.apache.juli.logging.Log;
 import org.junit.Test;
 
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.Region;
-import org.apache.geode.cache.query.FunctionDomainException;
-import org.apache.geode.cache.query.NameResolutionException;
-import org.apache.geode.cache.query.Query;
-import org.apache.geode.cache.query.QueryInvocationTargetException;
-import org.apache.geode.cache.query.SelectResults;
-import org.apache.geode.cache.query.TypeMismatchException;
-import org.apache.geode.cache.query.internal.InternalQueryService;
-import org.apache.geode.cache.query.internal.LinkedResultSet;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.modules.session.catalina.internal.DeltaSessionStatistics;
 
@@ -217,142 +195,15 @@ public abstract class AbstractDeltaSessionManagerTest<DeltaSessionManagerT exten
   }
 
   @Test
-  public void loadActivatesAndAddsSingleSessionWithValidIdAndMoreRecentAccessTime()
-      throws IOException, ClassNotFoundException {
-    final String contextPath = "contextPath";
-    final String expectedStoreDir = "";
-    final DeltaSession newSession = mock(DeltaSession.class);
-    final DeltaSession existingSession = mock(DeltaSession.class);
-
-    prepareMocksForLoadTest(contextPath, newSession, existingSession, expectedStoreDir);
-
-    manager.load();
-
-    verify(newSession).activate();
-    verify(manager).add(newSession);
-  }
-
-  @Test
-  public void loadLogsWarningAndDoesNotAddSessionWhenSessionStoreNotFound()
-      throws IOException, ClassNotFoundException {
-    final String contextPath = "contextPath";
-    final String expectedStoreDir = "";
-    final DeltaSession newSession = mock(DeltaSession.class);
-    final DeltaSession existingSession = mock(DeltaSession.class);
-
-    prepareMocksForLoadTest(contextPath, newSession, existingSession, expectedStoreDir);
-
-    doReturn(null).when(manager).getFileAtPath(any(), any());
-
-    manager.load();
-
-    verify(logger).debug("No session store file found");
-    verify(manager, times(0)).add(any());
-  }
-
-  @Test
-  public void loadDoesNotAddSessionToManagerWithValidIdAndLessRecentAccessTime()
-      throws IOException, ClassNotFoundException {
-    final String contextPath = "contextPath";
-    final String expectedStoreDir = "";
-    final DeltaSession newSession = mock(DeltaSession.class);
-    final DeltaSession existingSession = mock(DeltaSession.class);
-
-    prepareMocksForLoadTest(contextPath, newSession, existingSession, expectedStoreDir);
-
-    when(existingSession.getLastAccessedTime()).thenReturn(2L);
-
-    manager.load();
-
-    verify(newSession, times(0)).activate();
-    verify(manager, times(0)).add(newSession);
-  }
-
-  @Test
-  public void unloadWritesSingleSessionToDiskWhenIdIsValid()
-      throws IOException, NameResolutionException, TypeMismatchException,
-      QueryInvocationTargetException, FunctionDomainException {
-    final String sessionId = "sessionId";
-    final DeltaSession session = mock(DeltaSession.class);
-    final FileOutputStream fos = mock(FileOutputStream.class);
-    final BufferedOutputStream bos = mock(BufferedOutputStream.class);
-    final ObjectOutputStream oos = mock(ObjectOutputStream.class);
-
-    prepareMocksForUnloadTest(sessionId, fos, bos, oos, session);
-
-    manager.unload();
-
-    verify((StandardSession) session).writeObjectData(oos);
-  }
-
-  @Test
-  public void unloadDoesNotWriteSessionToDiskAndClosesOutputStreamsWhenOutputStreamThrowsIOException()
-      throws IOException, NameResolutionException, TypeMismatchException,
-      QueryInvocationTargetException, FunctionDomainException {
-    final String sessionId = "sessionId";
-    final DeltaSession session = mock(DeltaSession.class);
-    final FileOutputStream fos = mock(FileOutputStream.class);
-    final BufferedOutputStream bos = mock(BufferedOutputStream.class);
-    final ObjectOutputStream oos = mock(ObjectOutputStream.class);
-
-    prepareMocksForUnloadTest(sessionId, fos, bos, oos, session);
-
-    final String exceptionMessage = "Output Stream IOException";
-
-    final IOException exception = new IOException(exceptionMessage);
-
-    doThrow(exception).when(manager).getObjectOutputStream(bos);
-
-    assertThatThrownBy(() -> manager.unload()).isInstanceOf(IOException.class)
-        .hasMessage(exceptionMessage);
-
-    verify((StandardSession) session, times(0)).writeObjectData(oos);
-    verify(bos).close();
-    verify(fos).close();
-  }
-
-  @Test
-  public void unloadDoesNotWriteSessionToDiskAndClosesOutputStreamsWhenSessionIsWrongClass()
-      throws IOException, NameResolutionException, TypeMismatchException,
-      QueryInvocationTargetException, FunctionDomainException {
-    final String sessionId = "sessionId";
-    final DeltaSession session = mock(DeltaSession.class);
-    final FileOutputStream fos = mock(FileOutputStream.class);
-    final BufferedOutputStream bos = mock(BufferedOutputStream.class);
-    final ObjectOutputStream oos = mock(ObjectOutputStream.class);
-
-    prepareMocksForUnloadTest(sessionId, fos, bos, oos, session);
-
-    final Session invalidSession =
-        mock(Session.class, withSettings().extraInterfaces(DeltaSessionInterface.class));
-
-    doReturn(invalidSession).when(manager).findSession(sessionId);
-
-    assertThatThrownBy(() -> manager.unload()).isInstanceOf(IOException.class);
-
-    verify((StandardSession) session, times(0)).writeObjectData(oos);
-    verify(oos).close();
-  }
-
-  @Test
   public void successfulUnloadWithClientServerSessionCachePerformsLocalDestroy()
-      throws IOException, NameResolutionException, TypeMismatchException,
-      QueryInvocationTargetException, FunctionDomainException {
-    final String sessionId = "sessionId";
-    final DeltaSession session = mock(DeltaSession.class);
-    final FileOutputStream fos = mock(FileOutputStream.class);
-    final BufferedOutputStream bos = mock(BufferedOutputStream.class);
-    final ObjectOutputStream oos = mock(ObjectOutputStream.class);
-
-    prepareMocksForUnloadTest(sessionId, fos, bos, oos, session);
-
+      throws IOException {
+    when(sessionCache.getCache()).thenReturn(cache);
+    when(context.getPath()).thenReturn("contextPath");
     when(sessionCache.isClientServer()).thenReturn(true);
-    when(session.getId()).thenReturn(sessionId);
 
     manager.unload();
 
-    verify((StandardSession) session).writeObjectData(oos);
-    verify(operatingRegion).localDestroy(sessionId);
+    verify(operatingRegion).localClear();
   }
 
   @Test
@@ -403,70 +254,4 @@ public abstract class AbstractDeltaSessionManagerTest<DeltaSessionManagerT exten
     verify(manager).setMaxInactiveInterval(oldValue);
   }
 
-  public void prepareMocksForUnloadTest(final String sessionId, final FileOutputStream fos,
-      final BufferedOutputStream bos, final ObjectOutputStream oos,
-      final DeltaSession session)
-      throws NameResolutionException, TypeMismatchException, QueryInvocationTargetException,
-      FunctionDomainException, IOException {
-    final String regionName = "regionName";
-    final String contextPath = "contextPath";
-    final String catalinaBaseSystemProp = "Catalina/Base";
-    final String systemFileSeparator = "/";
-    final String expectedStoreDir = catalinaBaseSystemProp + systemFileSeparator + "temp";
-
-    final InternalQueryService queryService = mock(InternalQueryService.class);
-    final Query query = mock(Query.class);
-    final File store = mock(File.class);
-    final SelectResults results = new LinkedResultSet();
-
-    when(sessionCache.getCache()).thenReturn(cache);
-    when(context.getPath()).thenReturn(contextPath);
-    when(cache.getQueryService()).thenReturn(queryService);
-    when(queryService.newQuery(anyString())).thenReturn(query);
-    when(query.execute()).thenReturn(results);
-    doReturn(catalinaBaseSystemProp).when(manager)
-        .getSystemPropertyValue(DeltaSessionManager.catalinaBaseSystemProperty);
-    doReturn(systemFileSeparator).when(manager)
-        .getSystemPropertyValue(DeltaSessionManager.fileSeparatorSystemProperty);
-    doReturn(store).when(manager).getFileAtPath(expectedStoreDir, contextPath);
-    doReturn(fos).when(manager).getFileOutputStream(store);
-    doReturn(bos).when(manager).getBufferedOutputStream(fos);
-    doReturn(oos).when(manager).getObjectOutputStream(bos);
-    doReturn(regionName).when(manager).getRegionName();
-    doReturn(session).when(manager).findSession(sessionId);
-    doNothing().when(manager).writeToObjectOutputStream(any(), any());
-
-    results.add(sessionId);
-  }
-
-  public void prepareMocksForLoadTest(final String contextPath, final DeltaSession newSession,
-      final DeltaSession existingSession, String expectedStoreDir)
-      throws IOException, ClassNotFoundException {
-    final String catalinaBaseSystemProp = "Catalina/Base";
-    final String systemFileSeparator = "/";
-    expectedStoreDir = catalinaBaseSystemProp + systemFileSeparator + "temp";
-    final String newSessionId = "newSessionId";
-
-    final File store = mock(File.class);
-    final FileInputStream fis = mock(FileInputStream.class);
-    final BufferedInputStream bis = mock(BufferedInputStream.class);
-    final ObjectInputStream ois = mock(ObjectInputStream.class);
-    final Loader loader = mock(Loader.class);
-
-    when(context.getPath()).thenReturn(contextPath);
-    when(context.getLoader()).thenReturn(loader);
-    when(newSession.getId()).thenReturn(newSessionId);
-    when(newSession.getLastAccessedTime()).thenReturn(1L);
-    when(newSession.isValid()).thenReturn(true);
-    when(existingSession.getLastAccessedTime()).thenReturn(0L);
-    doReturn(catalinaBaseSystemProp).when(manager).getSystemPropertyValue("catalina.base");
-    doReturn(systemFileSeparator).when(manager).getSystemPropertyValue("file.separator");
-    doReturn(store).when(manager).getFileAtPath(expectedStoreDir, contextPath);
-    doReturn(fis).when(manager).getFileInputStream(store);
-    doReturn(bis).when(manager).getBufferedInputStream(fis);
-    doReturn(ois).when(manager).getObjectInputStream(bis);
-    doReturn(1).when(manager).getSessionCountFromObjectInputStream(ois);
-    doReturn(newSession).when(manager).getNewSession();
-    doReturn(existingSession).when(operatingRegion).get(newSessionId);
-  }
 }
diff --git a/extensions/geode-modules/src/main/java/org/apache/geode/modules/session/catalina/DeltaSessionManager.java b/extensions/geode-modules/src/main/java/org/apache/geode/modules/session/catalina/DeltaSessionManager.java
index 67455af..a42b0d8 100644
--- a/extensions/geode-modules/src/main/java/org/apache/geode/modules/session/catalina/DeltaSessionManager.java
+++ b/extensions/geode-modules/src/main/java/org/apache/geode/modules/session/catalina/DeltaSessionManager.java
@@ -16,20 +16,10 @@ package org.apache.geode.modules.session.catalina;
 
 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.List;
 import java.util.Set;
 import java.util.Timer;
 import java.util.TimerTask;
@@ -37,28 +27,19 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import javax.servlet.http.HttpSession;
-
 import org.apache.catalina.Container;
 import org.apache.catalina.Context;
 import org.apache.catalina.Lifecycle;
-import org.apache.catalina.Loader;
 import org.apache.catalina.Pipeline;
 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.juli.logging.Log;
 import org.apache.juli.logging.LogFactory;
 
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.CacheFactory;
-import org.apache.geode.cache.EntryNotFoundException;
-import org.apache.geode.cache.Region;
-import org.apache.geode.cache.query.Query;
-import org.apache.geode.cache.query.QueryService;
-import org.apache.geode.cache.query.SelectResults;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.modules.session.catalina.internal.DeltaSessionStatistics;
 import org.apache.geode.modules.util.ContextMapper;
@@ -586,13 +567,12 @@ public abstract class DeltaSessionManager<CommitSessionValveT extends AbstractCo
 
   @Override
   public void load() throws ClassNotFoundException, IOException {
-    doLoad();
     ContextMapper.addContext(getContextName(), this);
   }
 
   @Override
   public void unload() throws IOException {
-    doUnload();
+    clearLocalCache();
     ContextMapper.removeContext(getContextName());
   }
 
@@ -690,333 +670,22 @@ public abstract class DeltaSessionManager<CommitSessionValveT extends AbstractCo
   }
 
   /**
-   * 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
+   * Clear the local cache to avoid ClassCastException if container is being reloaded.
    */
-  private void doUnload() throws IOException {
-    QueryService querySvc = getSessionCache().getCache().getQueryService();
-    Context context = getTheContext();
-
-    if (context == null) {
-      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() + "'");
-
-    if (getLogger().isDebugEnabled()) {
-      getLogger().debug("Query: " + query.getQueryString());
-    }
-
-    SelectResults<String> 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;
-    final ObjectOutputStream oos;
-    boolean error = false;
-    try {
-      fos = getFileOutputStream(store);
-      bos = getBufferedOutputStream(fos);
-      oos = getObjectOutputStream(bos);
-    } catch (IOException e) {
-      error = true;
-      getLogger().error("Exception unloading sessions", e);
-      throw e;
-    } finally {
-      if (error) {
-        if (bos != null) {
-          try {
-            bos.close();
-          } catch (IOException ioe) {
-            // Ignore
-          }
-        }
-        if (fos != null) {
-          try {
-            fos.close();
-          } catch (IOException ioe) {
-            // Ignore
-          }
-        }
-      }
-    }
-
-    ArrayList<DeltaSessionInterface> list = new ArrayList<>();
-    for (final String id : results) {
-      DeltaSessionInterface session = (DeltaSessionInterface) findSession(id);
-      if (session != null) {
-        list.add(session);
-      }
-    }
+  private void clearLocalCache() {
+    final Log logger = getLogger();
+    final boolean debugEnabled = logger.isDebugEnabled();
 
-    // Write the number of active sessions, followed by the details
-    if (getLogger().isDebugEnabled()) {
-      getLogger().debug("Unloading " + list.size() + " sessions");
-    }
-    try {
-      writeToObjectOutputStream(oos, list);
-      for (DeltaSessionInterface session : list) {
-        if (session instanceof StandardSession) {
-          StandardSession standardSession = (StandardSession) session;
-          standardSession.passivate();
-          standardSession.writeObjectData(oos);
-        } else {
-          // All DeltaSessionInterfaces as of Geode 1.0 should be based on StandardSession
-          throw new IOException("Session should be of type StandardSession");
-        }
-      }
-    } 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 (DeltaSessionInterface session : list) {
-        if (getLogger().isDebugEnabled()) {
-          getLogger().debug("Locally destroying session " + session.getId());
-        }
-        try {
-          getSessionCache().getOperatingRegion().localDestroy(session.getId());
-        } catch (EntryNotFoundException ex) {
-          // This can be thrown if an entry is evicted during or immediately after a session is
-          // written
-          // to disk. This isn't a problem, but the resulting exception messages can be confusing in
-          // testing
-        }
+      if (debugEnabled) {
+        logger.debug("Locally clearing sessions.");
       }
+      getSessionCache().getOperatingRegion().localClear();
     }
 
-    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
-   */
-  private void doLoad() throws ClassNotFoundException, IOException {
-    Context context = getTheContext();
-    if (context == null) {
-      return;
+    if (debugEnabled) {
+      logger.debug("Unloading complete");
     }
-
-    // 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;
-    Loader loader = null;
-    ClassLoader classLoader = null;
-    try {
-      fis = getFileInputStream(store);
-      bis = getBufferedInputStream(fis);
-      if (getTheContext() != null) {
-        loader = getTheContext().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 = getObjectInputStream(bis);
-      }
-    } catch (FileNotFoundException e) {
-      if (getLogger().isDebugEnabled()) {
-        getLogger().debug("No persisted data file found");
-      }
-      return;
-    } catch (IOException e) {
-      getLogger().error("Exception loading sessions", e);
-      try {
-        fis.close();
-      } catch (IOException f) {
-        // Ignore
-      }
-      try {
-        bis.close();
-      } catch (IOException f) {
-        // Ignore
-      }
-      throw e;
-    }
-
-    // Load the previously unloaded active sessions
-    try {
-      int n = getSessionCountFromObjectInputStream(ois);
-      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);
-
-        final Region<String, HttpSession> region = getSessionCache().getOperatingRegion();
-        final DeltaSessionInterface existingSession =
-            (DeltaSessionInterface) 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 | 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()) {
-        if (!store.delete()) {
-          getLogger().warn("Couldn't delete persistent storage file " + store.getAbsolutePath());
-        }
-      }
-    }
-  }
-
-  /**
-   * Return a File object representing the pathname to our persistence file, if any.
-   */
-  private File sessionStore(String ctxPath) {
-    String storeDir = getSystemPropertyValue(catalinaBaseSystemProperty);
-    if (storeDir == null || storeDir.isEmpty()) {
-      storeDir = getSystemPropertyValue(javaTempDirSystemProperty);
-    } else {
-      storeDir += getSystemPropertyValue(fileSeparatorSystemProperty) + "temp";
-    }
-
-    return getFileAtPath(storeDir, ctxPath);
-  }
-
-  String getSystemPropertyValue(String propertyKey) {
-    return System.getProperty(propertyKey);
-  }
-
-  File getFileAtPath(String storeDir, String ctxPath) {
-    return (new File(storeDir, ctxPath.replaceAll("/", "_") + ".sessions.ser"));
-  }
-
-  FileInputStream getFileInputStream(File file) throws FileNotFoundException {
-    return new FileInputStream(file.getAbsolutePath());
-  }
-
-  BufferedInputStream getBufferedInputStream(FileInputStream fis) {
-    return new BufferedInputStream(fis);
-  }
-
-  ObjectInputStream getObjectInputStream(BufferedInputStream bis) throws IOException {
-    return new ObjectInputStream(bis);
-  }
-
-  FileOutputStream getFileOutputStream(File file) throws FileNotFoundException {
-    return new FileOutputStream(file.getAbsolutePath());
-  }
-
-  BufferedOutputStream getBufferedOutputStream(FileOutputStream fos) {
-    return new BufferedOutputStream(fos);
-  }
-
-  ObjectOutputStream getObjectOutputStream(BufferedOutputStream bos) throws IOException {
-    return new ObjectOutputStream(bos);
-  }
-
-  void writeToObjectOutputStream(ObjectOutputStream oos, List<?> listToWrite) throws IOException {
-    oos.writeObject(listToWrite.size());
-  }
-
-  int getSessionCountFromObjectInputStream(ObjectInputStream ois)
-      throws IOException, ClassNotFoundException {
-    return (Integer) ois.readObject();
   }
 
   @Override