Move Adaptor.init() calling to Application

This allows using GsaCommunicationHandler as a library.
diff --git a/src/com/google/enterprise/adaptor/Application.java b/src/com/google/enterprise/adaptor/Application.java
index e390776..88665ea 100644
--- a/src/com/google/enterprise/adaptor/Application.java
+++ b/src/com/google/enterprise/adaptor/Application.java
@@ -25,7 +25,7 @@
 import java.io.IOException;
 import java.net.*;
 import java.security.NoSuchAlgorithmException;
-import java.util.Arrays;
+import java.util.*;
 import java.util.concurrent.*;
 import java.util.logging.Level;
 import java.util.logging.Logger;
@@ -48,6 +48,14 @@
 
   private final Config config;
   private final GsaCommunicationHandler gsa;
+  private final ConfigModificationListener configModListener
+      = new ConfigModListener();
+  /**
+   * An "inverted" semaphore that has permits available when stop() is running;
+   * at all other times it has no permits. This allows start() to sleep on the
+   * semaphore and be woken when stop() is called.
+   */
+  private final Semaphore shutdownSemaphore = new Semaphore(0);
   private Thread shutdownHook;
   private HttpServer primaryServer;
   private HttpServer dashboardServer;
@@ -64,7 +72,7 @@
    * manual shutdown. A shutdown hook is automatically installed that calls
    * {@code stop()}.
    */
