You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@manifoldcf.apache.org by kw...@apache.org on 2013/12/02 20:47:15 UTC

svn commit: r1547167 - in /manifoldcf/trunk: ./ framework/agents/src/main/java/org/apache/manifoldcf/agents/ framework/agents/src/main/java/org/apache/manifoldcf/agents/system/ framework/combined-service/src/main/java/org/apache/manifoldcf/combinedserv...

Author: kwright
Date: Mon Dec  2 19:47:15 2013
New Revision: 1547167

URL: http://svn.apache.org/r1547167
Log:
As part of CONNECTORS-781, create at least one multiprocess test, and debug multiprocess code.

Added:
    manifoldcf/trunk/framework/agents/src/main/java/org/apache/manifoldcf/agents/system/AgentsDaemon.java
      - copied unchanged from r1547166, manifoldcf/branches/CONNECTORS-781/framework/agents/src/main/java/org/apache/manifoldcf/agents/system/AgentsDaemon.java
Modified:
    manifoldcf/trunk/   (props changed)
    manifoldcf/trunk/framework/agents/src/main/java/org/apache/manifoldcf/agents/AgentRun.java
    manifoldcf/trunk/framework/agents/src/main/java/org/apache/manifoldcf/agents/AgentStop.java
    manifoldcf/trunk/framework/agents/src/main/java/org/apache/manifoldcf/agents/system/ManifoldCF.java
    manifoldcf/trunk/framework/combined-service/src/main/java/org/apache/manifoldcf/combinedservice/ServletListener.java
    manifoldcf/trunk/framework/jetty-runner/src/main/java/org/apache/manifoldcf/jettyrunner/ManifoldCFJettyRunner.java
    manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/jobs/EventManager.java
    manifoldcf/trunk/framework/pull-agent/src/test/java/org/apache/manifoldcf/crawler/tests/ManifoldCFInstance.java
    manifoldcf/trunk/framework/pull-agent/src/test/java/org/apache/manifoldcf/crawler/tests/SchedulerHSQLDBTest.java
    manifoldcf/trunk/framework/pull-agent/src/test/java/org/apache/manifoldcf/crawler/tests/SchedulerTester.java

Propchange: manifoldcf/trunk/
------------------------------------------------------------------------------
  Merged /manifoldcf/branches/CONNECTORS-781:r1546969-1547166

Modified: manifoldcf/trunk/framework/agents/src/main/java/org/apache/manifoldcf/agents/AgentRun.java
URL: http://svn.apache.org/viewvc/manifoldcf/trunk/framework/agents/src/main/java/org/apache/manifoldcf/agents/AgentRun.java?rev=1547167&r1=1547166&r2=1547167&view=diff
==============================================================================
--- manifoldcf/trunk/framework/agents/src/main/java/org/apache/manifoldcf/agents/AgentRun.java (original)
+++ manifoldcf/trunk/framework/agents/src/main/java/org/apache/manifoldcf/agents/AgentRun.java Mon Dec  2 19:47:15 2013
@@ -59,8 +59,9 @@ public class AgentRun extends BaseAgents
       
       Logging.root.info("Running...");
       // Register hook first so stopAgents() not required
-      ManifoldCF.registerAgentsShutdownHook(tc, processID);
-      ManifoldCF.runAgents(tc, processID);
+      AgentsDaemon ad = new AgentsDaemon(processID);
+      ad.registerAgentsShutdownHook(tc);
+      ad.runAgents(tc);
       Logging.root.info("Shutting down...");
     }
     catch (ManifoldCFException e)

Modified: manifoldcf/trunk/framework/agents/src/main/java/org/apache/manifoldcf/agents/AgentStop.java
URL: http://svn.apache.org/viewvc/manifoldcf/trunk/framework/agents/src/main/java/org/apache/manifoldcf/agents/AgentStop.java?rev=1547167&r1=1547166&r2=1547167&view=diff
==============================================================================
--- manifoldcf/trunk/framework/agents/src/main/java/org/apache/manifoldcf/agents/AgentStop.java (original)
+++ manifoldcf/trunk/framework/agents/src/main/java/org/apache/manifoldcf/agents/AgentStop.java Mon Dec  2 19:47:15 2013
@@ -37,7 +37,7 @@ public class AgentStop extends BaseAgent
     // As part of the work for CONNECTORS-781, this method is now synchronous.
     // We assert the shutdown signal, and then wait until all active services have shut down.
     ILockManager lockManager = LockManagerFactory.make(tc);
-    ManifoldCF.assertAgentsShutdownSignal(tc);
+    AgentsDaemon.assertAgentsShutdownSignal(tc);
     try
     {
       Logging.root.info("Shutdown signal sent");
@@ -61,7 +61,7 @@ public class AgentStop extends BaseAgent
     finally
     {
       // Clear shutdown signal
-      ManifoldCF.clearAgentsShutdownSignal(tc);
+      AgentsDaemon.clearAgentsShutdownSignal(tc);
     }
   }
 

Modified: manifoldcf/trunk/framework/agents/src/main/java/org/apache/manifoldcf/agents/system/ManifoldCF.java
URL: http://svn.apache.org/viewvc/manifoldcf/trunk/framework/agents/src/main/java/org/apache/manifoldcf/agents/system/ManifoldCF.java?rev=1547167&r1=1547166&r2=1547167&view=diff
==============================================================================
--- manifoldcf/trunk/framework/agents/src/main/java/org/apache/manifoldcf/agents/system/ManifoldCF.java (original)
+++ manifoldcf/trunk/framework/agents/src/main/java/org/apache/manifoldcf/agents/system/ManifoldCF.java Mon Dec  2 19:47:15 2013
@@ -32,11 +32,6 @@ public class ManifoldCF extends org.apac
   // Agents initialized flag
   protected static boolean agentsInitialized = false;
   
-  /** This is the place we keep track of the agents we've started. */
-  protected static Map<String,IAgent> runningHash = new HashMap<String,IAgent>();
-  /** This flag prevents startAgents() from starting anything once stopAgents() has been called. */
-  protected static boolean stopAgentsRun = false;
-  
   /** Initialize environment.
   */
   public static void initializeEnvironment(IThreadContext threadContext)
@@ -88,10 +83,6 @@ public class ManifoldCF extends org.apac
     synchronized (initializeFlagLock)
     {
       org.apache.manifoldcf.core.system.ManifoldCF.resetEnvironment(threadContext);
-      synchronized (runningHash)
-      {
-        stopAgentsRun = false;
-      }
     }
   }
 
@@ -128,317 +119,6 @@ public class ManifoldCF extends org.apac
     mgr.deinstall();
   }
 