-  public void start() throws IOException, InterruptedException {
+  public synchronized void start() throws IOException, InterruptedException {
     synchronized (this) {
       daemonInit();
 
@@ -99,24 +107,62 @@
   /**
    * Really start. This must be called after a successful {@link #daemonInit}.
    */
-  void daemonStart() throws IOException, InterruptedException {
-    gsa.start(primaryServer, dashboardServer);
+  synchronized void daemonStart() throws IOException, InterruptedException {
+    AdaptorContext context = gsa.setup(primaryServer, dashboardServer, null);
+
+    long sleepDurationMillis = 8000;
+    // An hour.
+    long maxSleepDurationMillis = 60 * 60 * 1000;
+    // Loop until 1) the adaptor starts successfully, 2) stop() is called, or
+    // 3) Thread.interrupt() is called on this thread (which we don't do).
+    // Retrying to start the adaptor is helpful in cases where it needs
+    // initialization data from a repository that is temporarily down; if the
+    // adaptor is running as a service, we don't want to stop starting simply
+    // because another computer is down while we start (which would easily be
+    // the case after a power failure).
+    while (true) {
+      try {
+        gsa.tryToPutVersionIntoConfig();
+        String adaptorType = gsa.getAdaptor().getClass().getName();
+        log.log(Level.INFO, "about to init {0}", adaptorType); 
+        gsa.getAdaptor().init(context);
+        break;
+      } catch (InterruptedException ex) {
+        throw ex;
+      } catch (Exception ex) {
+        log.log(Level.WARNING, "Failed to initialize adaptor", ex);
+        if (shutdownSemaphore.tryAcquire(sleepDurationMillis,
+              TimeUnit.MILLISECONDS)) {
+          shutdownSemaphore.release();
+          // Shutdown initiated.
+          return;
+        }
+        sleepDurationMillis
+            = Math.min(sleepDurationMillis * 2, maxSleepDurationMillis);
+        gsa.ensureLatestConfigLoaded();
+      }
+    }
+
+    config.addConfigModificationListener(configModListener);
+    gsa.start();
   }
 
   /**
    * Stop processing incoming requests and background tasks, allowing graceful
    * shutdown.
    */
-  public synchronized void stop(long time, TimeUnit unit) {
+  public void stop(long time, TimeUnit unit) {
     daemonStop(time, unit);
     daemonDestroy(time, unit);
-    if (shutdownHook != null) {
-      try {
-        Runtime.getRuntime().removeShutdownHook(shutdownHook);
-      } catch (IllegalStateException ex) {
-        // Already executing hook.
+    synchronized (this) {
+      if (shutdownHook != null) {
+        try {
+          Runtime.getRuntime().removeShutdownHook(shutdownHook);
+        } catch (IllegalStateException ex) {
+          // Already executing hook.
+        }
+        shutdownHook = null;
       }
-      shutdownHook = null;
     }
   }
 
@@ -124,11 +170,35 @@
    * Stop all the services we provide. This is the opposite of {@link
    * #daemonStart}.
    */
-  synchronized void daemonStop(long time, TimeUnit unit) {
-    if (primaryServer == null) {
-      throw new IllegalStateException("Already stopped");
+  void daemonStop(long time, TimeUnit unit) {
+    shutdownSemaphore.release();
+    try {
+      synchronized (this) {
+        if (primaryServer == null) {
+          throw new IllegalStateException("Already stopped");
+        }
+        config.removeConfigModificationListener(configModListener);
+        gsa.stop(time, unit);
+        try {
+          gsa.getAdaptor().destroy();
+        } finally {
+          gsa.teardown();
+        }
+      }
+    } finally {
+      boolean interrupted = false;
+      while (true) {
+        try {
+          shutdownSemaphore.acquire();
+          break;
+        } catch (InterruptedException ex) {
+          interrupted = true;
+        }
+      }
+      if (interrupted) {
+        Thread.currentThread().interrupt();
+      }
     }
-    gsa.stop(time, unit);
   }
 
   /**
@@ -408,4 +478,35 @@
       }
     }
   }
+
+  private class ConfigModListener implements ConfigModificationListener {
+    @Override
+    public void configModified(ConfigModificationEvent ev) {
+      Set<String> modifiedKeys = ev.getModifiedKeys();
+      if (modifiedKeys.contains("adaptor.fullListingSchedule")) {
+        gsa.rescheduleFullListing(
+            ev.getNewConfig().getAdaptorFullListingSchedule());
+      }
+
+      // List of "safe" keys that can be updated without a restart.
+      List<String> safeKeys = Arrays.asList("adaptor.fullListingSchedule");
+      // Set of "unsafe" keys that have been modified.
+      Set<String> modifiedKeysRequiringRestart
+          = new HashSet<String>(modifiedKeys);
+      modifiedKeysRequiringRestart.removeAll(safeKeys);
+      // If there are modified "unsafe" keys, then we restart things to make
+      // sure all the code is up-to-date with the new values.
+      if (!modifiedKeysRequiringRestart.isEmpty()) {
+        log.warning("Unsafe configuration keys modified. To ensure a sane "
+                    + "state, the adaptor is restarting.");
+        daemonStop(3, TimeUnit.SECONDS);
+        try {
+          daemonStart();
+        } catch (Exception ex) {
+          log.log(Level.SEVERE, "Automatic restart failed", ex);
+          throw new RuntimeException(ex);
+        }
+      }
+    }
+  }
 }
diff --git a/src/com/google/enterprise/adaptor/GsaCommunicationHandler.java b/src/com/google/enterprise/adaptor/GsaCommunicationHandler.java
index 2bc3425..fabb141 100644
--- a/src/com/google/enterprise/adaptor/GsaCommunicationHandler.java
+++ b/src/com/google/enterprise/adaptor/GsaCommunicationHandler.java
@@ -46,8 +46,6 @@
   private final Adaptor adaptor;
   private final Config config;
   private final Journal journal;
-  private final GsaConfigModListener gsaConfigModListener
-      = new GsaConfigModListener();
   private AdaptorContextImpl adaptorContext;
   /**
    * Cron-style scheduler. Available for other uses, but necessary for
@@ -95,12 +93,8 @@
   private HttpServerScope dashboardScope;
   private Dashboard dashboard;
   private SensitiveValueCodec secureValueCodec;
-  /**
-   * Used to stop startup prematurely. This allows cancelling an already-running
-   * start(). If start fails, a stale shuttingDownLatch can remain, thus it does
-   * not provide any information as to whether a start() call is running.
-   */
-  private volatile CountDownLatch shuttingDownLatch;
+  private KeyPair keyPair;
+  private AclTransform aclTransform;
   /**
    * Used to stop startup prematurely. When greater than 0, start() should abort
    * immediately because stop() is currently processing. This allows cancelling
@@ -122,19 +116,12 @@
   }
 
   /**
-   * Starts listening for communications from GSA. {@code contextPrefix}
+   * Start services necessary for handling outgoing requests. {@code ""} is used
+   * for {@code contextPrefix} if the passed value is {@code null}.
    */
-  public synchronized void start(HttpServer server, HttpServer dashboardServer)
-      throws IOException, InterruptedException {
-    start(server, dashboardServer, null);
-  }
-
-  /**
-   * Starts listening for communications from GSA. {@code ""} is used for
-   * {@code contextPrefix} if the passed value is {@code null}.
-   */
-  public synchronized void start(HttpServer server, HttpServer dashboardServer,
-      String contextPrefix) throws IOException, InterruptedException {
+  public synchronized AdaptorContext setup(HttpServer server,
+      HttpServer dashboardServer, String contextPrefix) throws IOException,
+      InterruptedException {
     if (this.scope != null) {
       throw new IllegalStateException("Already listening");
     }
@@ -149,17 +136,12 @@
       throw new IllegalArgumentException(
           "Both servers must be HttpServers or both HttpsServers");
     }
-    shuttingDownLatch = new CountDownLatch(1);
-    if (shutdownCount.get() > 0) {
-      shuttingDownLatch = null;
-      return;
-    }
 
     boolean secure = server instanceof HttpsServer;
     if (secure != config.isServerSecure()) {
       config.setValue("server.secure", "" + secure);
     }
-    KeyPair keyPair = null;
+    keyPair = null;
     try {
       keyPair = getKeyPair(config.getServerKeyAlias());
     } catch (IOException ex) {
@@ -192,8 +174,6 @@
           30 * 60 * 1000 /* session lifetime: 30 minutes */,
           5 * 60 * 1000 /* max cleanup frequency: 5 minutes */);
 
-    config.addConfigModificationListener(gsaConfigModListener);
-
     URI baseUri = config.getServerBaseUri();
     URI docUri;
     try {
@@ -206,52 +186,29 @@
     GsaFeedFileSender fileSender = new GsaFeedFileSender(
         config.getGsaHostname(), config.isServerSecure(), // use secure bool?
         config.getGsaCharacterEncoding());
-    AclTransform aclTransform = createAclTransform();
+    aclTransform = createAclTransform();
     GsaFeedFileMaker fileMaker = new GsaFeedFileMaker(docIdCodec, aclTransform,
         config.isGsa614FeedWorkaroundEnabled(),
         config.isGsa70AuthMethodWorkaroundEnabled());
     docIdSender
         = new DocIdSender(fileMaker, fileSender, journal, config, adaptor);
 
+    // Could be done during start(), but then we would have to save
+    // dashboardServer and contextPrefix.
+    dashboardScope = new HttpServerScope(dashboardServer, contextPrefix);
+
     // We are about to start the Adaptor, so anything available through
     // AdaptorContext or other means must be initialized at this point. Any
     // reference to 'adaptor' before this point must be done very carefully to
     // ensure it doesn't call the adaptor until after Adaptor.init() completes.
+    return adaptorContext = new AdaptorContextImpl();
+  }
 
-    long sleepDurationMillis = 8000;
-
-    // An hour.
-    long maxSleepDurationMillis = 60 * 60 * 1000;
-    // Loop until 1) the adaptor starts successfully, 2) stop() is called, or
-    // 3) Thread.interrupt() is called on this thread (which we don't do).
-    // Retrying to start the adaptor is helpful in cases where it needs
-    // initialization data from a repository that is temporarily down; if the
-    // adaptor is running as a service, we don't want to stop starting simply
-    // because another computer is down while we start (which would easily be
-    // the case after a power failure).
-    while (true) {
-      try {
-        tryToPutVersionIntoConfig(secure);
-        adaptorContext = new AdaptorContextImpl();
-        String adaptorType = adaptor.getClass().getName();
-        log.log(Level.INFO, "about to init {0}", adaptorType); 
-        adaptor.init(adaptorContext);
-        break;
-      } catch (InterruptedException ex) {
-        throw ex;
-      } catch (Exception ex) {
-        log.log(Level.WARNING, "Failed to initialize adaptor", ex);
-        if (shuttingDownLatch.await(sleepDurationMillis,
-              TimeUnit.MILLISECONDS)) {
-          // Shutdown initiated.
-          break;
-        }
-        sleepDurationMillis
-            = Math.min(sleepDurationMillis * 2, maxSleepDurationMillis);
-        ensureLatestConfigLoaded();
-      }
-    }
-
+  /**
+   * Start servicing incoming requests. This makes use of the
+   * previously-provided HttpServers and configuration.
+   */
+  public synchronized void start() {
     // Since the Adaptor has been started, we can now issue other calls to it.
     // Usages of 'adaptor' are completely safe after this point.
     adaptorContext.freeze();
@@ -263,10 +220,8 @@
           (ConfigModificationListener) adaptor);
     }*/
 
-    dashboardScope = new HttpServerScope(dashboardServer, contextPrefix);
-
     SamlServiceProvider samlServiceProvider = null;
-    if (secure) {
+    if (config.isServerSecure()) {
       bootstrapOpenSaml();
       SamlMetadata metadata = new SamlMetadata(config.getServerHostname(),
           config.getServerPort(), config.getGsaHostname(),
@@ -358,14 +313,13 @@
     dashboard = new Dashboard(config, this, journal, sessionManager,
         secureValueCodec, adaptor, adaptorContext.statusSources);
     dashboard.start(dashboardScope);
-
-    shuttingDownLatch = null;
   }
    
-  private void tryToPutVersionIntoConfig(boolean secure) throws IOException { 
+  void tryToPutVersionIntoConfig() throws IOException {
     try {
       if ("GENERATE".equals(config.getGsaVersion())) {  // is not set
-        GsaVersion ver = GsaVersion.get(config.getGsaHostname(), secure);
+        GsaVersion ver = GsaVersion.get(config.getGsaHostname(),
+            config.isServerSecure());
         config.overrideKey("gsa.version", "" + ver);
       }
     } catch (FileNotFoundException fne) {
@@ -596,38 +550,28 @@
   }
 
   /**
-   * Stop the current services, allowing up to {@code maxDelay} seconds for
-   * things to shutdown.
+   * Stops servicing incoming requests, allowing up to {@code maxDelay} seconds
+   * for things to shutdown. After called, no requests will be sent to the
+   * Adaptor.
+   *
+   * @return {@code true} if shutdown cleanly, {@code false} if requests may
+   *     still be processing
    */
-  public void stop(long time, TimeUnit unit) {
-    // Prevent new start()s.
-    shutdownCount.incrementAndGet();
-    try {
-      CountDownLatch latch = shuttingDownLatch;
-      if (latch != null) {
-        // Cause existing start() to begin cancelling.
-        latch.countDown();
-      }
-      realStop(time, unit);
-    } finally {
-      // Permit new start()s.
-      shutdownCount.decrementAndGet();
-    }
-  }
-
-  private synchronized void realStop(long time, TimeUnit unit) {
+  public synchronized boolean stop(long time, TimeUnit unit) {
+    boolean clean = true;
     if (adaptorContext != null) {
       adaptorContext.freeze();
     }
-    config.removeConfigModificationListener(gsaConfigModListener);
     if (scope != null) {
       scope.close();
-      scope = null;
+      scope = new HttpServerScope(
+          scope.getHttpServer(), scope.getContextPrefix());
     }
-    if (dashboardScope != null) {
+    if (scheduleExecutor != null) {
       // Post-Adaptor.init() resources need to be stopped.
       dashboardScope.close();
-      dashboardScope = null;
+      dashboardScope = new HttpServerScope(
+          dashboardScope.getHttpServer(), dashboardScope.getContextPrefix());
 
       scheduleExecutor.shutdownNow();
       scheduleExecutor = null;
@@ -648,24 +592,35 @@
       docIdIncrementalPusher = null;
 
       try {
-        waiter.shutdown(time, unit);
+        clean = clean & waiter.shutdown(time, unit);
       } catch (InterruptedException ex) {
         Thread.currentThread().interrupt();
+        clean = false;
       }
-      waiter = null;
+      waiter = new ShutdownWaiter();
     }
-    try {
-      adaptor.destroy();
-    } finally {
-      // Wait until after adaptor.destroy() to shutdown things accessible by
-      // AdaptorContext, so that the AdaptorContext is usable until the very
-      // end.
-      secureValueCodec = null;
-      sessionManager = null;
-      docIdCodec = null;
-      docIdSender = null;
-      adaptorContext = null;
-    }
+    return clean;
+  }
+
+  /**
+   * Stop services necessary for handling outgoing requests. This call
+   * invalidates the {@link AdaptorContext} returned from {@link #setup}.
+   */
+  public synchronized void teardown() {
+    scope = null;
+    dashboardScope = null;
+    keyPair = null;
+    aclTransform = null;
+    waiter = null;
+
+    // Wait until after adaptor.destroy() to shutdown things accessible by
+    // AdaptorContext, so that the AdaptorContext is usable until the very
+    // end.
+    secureValueCodec = null;
+    sessionManager = null;
+    docIdCodec = null;
+    docIdSender = null;
+    adaptorContext = null;
   }
 
   /**
@@ -721,6 +676,17 @@
     }
   }
 
+  synchronized void rescheduleFullListing(String schedule) {
+    if (sendDocIdsFuture == null) {
+      return;
+    }
+    try {
+      scheduler.reschedule(sendDocIdsFuture, schedule);
+    } catch (IllegalArgumentException ex) {
+      log.log(Level.WARNING, "Invalid schedule pattern", ex);
+    }
+  }
+
   /** The adaptor instance being used. */
   public Adaptor getAdaptor() {
     return adaptor;
@@ -810,47 +776,6 @@
     }
   }
 
-  private class GsaConfigModListener implements ConfigModificationListener {
-    @Override
-    public void configModified(ConfigModificationEvent ev) {
-      Set<String> modifiedKeys = ev.getModifiedKeys();
-      synchronized (GsaCommunicationHandler.this) {
-        if (modifiedKeys.contains("adaptor.fullListingSchedule")
-            && sendDocIdsFuture != null) {
-          String schedule = ev.getNewConfig().getAdaptorFullListingSchedule();
-          try {
-            scheduler.reschedule(sendDocIdsFuture, schedule);
-          } catch (IllegalArgumentException ex) {
-            log.log(Level.WARNING, "Invalid schedule pattern", ex);
-          }
-        }
-      }
-
-      // List of "safe" keys that can be updated without a restart.
-      List<String> safeKeys = Arrays.asList("adaptor.fullListingSchedule");
-      // Set of "unsafe" keys that have been modified.
-      Set<String> modifiedKeysRequiringRestart
-          = new HashSet<String>(modifiedKeys);
-      modifiedKeysRequiringRestart.removeAll(safeKeys);
-      // If there are modified "unsafe" keys, then we restart things to make
-      // sure all the code is up-to-date with the new values.
-      if (!modifiedKeysRequiringRestart.isEmpty()) {
-        log.warning("Unsafe configuration keys modified. To ensure a sane "
-                    + "state, the adaptor is restarting.");
-        HttpServer existingServer = scope.getHttpServer();
-        HttpServer existingDashboardServer
-            = dashboardScope.getHttpServer();
-        stop(3, TimeUnit.SECONDS);
-        try {
-          start(existingServer, existingDashboardServer);
-        } catch (Exception ex) {
-          log.log(Level.SEVERE, "Automatic restart failed", ex);
-          throw new RuntimeException(ex);
-        }
-      }
-    }
-  }
-
   /**
    * This class is thread-safe.
    */
diff --git a/src/com/google/enterprise/adaptor/HttpServerScope.java b/src/com/google/enterprise/adaptor/HttpServerScope.java
index f429331..72af660 100644
--- a/src/com/google/enterprise/adaptor/HttpServerScope.java
+++ b/src/com/google/enterprise/adaptor/HttpServerScope.java
@@ -65,4 +65,8 @@
   public HttpServer getHttpServer() {
     return server;
   }
+
+  public String getContextPrefix() {
+    return contextPrefix;
+  }
 }
diff --git a/test/com/google/enterprise/adaptor/ApplicationTest.java b/test/com/google/enterprise/adaptor/ApplicationTest.java
index f2e059a..3813d94 100644
--- a/test/com/google/enterprise/adaptor/ApplicationTest.java
+++ b/test/com/google/enterprise/adaptor/ApplicationTest.java
@@ -142,6 +142,27 @@
     conn.getContent();
   }
 
+  @Test
+  public void testFailOnceInitAdaptor() throws Exception {
+    class FailFirstAdaptor extends NullAdaptor {
+      private int count = 0;
+      public boolean started = false;
+
+      @Override
+      public void init(AdaptorContext context) {
+        if (count == 0) {
+          count++;
+          throw new RuntimeException();
+        }
+        started = true;
+      }
+    }
+    FailFirstAdaptor adaptor = new FailFirstAdaptor();
+    app = new Application(adaptor, config);
+    app.start();
+    assertTrue(adaptor.started);
+  }
+
   private static class NullAdaptor extends AbstractAdaptor {
     private boolean inited;
 
diff --git a/test/com/google/enterprise/adaptor/GsaCommunicationHandlerTest.java b/test/com/google/enterprise/adaptor/GsaCommunicationHandlerTest.java
index 7db5785..a673980 100644
--- a/test/com/google/enterprise/adaptor/GsaCommunicationHandlerTest.java
+++ b/test/com/google/enterprise/adaptor/GsaCommunicationHandlerTest.java
@@ -68,78 +68,50 @@
   @After
   public void teardown() {
     gsa.stop(0, TimeUnit.SECONDS);
+    gsa.teardown();
   }
 
   @Test
   public void testAdaptorContext() throws Exception {
-    final AtomicReference<Throwable> error = new AtomicReference<Throwable>();
-    class PollingIncrNullAdaptor extends NullAdaptor {
+    gsa = new GsaCommunicationHandler(new NullAdaptor(), config);
+    AdaptorContext context = gsa.setup(mockServer, mockServer, null);
+    assertSame(config, context.getConfig());
+    assertNotNull(context.getDocIdPusher());
+    assertNotNull(context.getDocIdEncoder());
+    assertNotNull(context.getSensitiveValueDecoder());
+    ExceptionHandler originalHandler
+        = context.getGetDocIdsFullErrorHandler();
+    ExceptionHandler replacementHandler
+        = ExceptionHandlers.exponentialBackoffHandler(
+            1, 1, TimeUnit.SECONDS);
+    assertNotNull(originalHandler);
+    context.setGetDocIdsFullErrorHandler(replacementHandler);
+    assertSame(replacementHandler,
+        context.getGetDocIdsFullErrorHandler());
+
+    StatusSource source = new MockStatusSource("test",
+        new MockStatus(Status.Code.NORMAL));
+    context.addStatusSource(source);
+
+    assertNotNull(context.createHttpContext("/test", new HttpHandler() {
       @Override
-      public void init(AdaptorContext context) {
-        try {
-          assertSame(config, context.getConfig());
-          assertNotNull(context.getDocIdPusher());
-          assertNotNull(context.getDocIdEncoder());
-          assertNotNull(context.getSensitiveValueDecoder());
-          ExceptionHandler originalHandler
-              = context.getGetDocIdsFullErrorHandler();
-          ExceptionHandler replacementHandler
-              = ExceptionHandlers.exponentialBackoffHandler(
-                  1, 1, TimeUnit.SECONDS);
-          assertNotNull(originalHandler);
-          context.setGetDocIdsFullErrorHandler(replacementHandler);
-          assertSame(replacementHandler,
-              context.getGetDocIdsFullErrorHandler());
-
-          StatusSource source = new MockStatusSource("test",
-              new MockStatus(Status.Code.NORMAL));
-          context.addStatusSource(source);
-
-          assertNotNull(context.createHttpContext("/test", new HttpHandler() {
-            @Override
-            public void handle(HttpExchange ex) {}
-          }));
-        } catch (Throwable t) {
-          error.set(t);
-        }
-      }
-    }
-    PollingIncrNullAdaptor adaptor = new PollingIncrNullAdaptor();
-    gsa = new GsaCommunicationHandler(adaptor, config);
-    gsa.start(mockServer, mockServer);
-    Throwable t = error.get();
-    if (t != null) {
-      if (t instanceof Exception) {
-        throw (Exception) t;
-      } else if (t instanceof Error) {
-        throw (Error) t;
-      } else {
-        throw new AssertionError();
-      }
-    }
+      public void handle(HttpExchange ex) {}
+    }));
   }
 
   @Test
   public void testPollingIncrementalAdaptor() throws Exception {
-    class PollingIncrNullAdaptor extends NullAdaptor
-        implements PollingIncrementalLister {
-      public final ArrayBlockingQueue<Object> queue
-          = new ArrayBlockingQueue<Object>(1);
-
-      @Override
-      public void init(AdaptorContext context) {
-        context.setPollingIncrementalLister(this);
-      }
-
+    gsa = new GsaCommunicationHandler(new NullAdaptor(), config);
+    AdaptorContext context = gsa.setup(mockServer, mockServer, null);
+    final ArrayBlockingQueue<Object> queue = new ArrayBlockingQueue<Object>(1);
+    context.setPollingIncrementalLister(new PollingIncrementalLister() {
       @Override
       public void getModifiedDocIds(DocIdPusher pusher) {
         queue.offer(new Object());
       }
-    }
-    PollingIncrNullAdaptor adaptor = new PollingIncrNullAdaptor();
-    gsa = new GsaCommunicationHandler(adaptor, config);
-    gsa.start(mockServer, mockServer);
-    assertNotNull(adaptor.queue.poll(1, TimeUnit.SECONDS));
+    });
+    gsa.start();
+    assertNotNull(queue.poll(1, TimeUnit.SECONDS));
   }
 
   @Test
@@ -147,93 +119,21 @@
     NullAdaptor adaptor = new NullAdaptor();
     config.setValue("adaptor.pushDocIdsOnStartup", "false");
     gsa = new GsaCommunicationHandler(adaptor, config);
-    gsa.start(mockServer, mockServer);
+    gsa.setup(mockServer, mockServer, null);
+    gsa.start();
     gsa.stop(1, TimeUnit.SECONDS);
-    gsa.start(mockServer, mockServer);
+    gsa.start();
     assertTrue(gsa.checkAndScheduleImmediatePushOfDocIds());
   }
 
-  @Test
-  public void testFailOnceInitAdaptor() throws Exception {
-    class FailFirstAdaptor extends NullAdaptor {
-      private int count = 0;
-      public boolean started = false;
-
-      @Override
-      public void init(AdaptorContext context) {
-        if (count == 0) {
-          count++;
-          throw new RuntimeException();
-        }
-        started = true;
-      }
-    }
-    FailFirstAdaptor adaptor = new FailFirstAdaptor();
-    gsa = new GsaCommunicationHandler(adaptor, config);
-    gsa.start(mockServer, mockServer);
-    assertTrue(adaptor.started);
-  }
-
   /**
-   * Tests that Adaptor is properly initialized before HTTP serving is started.
+   * Tests that HTTP serving not is started during setup().
    */
   @Test
-  public void testInitBeforeServing() throws Exception {
-    class SlowAdaptor extends NullAdaptor {
-      public AtomicBoolean initHasFinished = new AtomicBoolean();
-      public AtomicBoolean getCalledBeforeInitFinished = new AtomicBoolean();
-
-      @Override
-      public void init(AdaptorContext context) {
-        try {
-          Thread.sleep(500);  // the slowness of this adaptor
-          initHasFinished.set(true);
-        } catch (InterruptedException e) {
-          throw new AssertionError(e);
-        }
-      }
-
-      @Override
-      public void getDocContent(Request req, Response resp)
-          throws IOException {
-        if (!initHasFinished.get()) {
-          getCalledBeforeInitFinished.set(true);
-        }
-        resp.respondNotFound();
-      }
-    }
-    SlowAdaptor adaptor = new SlowAdaptor();
+  public void testNoServingBeforeStart() throws Exception {
     gsa = new GsaCommunicationHandler(adaptor, config);
-
-    Thread tryFetch = new Thread(new Runnable() {
-      @Override
-      public void run() {
-        while (true) {
-          HttpExchange ex = mockServer.createExchange("GET", "/doc/1");
-          if (ex != null) {
-            try {
-              mockServer.handle(ex);
-            } catch (IOException e) {
-              // We will already be retrying.
-            }
-          }
-
-          try {
-            Thread.sleep(20);
-          } catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-            return;
-          }
-        }
-      }
-    });
-    tryFetch.start();
-
-    gsa.start(mockServer, mockServer);
-    tryFetch.interrupt();
-    tryFetch.join();
-
-    assertEquals(false, adaptor.getCalledBeforeInitFinished.get());
+    gsa.setup(mockServer, mockServer, null);
+    assertEquals(0, mockServer.contexts.size());
   }
 
   @Test
@@ -249,9 +149,11 @@
     };
     config.setValue("adaptor.pushDocIdsOnStartup", "false");
     gsa = new GsaCommunicationHandler(adaptor, config);
-    gsa.start(mockServer, mockServer, "/path");
+    gsa.setup(mockServer, mockServer, "/path");
+    gsa.start();
     GsaCommunicationHandler gsa2 = new GsaCommunicationHandler(adaptor, config);
-    gsa2.start(mockServer, mockServer, "/path2");
+    gsa2.setup(mockServer, mockServer, "/path2");
+    gsa2.start();
 
     try {
       MockHttpExchange ex = mockServer.createExchange("GET", "/path/doc/1");
@@ -262,6 +164,7 @@
       assertEquals("2", new String(ex.getResponseBytes(), charset));
     } finally {
       gsa2.stop(0, TimeUnit.SECONDS);
+      gsa2.teardown();
       // gsa.stop() is called in @After, so no need for second finally for
       // shutting 'gsa' down.
     }
diff --git a/test/com/google/enterprise/adaptor/MockHttpServer.java b/test/com/google/enterprise/adaptor/MockHttpServer.java
index c324d5c..4e91988 100644
--- a/test/com/google/enterprise/adaptor/MockHttpServer.java
+++ b/test/com/google/enterprise/adaptor/MockHttpServer.java
@@ -26,7 +26,7 @@
  */
 public class MockHttpServer extends HttpServer {
   private final InetSocketAddress addr;
-  private final List<HttpContext> contexts = new ArrayList<HttpContext>();
+  final List<HttpContext> contexts = new ArrayList<HttpContext>();
 
   public MockHttpServer() {
     this(new InetSocketAddress(80));