-  // There are a number of different ways of running the agents framework.
-  // (1) Repeatedly call checkAgents(), and when all done make sure to call stopAgents().
-  // (2) Call registerAgentsShutdownHook(), then repeatedly run checkAgents(),  Agent shutdown happens on JVM exit.
-  // (3) Call runAgents(), which will wait for someone else to call assertAgentsShutdownSignal().  Before exit, stopAgents() must be called.
-  // (4) Call registerAgentsShutdownHook(), then call runAgents(), which will wait for someone else to call assertAgentsShutdownSignal().  Shutdown happens on JVM exit.
-  
-  /** Assert shutdown signal.
-  */
-  public static void assertAgentsShutdownSignal(IThreadContext threadContext)
-    throws ManifoldCFException
-  {
-    ILockManager lockManager = LockManagerFactory.make(threadContext);
-    lockManager.setGlobalFlag(agentShutdownSignal);
-  }
-  
-  /** Clear shutdown signal.
-  */
-  public static void clearAgentsShutdownSignal(IThreadContext threadContext)
-    throws ManifoldCFException
-  {
-    ILockManager lockManager = LockManagerFactory.make(threadContext);
-    lockManager.clearGlobalFlag(agentShutdownSignal);
-  }
-
-
-  /** Register agents shutdown hook.
-  * Call this ONCE before calling startAgents or checkAgents the first time, if you want automatic cleanup of agents on JVM stop.
-  */
-  public static void registerAgentsShutdownHook(IThreadContext threadContext, String processID)
-    throws ManifoldCFException
-  {
-    // Create the shutdown hook for agents.  All activity will be keyed off of runningHash, so it is safe to do this under all conditions.
-    org.apache.manifoldcf.core.system.ManifoldCF.addShutdownHook(new AgentsShutdownHook(processID));
-  }
-  
-  /** Agent service name prefix (followed by agent class name) */
-  public static final String agentServicePrefix = "AGENT_";
-  
-  protected static AgentsThread agentsThread = null;
-
-  /** Run agents process.
-  * This method will not return until a shutdown signal is sent.
-  */
-  public static void runAgents(IThreadContext threadContext, String processID)
-    throws ManifoldCFException
-  {
-    ILockManager lockManager = LockManagerFactory.make(threadContext);
-
-    // Don't come up at all if shutdown signal in force
-    if (lockManager.checkGlobalFlag(agentShutdownSignal))
-      return;
-
-    // Create and start agents thread.
-    startAgents(threadContext, processID);
-    
-    while (true)
-    {
-      // Any shutdown signal yet?
-      if (lockManager.checkGlobalFlag(agentShutdownSignal))
-        break;
-          
-      try
-      {
-        ManifoldCF.sleep(5000L);
-      }
-      catch (InterruptedException e)
-      {
-        break;
-      }
-    }
-    
-  }
-
-  /** Start agents thread.
-  */
-  public static void startAgents(IThreadContext threadContext, String processID)
-    throws ManifoldCFException
-  {
-    // Create and start agents thread.
-    agentsThread = new AgentsThread(processID);
-    agentsThread.start();
-  }
-  
-  /** Stop all started agents.
-  */
-  public static void stopAgents(IThreadContext threadContext, String processID)
-    throws ManifoldCFException
-  {
-    // Shut down agents background thread.
-    while (agentsThread != null)
-    {
-      agentsThread.interrupt();
-      if (!agentsThread.isAlive())
-        agentsThread = null;
-    }
-    
-    // Shut down running agents services directly.
-    ILockManager lockManager = LockManagerFactory.make(threadContext);
-    synchronized (runningHash)
-    {
-      // This is supposedly safe; iterator remove is used
-      Iterator<String> iter = runningHash.keySet().iterator();
-      while (iter.hasNext())
-      {
-        String className = iter.next();
-        IAgent agent = runningHash.get(className);
-        // Stop it
-        agent.stopAgent(threadContext);
-        lockManager.endServiceActivity(getAgentsClassServiceType(className), processID);
-        iter.remove();
-        agent.cleanUp(threadContext);
-      }
-    }
-    // Done.
-  }
-
-  protected static String getAgentsClassServiceType(String agentClassName)
-  {
-    return agentServicePrefix + agentClassName;
-  }
-  
-  /** Agents thread.  This runs in background until interrupted, at which point
-  * it shuts down.  Its responsibilities include cleaning up after dead processes,
-  * as well as starting newly-registered agent processes, and terminating ones that disappear.
-  */
-  protected static class AgentsThread extends Thread
-  {
-    protected final String processID;
-    
-    public AgentsThread(String processID)
-    {
-      super();
-      this.processID = processID;
-      setName("Agents thread");
-      setDaemon(true);
-    }
-    
-    public void run()
-    {
-      try
-      {
-        IThreadContext threadContext = ThreadContextFactory.make();
-        while (true)
-        {
-          try
-          {
-            if (Thread.currentThread().isInterrupted())
-              throw new ManifoldCFException("Interrupted",ManifoldCFException.INTERRUPTED);
-
-            checkAgents(threadContext, processID);
-            ManifoldCF.sleep(5000L);
-          }
-          catch (InterruptedException e)
-          {
-            break;
-          }
-          catch (ManifoldCFException e)
-          {
-            if (e.getErrorCode() == ManifoldCFException.INTERRUPTED)
-              break;
-            Logging.agents.error("Exception tossed: "+e.getMessage(),e);
-          }
-          catch (OutOfMemoryError e)
-          {
-            System.err.println("Agents process ran out of memory - shutting down");
-            e.printStackTrace(System.err);
-            System.exit(-200);
-          }
-          catch (Throwable e)
-          {
-            Logging.agents.fatal("Error tossed: "+e.getMessage(),e);
-          }
-        }
-      }
-      catch (Throwable e)
-      {
-        // Severe error on initialization
-        System.err.println("Agents process could not start - shutting down");
-        Logging.agents.fatal("AgentThread initialization error tossed: "+e.getMessage(),e);
-        System.exit(-300);
-      }
-    }
-  }
-
-  /** Start all not-running agents.
-  *@param threadContext is the thread context.
-  */
-  protected static void checkAgents(IThreadContext threadContext, String processID)
-    throws ManifoldCFException
-  {
-    ILockManager lockManager = LockManagerFactory.make(threadContext);
-    // Get agent manager
-    IAgentManager manager = AgentManagerFactory.make(threadContext);
-    synchronized (runningHash)
-    {
-      String[] classes = manager.getAllAgents();
-      Set<String> currentAgentClasses = new HashSet<String>();
-
-      int i = 0;
-      while (i < classes.length)
-      {
-        String className = classes[i++];
-        if (runningHash.get(className) == null)
-        {
-          // Start this agent
-          IAgent agent = AgentFactory.make(className);
-          agent.initialize(threadContext);
-          try
-          {
-            // Throw a lock, so that cleanup processes and startup processes don't collide.
-            String serviceType = getAgentsClassServiceType(className);
-            lockManager.registerServiceBeginServiceActivity(serviceType, processID, new CleanupAgent(threadContext, agent, processID));
-            // There is a potential race condition where the agent has been started but hasn't yet appeared in runningHash.
-            // But having runningHash be the synchronizer for this activity will prevent any problems.
-            agent.startAgent(threadContext, processID);
-            // Successful!
-            runningHash.put(className,agent);
-          }
-          catch (ManifoldCFException e)
-          {
-            if (e.getErrorCode() != ManifoldCFException.INTERRUPTED)
-              agent.cleanUp(threadContext);
-            throw e;
-          }
-        }
-        currentAgentClasses.add(className);
-      }
-
-      // Go through running hash and look for agents processes that have left
-      Iterator<String> runningAgentsIterator = runningHash.keySet().iterator();
-      while (runningAgentsIterator.hasNext())
-      {
-        String runningAgentClass = runningAgentsIterator.next();
-        if (!currentAgentClasses.contains(runningAgentClass))
-        {
-          // Shut down this one agent.
-          IAgent agent = runningHash.get(runningAgentClass);
-          // Stop it
-          agent.stopAgent(threadContext);
-          lockManager.endServiceActivity(getAgentsClassServiceType(runningAgentClass), processID);
-          runningAgentsIterator.remove();
-          agent.cleanUp(threadContext);
-        }
-      }
-    }
-
-    synchronized (runningHash)
-    {
-      // For every class we're supposed to be running, find registered but no-longer-active instances and clean
-      // up after them.
-      for (String agentsClass : runningHash.keySet())
-      {
-        IAgent agent = runningHash.get(agentsClass);
-        IServiceCleanup cleanup = new CleanupAgent(threadContext, agent, processID);
-        String agentsClassServiceType = getAgentsClassServiceType(agentsClass);
-        while (!lockManager.cleanupInactiveService(agentsClassServiceType, cleanup))
-        {
-          // Loop until no more inactive services
-        }
-      }
-    }
-    
-  }
-
-  
-  protected static class CleanupAgent implements IServiceCleanup
-  {
-    protected final IAgent agent;
-    protected final IThreadContext threadContext;
-    protected final String processID;
-
-    public CleanupAgent(IThreadContext threadContext, IAgent agent, String processID)
-    {
-      this.agent = agent;
-      this.threadContext = threadContext;
-      this.processID = processID;
-    }
-    
-    /** Clean up after the specified service.  This method will block any startup of the specified
-    * service for as long as it runs.
-    *@param serviceName is the name of the service.
-    */
-    @Override
-    public void cleanUpService(String serviceName)
-      throws ManifoldCFException
-    {
-      agent.cleanUpAgentData(threadContext, processID, serviceName);
-    }
-
-    /** Clean up after ALL services of the type on the cluster.
-    */
-    @Override
-    public void cleanUpAllServices()
-      throws ManifoldCFException
-    {
-      agent.cleanUpAllAgentData(threadContext, processID);
-    }
-    
-    /** Perform cluster initialization - that is, whatever is needed presuming that the
-    * cluster has been down for an indeterminate period of time, but is otherwise in a clean
-    * state.
-    */
-    @Override
-    public void clusterInit()
-      throws ManifoldCFException
-    {
-      agent.clusterInit(threadContext);
-    }
-
-  }
-  
   /** Signal output connection needs redoing.
   * This is called when something external changed on an output connection, and
   * therefore all associated documents must be reindexed.
@@ -456,30 +136,6 @@ public class ManifoldCF extends org.apac
     AgentManagerFactory.noteOutputConnectionChange(threadContext,connectionName);
   }
   
-  /** Agents shutdown hook class */
-  protected static class AgentsShutdownHook implements IShutdownHook
-  {
-    protected final String processID;
-
-    public AgentsShutdownHook(String processID)
-    {
-      this.processID = processID;
-    }
-    
-    public void doCleanup()
-      throws ManifoldCFException
-    {
-      // Shutting down in this way must prevent startup from taking place.
-      synchronized (runningHash)
-      {
-        stopAgentsRun = true;
-      }
-      IThreadContext tc = ThreadContextFactory.make();
-      stopAgents(tc,processID);
-    }
-    
-  }
-  
   // Helper methods for API support.  These are made public so connectors can use them to implement the executeCommand method.
   
   // These are the universal node types.

Modified: manifoldcf/trunk/framework/combined-service/src/main/java/org/apache/manifoldcf/combinedservice/ServletListener.java
URL: http://svn.apache.org/viewvc/manifoldcf/trunk/framework/combined-service/src/main/java/org/apache/manifoldcf/combinedservice/ServletListener.java?rev=1547167&r1=1547166&r2=1547167&view=diff
==============================================================================
--- manifoldcf/trunk/framework/combined-service/src/main/java/org/apache/manifoldcf/combinedservice/ServletListener.java (original)
+++ manifoldcf/trunk/framework/combined-service/src/main/java/org/apache/manifoldcf/combinedservice/ServletListener.java Mon Dec  2 19:47:15 2013
@@ -20,6 +20,7 @@ package org.apache.manifoldcf.combinedse
 
 import org.apache.manifoldcf.core.interfaces.*;
 import org.apache.manifoldcf.crawler.system.ManifoldCF;
+import org.apache.manifoldcf.agents.system.AgentsDaemon;
 import javax.servlet.*;
 
 /** This class furnishes a servlet shutdown hook for ManifoldCF.  It should be referenced in the
@@ -59,10 +60,10 @@ public class ServletListener implements 
     {
       if (agentsThread != null)
       {
-        ManifoldCF.assertAgentsShutdownSignal(tc);
+        AgentsDaemon.assertAgentsShutdownSignal(tc);
         agentsThread.finishUp();
         agentsThread = null;
-        ManifoldCF.clearAgentsShutdownSignal(tc);
+        AgentsDaemon.clearAgentsShutdownSignal(tc);
       }
     }
     catch (InterruptedException e)
@@ -94,14 +95,15 @@ public class ServletListener implements 
       IThreadContext tc = ThreadContextFactory.make();
       try
       {
-        ManifoldCF.clearAgentsShutdownSignal(tc);
+        AgentsDaemon.clearAgentsShutdownSignal(tc);
+        AgentsDaemon ad = new AgentsDaemon(processID);
         try
         {
-          ManifoldCF.runAgents(tc, processID);
+          ad.runAgents(tc);
         }
         finally
         {
-          ManifoldCF.stopAgents(tc, processID);
+          ad.stopAgents(tc);
         }
       }
       catch (Throwable e)

Modified: manifoldcf/trunk/framework/jetty-runner/src/main/java/org/apache/manifoldcf/jettyrunner/ManifoldCFJettyRunner.java
URL: http://svn.apache.org/viewvc/manifoldcf/trunk/framework/jetty-runner/src/main/java/org/apache/manifoldcf/jettyrunner/ManifoldCFJettyRunner.java?rev=1547167&r1=1547166&r2=1547167&view=diff
==============================================================================
--- manifoldcf/trunk/framework/jetty-runner/src/main/java/org/apache/manifoldcf/jettyrunner/ManifoldCFJettyRunner.java (original)
+++ manifoldcf/trunk/framework/jetty-runner/src/main/java/org/apache/manifoldcf/jettyrunner/ManifoldCFJettyRunner.java Mon Dec  2 19:47:15 2013
@@ -21,6 +21,7 @@ import java.io.*;
 import org.apache.manifoldcf.core.interfaces.*;
 import org.apache.manifoldcf.crawler.system.*;
 import org.apache.manifoldcf.crawler.*;
+import org.apache.manifoldcf.agents.system.AgentsDaemon;
 
 import java.io.IOException;
 import java.io.InputStream;
@@ -138,8 +139,9 @@ public class ManifoldCFJettyRunner
   {
     String processID = ManifoldCF.getProcessID();
     // Do this so we don't have to call stopAgents() ourselves.
-    ManifoldCF.registerAgentsShutdownHook(tc, processID);
-    ManifoldCF.runAgents(tc, processID);
+    AgentsDaemon ad = new AgentsDaemon(processID);
+    ad.registerAgentsShutdownHook(tc);
+    ad.runAgents(tc);
   }
 
   /**
@@ -198,7 +200,7 @@ public class ManifoldCFJettyRunner
       if (useParentClassLoader)
       {
         // Clear the agents shutdown signal.
-        ManifoldCF.clearAgentsShutdownSignal(tc);
+        AgentsDaemon.clearAgentsShutdownSignal(tc);
         
         // Do the basic initialization of the database and its schema
         ManifoldCF.createSystemDatabase(tc);

Modified: manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/jobs/EventManager.java
URL: http://svn.apache.org/viewvc/manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/jobs/EventManager.java?rev=1547167&r1=1547166&r2=1547167&view=diff
==============================================================================
--- manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/jobs/EventManager.java (original)
+++ manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/jobs/EventManager.java Mon Dec  2 19:47:15 2013
@@ -127,7 +127,7 @@ public class EventManager extends org.ap
     ArrayList list = new ArrayList();
     String query = buildConjunctionClause(list,new ClauseDescription[]{
       new UnitaryClause(processIDField,processID)});
-    performDelete("WHERE "+query,null,null);
+    performDelete("WHERE "+query,list,null);
   }
 
   /** Clean up after all processIDs.

Modified: manifoldcf/trunk/framework/pull-agent/src/test/java/org/apache/manifoldcf/crawler/tests/ManifoldCFInstance.java
URL: http://svn.apache.org/viewvc/manifoldcf/trunk/framework/pull-agent/src/test/java/org/apache/manifoldcf/crawler/tests/ManifoldCFInstance.java?rev=1547167&r1=1547166&r2=1547167&view=diff
==============================================================================
--- manifoldcf/trunk/framework/pull-agent/src/test/java/org/apache/manifoldcf/crawler/tests/ManifoldCFInstance.java (original)
+++ manifoldcf/trunk/framework/pull-agent/src/test/java/org/apache/manifoldcf/crawler/tests/ManifoldCFInstance.java Mon Dec  2 19:47:15 2013
@@ -22,6 +22,7 @@ import org.apache.manifoldcf.core.interf
 import org.apache.manifoldcf.agents.interfaces.*;
 import org.apache.manifoldcf.crawler.interfaces.*;
 import org.apache.manifoldcf.crawler.system.ManifoldCF;
+import org.apache.manifoldcf.agents.system.AgentsDaemon;
 
 import java.io.*;
 import java.util.*;
@@ -56,39 +57,57 @@ import org.apache.http.entity.ContentTyp
 /** Tests that run the "agents daemon" should be derived from this */
 public class ManifoldCFInstance
 {
-  protected boolean webapps = true;
-  protected boolean singleWar = false;
-  protected int testPort = 8346;
+  protected final boolean webapps;
+  protected final boolean singleWar;
+  protected final int testPort;
+  protected final String processID;
   
   protected DaemonThread daemonThread = null;
   protected Server server = null;
 
   public ManifoldCFInstance()
   {
+    this(ManifoldCF.getProcessID(), 8346, false, true);
   }
   
+  public ManifoldCFInstance(String processID)
+  {
+    this(processID, 8346, false, true);
+  }
+
   public ManifoldCFInstance(boolean singleWar)
   {
-    this(8346,singleWar,true);
+    this(ManifoldCF.getProcessID(), 8346, singleWar, true);
+  }
+  
+  public ManifoldCFInstance(String processID, boolean singleWar)
+  {
+    this(processID, 8346, singleWar, true);
   }
 
   public ManifoldCFInstance(boolean singleWar, boolean webapps)
   {
-    this(8346,singleWar,webapps);
+    this(ManifoldCF.getProcessID(), 8346, singleWar, webapps);
+  }
+
+  public ManifoldCFInstance(String processID, boolean singleWar, boolean webapps)
+  {
+    this(processID, 8346, singleWar, webapps);
   }
   
-  public ManifoldCFInstance(int testPort)
+  public ManifoldCFInstance(String processID, int testPort)
   {
-    this(testPort,false,true);
+    this(processID, testPort, false, true);
   }
 
-  public ManifoldCFInstance(int testPort, boolean singleWar)
+  public ManifoldCFInstance(String processID, int testPort, boolean singleWar)
   {
-    this(testPort,singleWar,true);
+    this(processID, testPort, singleWar, true);
   }
 
-  public ManifoldCFInstance(int testPort, boolean singleWar, boolean webapps)
+  public ManifoldCFInstance(String processID, int testPort, boolean singleWar, boolean webapps)
   {
+    this.processID = processID;
     this.webapps = webapps;
     this.testPort = testPort;
     this.singleWar = singleWar;
@@ -575,9 +594,9 @@ public class ManifoldCFInstance
       // If all worked, then we can start the daemon.
       // Clear the agents shutdown signal.
       IThreadContext tc = ThreadContextFactory.make();
-      ManifoldCF.clearAgentsShutdownSignal(tc);
+      AgentsDaemon.clearAgentsShutdownSignal(tc);
 
-      daemonThread = new DaemonThread();
+      daemonThread = new DaemonThread(processID);
       daemonThread.start();
     }
   }
@@ -667,30 +686,48 @@ public class ManifoldCFInstance
         }
       }
 
-      if (!singleWar)
+      try
+      {
+        stopNoCleanup();
+      }
+      catch (Exception e)
       {
-        // Shut down daemon
-        ManifoldCF.assertAgentsShutdownSignal(tc);
+        if (currentException == null)
+          currentException = e;
+      }
+    }
+  }
+  
+  public void stopNoCleanup()
+    throws Exception
+  {
+    if (daemonThread != null)
+    {
+      Exception currentException = null;
+      
+      // Shut down daemon - but only ONE daemon
+      //AgentsDaemon.assertAgentsShutdownSignal(tc);
         
-        // Wait for daemon thread to exit.
-        while (true)
+      // Wait for daemon thread to exit.
+      while (true)
+      {
+        daemonThread.interrupt();
+        if (daemonThread.isAlive())
         {
-          if (daemonThread.isAlive())
-          {
-            Thread.sleep(1000L);
-            continue;
-          }
-          break;
+          Thread.sleep(1000L);
+          continue;
         }
-
-        Exception e = daemonThread.getDaemonException();
-        if (e != null)
-          currentException = e;
+        break;
       }
-        
+
+      Exception e = daemonThread.getDaemonException();
+      if (e != null || !(e instanceof InterruptedException))
+        currentException = e;
+
       if (currentException != null)
         throw currentException;
     }
+        
   }
   
   public void unload()
@@ -708,22 +745,24 @@ public class ManifoldCFInstance
   
   protected static class DaemonThread extends Thread
   {
+    protected final String processID;
     protected Exception daemonException = null;
     
-    public DaemonThread()
+    public DaemonThread(String processID)
     {
+      this.processID = processID;
       setName("Daemon thread");
     }
     
     public void run()
     {
-      String processID = ManifoldCF.getProcessID();
       IThreadContext tc = ThreadContextFactory.make();
       // Now, start the server, and then wait for the shutdown signal.  On shutdown, we have to actually do the cleanup,
       // because the JVM isn't going away.
+      AgentsDaemon ad = new AgentsDaemon(processID);
       try
       {
-        ManifoldCF.runAgents(tc, processID);
+        ad.runAgents(tc);
       }
       catch (ManifoldCFException e)
       {
@@ -733,7 +772,7 @@ public class ManifoldCFInstance
       {
         try
         {
-          ManifoldCF.stopAgents(tc, processID);
+          ad.stopAgents(tc);
         }
         catch (ManifoldCFException e)
         {

Modified: manifoldcf/trunk/framework/pull-agent/src/test/java/org/apache/manifoldcf/crawler/tests/SchedulerHSQLDBTest.java
URL: http://svn.apache.org/viewvc/manifoldcf/trunk/framework/pull-agent/src/test/java/org/apache/manifoldcf/crawler/tests/SchedulerHSQLDBTest.java?rev=1547167&r1=1547166&r2=1547167&view=diff
==============================================================================
--- manifoldcf/trunk/framework/pull-agent/src/test/java/org/apache/manifoldcf/crawler/tests/SchedulerHSQLDBTest.java (original)
+++ manifoldcf/trunk/framework/pull-agent/src/test/java/org/apache/manifoldcf/crawler/tests/SchedulerHSQLDBTest.java Mon Dec  2 19:47:15 2013
@@ -28,15 +28,18 @@ import org.junit.*;
 
 /** This is a test of the scheduler.  If the test succeeds, it is because
 * the scheduler has properly distributed requests for all bins evenly. */
-public class SchedulerHSQLDBTest extends BaseITHSQLDB
+public class SchedulerHSQLDBTest extends ConnectorBaseHSQLDB
 {
-  
+  protected final ManifoldCFInstance mcfInstance1;
+  protected final ManifoldCFInstance mcfInstance2;
   protected SchedulerTester tester;
 
   public SchedulerHSQLDBTest()
   {
-    super(false,false);
-    tester = new SchedulerTester(mcfInstance);
+    super();
+    mcfInstance1 = new ManifoldCFInstance("A",false,false);
+    mcfInstance2 = new ManifoldCFInstance("B",false,false);
+    tester = new SchedulerTester(mcfInstance1,mcfInstance2);
   }
   
   @Override
@@ -70,5 +73,68 @@ public class SchedulerHSQLDBTest extends
     tester.executeTest();
   }
   
+  @Before
+  public void setUp()
+    throws Exception
+  {
+    initializeSystem();
+    try
+    {
+      localReset();
+    }
+    catch (Exception e)
+    {
+      System.out.println("Warning: Preclean failed: "+e.getMessage());
+    }
+    try
+    {
+      localSetUp();
+    }
+    catch (Exception e)
+    {
+      e.printStackTrace();
+      throw e;
+    }
+  }
+  
+  @After
+  public void cleanUp()
+    throws Exception
+  {
+    Exception currentException = null;
+    // Last, shut down the web applications.
+    // If this is done too soon it closes the database before the rest of the cleanup happens.
+    try
+    {
+      mcfInstance1.unload();
+    }
+    catch (Exception e)
+    {
+      if (currentException == null)
+        currentException = e;
+    }
+    try
+    {
+      mcfInstance2.unload();
+    }
+    catch (Exception e)
+    {
+      if (currentException == null)
+        currentException = e;
+    }
+    try
+    {
+      localCleanUp();
+    }
+    catch (Exception e)
+    {
+      e.printStackTrace();
+      throw e;
+    }
+    if (currentException != null)
+      throw currentException;
+    cleanupSystem();
+  }
+  
 
 }

Modified: manifoldcf/trunk/framework/pull-agent/src/test/java/org/apache/manifoldcf/crawler/tests/SchedulerTester.java
URL: http://svn.apache.org/viewvc/manifoldcf/trunk/framework/pull-agent/src/test/java/org/apache/manifoldcf/crawler/tests/SchedulerTester.java?rev=1547167&r1=1547166&r2=1547167&view=diff
==============================================================================
--- manifoldcf/trunk/framework/pull-agent/src/test/java/org/apache/manifoldcf/crawler/tests/SchedulerTester.java (original)
+++ manifoldcf/trunk/framework/pull-agent/src/test/java/org/apache/manifoldcf/crawler/tests/SchedulerTester.java Mon Dec  2 19:47:15 2013
@@ -29,16 +29,20 @@ import java.util.*;
 /** This is a very basic sanity check */
 public class SchedulerTester
 {
-  protected ManifoldCFInstance instance;
+  protected final ManifoldCFInstance instance1;
+  protected final ManifoldCFInstance instance2;
   
-  public SchedulerTester(ManifoldCFInstance instance)
+  public SchedulerTester(ManifoldCFInstance instance1, ManifoldCFInstance instance2)
   {
-    this.instance = instance;
+    this.instance1 = instance1;
+    this.instance2 = instance2;
   }
   
   public void executeTest()
     throws Exception
   {
+    instance1.start();
+    
     // Hey, we were able to install the file system connector etc.
     // Now, create a local test job and run it.
     IThreadContext tc = ThreadContextFactory.make();
@@ -78,20 +82,31 @@ public class SchedulerTester
 
     // Now, start the job, and wait until it is running.
     jobManager.manualStart(job.getID());
-    instance.waitJobRunningNative(jobManager,job.getID(),30000L);
+    instance1.waitJobRunningNative(jobManager,job.getID(),30000L);
+    
+    // Start the second instance.
+    instance2.start();
+    // Wait long enough for the stuffing etc to take place once
+    Thread.sleep(5000L);
+    // Terminate instance1.  Instance2 should keep going.
+    instance1.stopNoCleanup();
     
     // Wait for the job to become inactive.  The time should be at least long enough to handle
     // 100 documents per bin, but not significantly greater than that.  Let's say 120 seconds.
     long startTime = System.currentTimeMillis();
-    instance.waitJobInactiveNative(jobManager,job.getID(),150000L);
+    instance2.waitJobInactiveNative(jobManager,job.getID(),150000L);
     long endTime = System.currentTimeMillis();
     if (jobManager.getStatus(job.getID()).getDocumentsProcessed() != 10+10*200)
       throw new Exception("Expected 2010 documents, saw "+jobManager.getStatus(job.getID()).getDocumentsProcessed());
     if (endTime-startTime < 96000L)
       throw new Exception("Job finished too quickly; throttling clearly failed");
     System.out.println("Crawl took "+(endTime-startTime)+" milliseconds");
+    
     // Now, delete the job.
     jobManager.deleteJob(job.getID());
-    instance.waitJobDeletedNative(jobManager,job.getID(),120000L);
+    instance2.waitJobDeletedNative(jobManager,job.getID(),120000L);
+
+    // Shut down instance2
+    instance2.stop();
   }
 }