Merge pull request #48 from neonichu/update-filecheck-cmake

Add path for llvm-3.6-tools
diff --git a/include/llbuild/BuildSystem/BuildExecutionQueue.h b/include/llbuild/BuildSystem/BuildExecutionQueue.h
index 75e4f7f..93cbef2 100644
--- a/include/llbuild/BuildSystem/BuildExecutionQueue.h
+++ b/include/llbuild/BuildSystem/BuildExecutionQueue.h
@@ -85,6 +85,9 @@
   /// Add a job to be executed.
   virtual void addJob(QueueJob job) = 0;
 
+  /// Cancel all jobs and subprocesses of this queue.
+  virtual void cancelAllJobs() = 0;
+
   /// @name Execution Interfaces
   ///
   /// These are additional interfaces provided by the execution queue which can
diff --git a/include/llbuild/BuildSystem/BuildSystem.h b/include/llbuild/BuildSystem/BuildSystem.h
index b873c2f..96da248 100644
--- a/include/llbuild/BuildSystem/BuildSystem.h
+++ b/include/llbuild/BuildSystem/BuildSystem.h
@@ -201,6 +201,9 @@
   /// if a cycle was discovered).
   bool build(StringRef target);
 
+  /// Cancel the current build
+  void cancel();
+
   /// @}
 };
 
diff --git a/include/llbuild/BuildSystem/BuildSystemFrontend.h b/include/llbuild/BuildSystem/BuildSystemFrontend.h
index 1185184..75979c8 100644
--- a/include/llbuild/BuildSystem/BuildSystemFrontend.h
+++ b/include/llbuild/BuildSystem/BuildSystemFrontend.h
@@ -92,6 +92,7 @@
   
 private:
   void* impl;
+  std::atomic<bool> isCancelled_;
 
   /// Default implementation, cannot be overriden by subclasses.
   virtual void setFileContentsBeingParsed(StringRef buffer) override;
@@ -127,6 +128,12 @@
   /// Provides a default cancellation implementation that will cancel when any
   /// command has failed.
   virtual bool isCancelled() override;
+
+  /// Cancels the current build.
+  virtual void cancel();
+
+  /// Reset mutable build state before a new build operation.
+  void resetForBuild();
   
   /// Provides a default handler.
   ///
diff --git a/lib/BuildSystem/BuildSystem.cpp b/lib/BuildSystem/BuildSystem.cpp
index 0bbfeac..02113ae 100644
--- a/lib/BuildSystem/BuildSystem.cpp
+++ b/lib/BuildSystem/BuildSystem.cpp
@@ -281,6 +281,10 @@
   return static_cast<BuildSystemEngineDelegate*>(
       engine.getDelegate())->getBuildSystem();
 }
+
+static bool isCancelled(BuildEngine& engine) {
+  return getBuildSystem(engine).getCommandInterface().getDelegate().isCancelled();
+}
   
 /// This is the task used to "build" a target, it translates between the request
 /// for building a target key and the requests for all of its nodes.
@@ -328,6 +332,12 @@
   }
 
   virtual void inputsAvailable(BuildEngine& engine) override {
+    // If the build should cancel, do nothing.
+    if (isCancelled(engine)) {
+      engine.taskIsComplete(this, BuildValue::makeSkippedCommand().toData());
+      return;
+    }
+
     if (hasMissingInput) {
       // FIXME: Design the logging and status output APIs.
       auto& system = getBuildSystem(engine);
@@ -2041,3 +2051,10 @@
 bool BuildSystem::build(StringRef name) {
   return static_cast<BuildSystemImpl*>(impl)->build(name);
 }
+
+void BuildSystem::cancel() {
+  if (impl) {
+    auto buildSystemImpl = static_cast<BuildSystemImpl*>(impl);
+    buildSystemImpl->getCommandInterface().getExecutionQueue().cancelAllJobs();
+  }
+}
diff --git a/lib/BuildSystem/BuildSystemFrontend.cpp b/lib/BuildSystem/BuildSystemFrontend.cpp
index 9ac4eca..89ccd0c 100644
--- a/lib/BuildSystem/BuildSystemFrontend.cpp
+++ b/lib/BuildSystem/BuildSystemFrontend.cpp
@@ -221,7 +221,7 @@
                             StringRef name,
                             uint32_t version)
     : BuildSystemDelegate(name, version),
-      impl(new BuildSystemFrontendDelegateImpl(sourceMgr, invocation))
+      impl(new BuildSystemFrontendDelegateImpl(sourceMgr, invocation)), isCancelled_(false)
 {
   
 }
@@ -330,7 +330,19 @@
 
 bool BuildSystemFrontendDelegate::isCancelled() {
   // Stop the build after any command failures.
-  return getNumFailedCommands() > 0;
+  return getNumFailedCommands() > 0 || isCancelled_;
+}
+
+void BuildSystemFrontendDelegate::cancel() {
+  // FIXME: We should audit that a build is happening.
+  isCancelled_ = true;
+
+  auto delegateImpl = static_cast<BuildSystemFrontendDelegateImpl*>(impl);
+  delegateImpl->system->cancel();
+}
+
+void BuildSystemFrontendDelegate::resetForBuild() {
+  isCancelled_ = false;
 }
 
 void BuildSystemFrontendDelegate::hadCommandFailure() {
diff --git a/lib/BuildSystem/LaneBasedExecutionQueue.cpp b/lib/BuildSystem/LaneBasedExecutionQueue.cpp
index 2954e51..5539337 100644
--- a/lib/BuildSystem/LaneBasedExecutionQueue.cpp
+++ b/lib/BuildSystem/LaneBasedExecutionQueue.cpp
@@ -16,6 +16,7 @@
 
 #include "llvm/ADT/ArrayRef.h"
 #include "llvm/ADT/SmallString.h"
+#include "llvm/ADT/STLExtras.h"
 #include "llvm/ADT/Twine.h"
 #include "llvm/Support/Path.h"
 #include "llvm/Support/Program.h"
@@ -28,6 +29,7 @@
 #include <thread>
 #include <vector>
 #include <string>
+#include <unordered_set>
 
 #include <fcntl.h>
 #include <pthread.h>
@@ -64,6 +66,16 @@
   std::mutex readyJobsMutex;
   std::condition_variable readyJobsCondition;
   
+  /// The set of spawned processes to terminate if we get cancelled.
+  std::unordered_set<pid_t> spawnedProcesses;
+  std::mutex spawnedProcessesMutex;
+
+  /// Management of cancellation and SIGKILL escalation
+  std::unique_ptr<std::thread> killAfterTimeoutThread = nullptr;
+  std::atomic<bool> cancelled { false };
+  std::condition_variable stopKillingCondition;
+  std::mutex stopKillingMutex;
+
   void executeLane(unsigned laneNumber) {
     // Set the thread name, if available.
 #if defined(__APPLE__)
@@ -78,7 +90,7 @@
 #endif
     
     // Execute items from the queue until shutdown.
-    while (true) {
+    while (!cancelled) {
       // Take a job from the ready queue.
       QueueJob job{};
       {
@@ -87,6 +99,10 @@
         // While the queue is empty, wait for an item.
         while (readyJobs.empty()) {
           readyJobsCondition.wait(lock);
+
+          if (cancelled) {
+            return;
+          }
         }
 
         // Take an item according to the chosen policy.
@@ -106,6 +122,21 @@
     }
   }
 
+  void killAfterTimeout() {
+    std::unique_lock<std::mutex> lock(stopKillingMutex);
+    stopKillingCondition.wait_for(lock, std::chrono::seconds(10));
+    sendSignalToProcesses(SIGKILL);
+  }
+
+  void sendSignalToProcesses(int signal) {
+    std::unique_lock<std::mutex> lock(spawnedProcessesMutex);
+
+    for (pid_t pid: spawnedProcesses) {
+      // We are killing the whole process group here, this depends on us spawning each process in its own group earlier
+      ::kill(-pid, signal);
+    }
+  }
+
 public:
   LaneBasedExecutionQueue(BuildExecutionQueueDelegate& delegate,
                           unsigned numLanes)
@@ -120,20 +151,43 @@
   
   virtual ~LaneBasedExecutionQueue() {
     // Shut down the lanes.
-    for (unsigned i = 0; i != numLanes; ++i) {
-      addJob({});
-    }
+    cancelled = true;
+    readyJobsCondition.notify_all();
+
     for (unsigned i = 0; i != numLanes; ++i) {
       lanes[i]->join();
     }
+
+    if (killAfterTimeoutThread) {
+      stopKillingCondition.notify_all();
+      killAfterTimeoutThread->join();
+    }
   }
 
   virtual void addJob(QueueJob job) override {
+    if (cancelled) {
+      // FIXME: We should eventually raise an error here as new work should not be enqueued after cancellation
+      return;
+    }
+
     std::lock_guard<std::mutex> guard(readyJobsMutex);
     readyJobs.push_back(job);
     readyJobsCondition.notify_one();
   }
 
+  virtual void cancelAllJobs() override {
+    auto wasAlreadyCancelled = cancelled.exchange(true);
+    // If we were already cancelled, do nothing.
+    if (wasAlreadyCancelled) {
+      return;
+    }
+
+    readyJobsCondition.notify_all();
+
+    sendSignalToProcesses(SIGINT);
+    killAfterTimeoutThread = llvm::make_unique<std::thread>(&LaneBasedExecutionQueue::killAfterTimeout, this);
+  }
+
   virtual bool
   executeProcess(QueueJobContext* opaqueContext,
                  ArrayRef<StringRef> commandLine,
@@ -274,19 +328,24 @@
     }
       
     // Spawn the command.
-    //
-    // FIXME: Need to track spawned processes for the purposes of cancellation.
-    
     pid_t pid;
-    if (posix_spawn(&pid, args[0], /*file_actions=*/&fileActions,
-                    /*attrp=*/&attributes, const_cast<char**>(args.data()),
-                    envp) != 0) {
-      getDelegate().commandProcessHadError(
-          context.job.getForCommand(), handle,
-          Twine("unable to spawn process (") + strerror(errno) + ")");
-      getDelegate().commandProcessFinished(context.job.getForCommand(), handle,
-                                           -1);
-      return false;
+    {
+      // We need to hold the spawn processes lock when we spawn, to ensure that
+      // we don't create a process in between when we are cancelled.
+      std::lock_guard<std::mutex> guard(spawnedProcessesMutex);
+
+      if (posix_spawn(&pid, args[0], /*file_actions=*/&fileActions,
+                      /*attrp=*/&attributes, const_cast<char**>(args.data()),
+                      envp) != 0) {
+        getDelegate().commandProcessHadError(
+            context.job.getForCommand(), handle,
+            Twine("unable to spawn process (") + strerror(errno) + ")");
+        getDelegate().commandProcessFinished(context.job.getForCommand(), handle,
+                                             -1);
+        return false;
+      }
+
+      spawnedProcesses.insert(pid);
     }
 
     posix_spawn_file_actions_destroy(&fileActions);
@@ -323,6 +382,13 @@
     int status, result = waitpid(pid, &status, 0);
     while (result == -1 && errno == EINTR)
       result = waitpid(pid, &status, 0);
+
+    // Update the set of spawned processes.
+    {
+        std::lock_guard<std::mutex> guard(spawnedProcessesMutex);
+        spawnedProcesses.erase(pid);
+    }
+
     if (result == -1) {
       getDelegate().commandProcessHadError(
           context.job.getForCommand(), handle,
diff --git a/lib/BuildSystem/SwiftTools.cpp b/lib/BuildSystem/SwiftTools.cpp
index 788efc0..24a1a82 100644
--- a/lib/BuildSystem/SwiftTools.cpp
+++ b/lib/BuildSystem/SwiftTools.cpp
@@ -106,6 +106,12 @@
 
   virtual void inputsAvailable(BuildSystemCommandInterface& bsci,
                                core::Task* task) override {
+    // If the build should cancel, do nothing.
+    if (bsci.getDelegate().isCancelled()) {
+      bsci.taskIsComplete(task, BuildValue::makeSkippedCommand());
+      return;
+    }
+
     // Dispatch a task to query the compiler version.
     auto fn = [this, &bsci=bsci, task=task](QueueJobContext* context) {
       // Suppress static analyzer false positive on generalized lambda capture
diff --git a/lib/Commands/BuildSystemCommand.cpp b/lib/Commands/BuildSystemCommand.cpp
index ac40f75..05fae2f 100644
--- a/lib/Commands/BuildSystemCommand.cpp
+++ b/lib/Commands/BuildSystemCommand.cpp
@@ -26,6 +26,10 @@
 #include "CommandUtil.h"
 
 #include <cerrno>
+#include <thread>
+
+#include <signal.h>
+#include <unistd.h>
 
 using namespace llbuild;
 using namespace llbuild::commands;
@@ -405,13 +409,87 @@
 
 class BasicBuildSystemFrontendDelegate : public BuildSystemFrontendDelegate {
   std::unique_ptr<basic::FileSystem> fileSystem;
+
+  /// The previous SIGINT handler.
+  struct sigaction previousSigintHandler;
+
+  /// Low-level flag for when a SIGINT has been received.
+  static std::atomic<bool> wasInterrupted;
+
+  /// Pipe used to allow detection of signals.
+  static int signalWatchingPipe[2];
+
+  static void sigintHandler(int) {
+    // Set the atomic interrupt flag.
+    BasicBuildSystemFrontendDelegate::wasInterrupted = true;
+
+    // Write to wake up the signal monitoring thread.
+    char byte{};
+    write(signalWatchingPipe[1], &byte, 1);
+  }
+
+  /// Check if an interrupt has occurred.
+  void checkForInterrupt() {
+    // Save and clear the interrupt flag, atomically.
+    bool wasInterrupted = BasicBuildSystemFrontendDelegate::wasInterrupted.exchange(false);
+
+    // Process the interrupt flag, if present.
+    if (wasInterrupted) {
+      // Otherwise, cancel the build.
+      printf("cancelling build.\n");
+      cancel();
+    }
+  }
+
+  /// Thread function to wait for indications that signals have arrived and to
+  /// process them.
+  void signalWaitThread() {
+    // Wait for signal arrival indications.
+    while (true) {
+      char byte;
+      int res = read(signalWatchingPipe[0], &byte, 1);
+
+      // If nothing was read, the pipe has been closed and we should shut down.
+      if (res == 0)
+        break;
+
+      // Otherwise, check if we were awoke because of an interrupt.
+      checkForInterrupt();
+    }
+
+    // Shut down the pipe.
+    close(signalWatchingPipe[0]);
+    signalWatchingPipe[0] = -1;
+  }
   
 public:
   BasicBuildSystemFrontendDelegate(llvm::SourceMgr& sourceMgr,
                                    const BuildSystemInvocation& invocation)
       : BuildSystemFrontendDelegate(sourceMgr, invocation,
                                     "basic", /*version=*/0),
-        fileSystem(basic::createLocalFileSystem()) {}
+        fileSystem(basic::createLocalFileSystem()) {
+    // Register an interrupt handler.
+    struct sigaction action{};
+    action.sa_handler = &BasicBuildSystemFrontendDelegate::sigintHandler;
+    sigaction(SIGINT, &action, &previousSigintHandler);
+
+    // Create a pipe and thread to watch for signals.
+    assert(BasicBuildSystemFrontendDelegate::signalWatchingPipe[0] == -1 &&
+           BasicBuildSystemFrontendDelegate::signalWatchingPipe[1] == -1);
+    if (::pipe(BasicBuildSystemFrontendDelegate::signalWatchingPipe) < 0) {
+      perror("pipe");
+    }
+    new std::thread(&BasicBuildSystemFrontendDelegate::signalWaitThread, this);
+  }
+
+  ~BasicBuildSystemFrontendDelegate() {
+    // Restore any previous SIGINT handler.
+    sigaction(SIGINT, &previousSigintHandler, NULL);
+
+    // Close the signal watching pipe.
+    ::close(BasicBuildSystemFrontendDelegate::signalWatchingPipe[1]);
+    signalWatchingPipe[1] = -1;
+  }
 
   virtual basic::FileSystem& getFileSystem() override { return *fileSystem; }
 
@@ -421,6 +499,9 @@
   }
 };
 
+std::atomic<bool> BasicBuildSystemFrontendDelegate::wasInterrupted{false};
+int BasicBuildSystemFrontendDelegate::signalWatchingPipe[2]{-1, -1};
+
 static void buildUsage(int exitCode) {
   int optionWidth = 25;
   fprintf(stderr, "Usage: %s buildsystem build [options] [<target>]\n",
diff --git a/llbuild.xcodeproj/project.pbxproj b/llbuild.xcodeproj/project.pbxproj
index 6655bed..39ad4c5 100644
--- a/llbuild.xcodeproj/project.pbxproj
+++ b/llbuild.xcodeproj/project.pbxproj
@@ -55,6 +55,7 @@
 				E1A2254719F9A2140059043E /* PBXTargetDependency */,
 				E147DF1C1BA81DAE0032D08E /* PBXTargetDependency */,
 				E1A2254919F9A2170059043E /* PBXTargetDependency */,
+				9DB047BF1DF9D4B8006CDF52 /* PBXTargetDependency */,
 				E1A2254B19F9A2190059043E /* PBXTargetDependency */,
 			);
 			name = test;
@@ -63,6 +64,12 @@
 /* End PBXAggregateTarget section */
 
 /* Begin PBXBuildFile section */
+		9D2107C61DFADDFA00BE26FF /* libcurses.dylib in Frameworks */ = {isa = PBXBuildFile; fileRef = E15B6EC61B546A2C00643066 /* libcurses.dylib */; };
+		9DB047BA1DF9D4A4006CDF52 /* libgtest_main.a in Frameworks */ = {isa = PBXBuildFile; fileRef = E1A224E619F99C580059043E /* libgtest_main.a */; };
+		9DB047BB1DF9D4A4006CDF52 /* libgtest.a in Frameworks */ = {isa = PBXBuildFile; fileRef = E1A224DD19F99B0E0059043E /* libgtest.a */; };
+		9DB047BC1DF9D4AA006CDF52 /* libllvmSupport.a in Frameworks */ = {isa = PBXBuildFile; fileRef = E1B838A21B52E7DE00DB876B /* libllvmSupport.a */; };
+		9DB047BD1DF9D4B0006CDF52 /* libllbuildBuildSystem.a in Frameworks */ = {isa = PBXBuildFile; fileRef = E1B839571B541BFD00DB876B /* libllbuildBuildSystem.a */; };
+		9DB047C01DF9F592006CDF52 /* LaneBasedExecutionQueueTest.cpp in Sources */ = {isa = PBXBuildFile; fileRef = 9DB0478B1DF9D3E2006CDF52 /* LaneBasedExecutionQueueTest.cpp */; };
 		E104FAF71B655A97005C68A0 /* BuildSystemPerfTests.mm in Sources */ = {isa = PBXBuildFile; fileRef = E104FAF61B655A97005C68A0 /* BuildSystemPerfTests.mm */; };
 		E104FAFA1B655BBA005C68A0 /* libllbuildBuildSystem.a in Frameworks */ = {isa = PBXBuildFile; fileRef = E1B839571B541BFD00DB876B /* libllbuildBuildSystem.a */; };
 		E104FAFB1B655C33005C68A0 /* libllvmSupport.a in Frameworks */ = {isa = PBXBuildFile; fileRef = E1B838A21B52E7DE00DB876B /* libllvmSupport.a */; };
@@ -210,6 +217,48 @@
 /* End PBXBuildFile section */
 
 /* Begin PBXContainerItemProxy section */
+		9D2107C31DF9FBFA00BE26FF /* PBXContainerItemProxy */ = {
+			isa = PBXContainerItemProxy;
+			containerPortal = E1A223E919F98F1C0059043E /* Project object */;
+			proxyType = 1;
+			remoteGlobalIDString = 9DB047A71DF9D43D006CDF52;
+			remoteInfo = BuildSystemTests;
+		};
+		9DB047AF1DF9D464006CDF52 /* PBXContainerItemProxy */ = {
+			isa = PBXContainerItemProxy;
+			containerPortal = E1A223E919F98F1C0059043E /* Project object */;
+			proxyType = 1;
+			remoteGlobalIDString = E1A224DC19F99B0E0059043E;
+			remoteInfo = gtest;
+		};
+		9DB047B11DF9D469006CDF52 /* PBXContainerItemProxy */ = {
+			isa = PBXContainerItemProxy;
+			containerPortal = E1A223E919F98F1C0059043E /* Project object */;
+			proxyType = 1;
+			remoteGlobalIDString = E1A224E519F99C580059043E;
+			remoteInfo = gtest_main;
+		};
+		9DB047B31DF9D471006CDF52 /* PBXContainerItemProxy */ = {
+			isa = PBXContainerItemProxy;
+			containerPortal = E1A223E919F98F1C0059043E /* Project object */;
+			proxyType = 1;
+			remoteGlobalIDString = E1B838981B52E7DE00DB876B;
+			remoteInfo = llvmSupport;
+		};
+		9DB047B71DF9D483006CDF52 /* PBXContainerItemProxy */ = {
+			isa = PBXContainerItemProxy;
+			containerPortal = E1A223E919F98F1C0059043E /* Project object */;
+			proxyType = 1;
+			remoteGlobalIDString = E1B839481B541BFD00DB876B;
+			remoteInfo = llbuildBuildSystem;
+		};
+		9DB047BE1DF9D4B8006CDF52 /* PBXContainerItemProxy */ = {
+			isa = PBXContainerItemProxy;
+			containerPortal = E1A223E919F98F1C0059043E /* Project object */;
+			proxyType = 1;
+			remoteGlobalIDString = 9DB047A71DF9D43D006CDF52;
+			remoteInfo = BuildSystemTests;
+		};
 		E104FAF81B655BB2005C68A0 /* PBXContainerItemProxy */ = {
 			isa = PBXContainerItemProxy;
 			containerPortal = E1A223E919F98F1C0059043E /* Project object */;
@@ -577,6 +626,15 @@
 /* End PBXContainerItemProxy section */
 
 /* Begin PBXCopyFilesBuildPhase section */
+		9DB047A61DF9D43D006CDF52 /* CopyFiles */ = {
+			isa = PBXCopyFilesBuildPhase;
+			buildActionMask = 2147483647;
+			dstPath = /usr/share/man/man1/;
+			dstSubfolderSpec = 0;
+			files = (
+			);
+			runOnlyForDeploymentPostprocessing = 1;
+		};
 		E147DF111BA81D330032D08E /* CopyFiles */ = {
 			isa = PBXCopyFilesBuildPhase;
 			buildActionMask = 2147483647;
@@ -628,6 +686,8 @@
 		54E187B61CD296EA00F7EC89 /* BuildNode.h */ = {isa = PBXFileReference; lastKnownFileType = sourcecode.c.h; path = BuildNode.h; sourceTree = "<group>"; };
 		54E187B71CD296EA00F7EC89 /* ExternalCommand.h */ = {isa = PBXFileReference; lastKnownFileType = sourcecode.c.h; path = ExternalCommand.h; sourceTree = "<group>"; };
 		54E187B81CD296EA00F7EC89 /* SwiftTools.h */ = {isa = PBXFileReference; lastKnownFileType = sourcecode.c.h; path = SwiftTools.h; sourceTree = "<group>"; };
+		9DB0478B1DF9D3E2006CDF52 /* LaneBasedExecutionQueueTest.cpp */ = {isa = PBXFileReference; lastKnownFileType = sourcecode.cpp.cpp; path = LaneBasedExecutionQueueTest.cpp; sourceTree = "<group>"; };
+		9DB047A81DF9D43D006CDF52 /* BuildSystemTests */ = {isa = PBXFileReference; explicitFileType = "compiled.mach-o.executable"; includeInIndex = 0; path = BuildSystemTests; sourceTree = BUILT_PRODUCTS_DIR; };
 		E104FAF61B655A97005C68A0 /* BuildSystemPerfTests.mm */ = {isa = PBXFileReference; fileEncoding = 4; lastKnownFileType = sourcecode.cpp.objcpp; path = BuildSystemPerfTests.mm; sourceTree = "<group>"; };
 		E104FAFF1B6568E0005C68A0 /* BuildSystem.cpp */ = {isa = PBXFileReference; fileEncoding = 4; lastKnownFileType = sourcecode.cpp.cpp; path = BuildSystem.cpp; sourceTree = "<group>"; };
 		E1066C071BC5ACAB00B892CE /* LaneBasedExecutionQueue.cpp */ = {isa = PBXFileReference; fileEncoding = 4; lastKnownFileType = sourcecode.cpp.cpp; path = LaneBasedExecutionQueue.cpp; sourceTree = "<group>"; };
@@ -911,6 +971,18 @@
 /* End PBXFileReference section */
 
 /* Begin PBXFrameworksBuildPhase section */
+		9DB047A51DF9D43D006CDF52 /* Frameworks */ = {
+			isa = PBXFrameworksBuildPhase;
+			buildActionMask = 2147483647;
+			files = (
+				9D2107C61DFADDFA00BE26FF /* libcurses.dylib in Frameworks */,
+				9DB047BD1DF9D4B0006CDF52 /* libllbuildBuildSystem.a in Frameworks */,
+				9DB047BC1DF9D4AA006CDF52 /* libllvmSupport.a in Frameworks */,
+				9DB047BA1DF9D4A4006CDF52 /* libgtest_main.a in Frameworks */,
+				9DB047BB1DF9D4A4006CDF52 /* libgtest.a in Frameworks */,
+			);
+			runOnlyForDeploymentPostprocessing = 0;
+		};
 		E10D5CD719FEBF6A00211ED4 /* Frameworks */ = {
 			isa = PBXFrameworksBuildPhase;
 			buildActionMask = 2147483647;
@@ -1084,6 +1156,14 @@
 /* End PBXFrameworksBuildPhase section */
 
 /* Begin PBXGroup section */
+		9DB0478A1DF9D39E006CDF52 /* BuildSystem */ = {
+			isa = PBXGroup;
+			children = (
+				9DB0478B1DF9D3E2006CDF52 /* LaneBasedExecutionQueueTest.cpp */,
+			);
+			path = BuildSystem;
+			sourceTree = "<group>";
+		};
 		E10D5CDB19FEBF6A00211ED4 /* LitXCTestAdaptor */ = {
 			isa = PBXGroup;
 			children = (
@@ -1167,6 +1247,7 @@
 				E1B839571B541BFD00DB876B /* libllbuildBuildSystem.a */,
 				E147DF161BA81D330032D08E /* BasicTests */,
 				E1604CB11BB9E01D001153A1 /* swift-build-tool */,
+				9DB047A81DF9D43D006CDF52 /* BuildSystemTests */,
 			);
 			name = Products;
 			sourceTree = "<group>";
@@ -1420,6 +1501,7 @@
 			children = (
 				E147DF171BA81D4E0032D08E /* Basic */,
 				E1A224B419F998D40059043E /* Core */,
+				9DB0478A1DF9D39E006CDF52 /* BuildSystem */,
 				E1A224B819F998D40059043E /* Ninja */,
 				E1A224B319F998D40059043E /* CMakeLists.txt */,
 			);
@@ -1837,6 +1919,28 @@
 /* End PBXHeadersBuildPhase section */
 
 /* Begin PBXNativeTarget section */
+		9DB047A71DF9D43D006CDF52 /* BuildSystemTests */ = {
+			isa = PBXNativeTarget;
+			buildConfigurationList = 9DB047AC1DF9D43D006CDF52 /* Build configuration list for PBXNativeTarget "BuildSystemTests" */;
+			buildPhases = (
+				9DB047A41DF9D43D006CDF52 /* Sources */,
+				9DB047A51DF9D43D006CDF52 /* Frameworks */,
+				9DB047A61DF9D43D006CDF52 /* CopyFiles */,
+				9D2107C51DFA07D700BE26FF /* Create Target Link */,
+			);
+			buildRules = (
+			);
+			dependencies = (
+				9DB047B81DF9D483006CDF52 /* PBXTargetDependency */,
+				9DB047B41DF9D471006CDF52 /* PBXTargetDependency */,
+				9DB047B21DF9D469006CDF52 /* PBXTargetDependency */,
+				9DB047B01DF9D464006CDF52 /* PBXTargetDependency */,
+			);
+			name = BuildSystemTests;
+			productName = BuildSystemTests;
+			productReference = 9DB047A81DF9D43D006CDF52 /* BuildSystemTests */;
+			productType = "com.apple.product-type.tool";
+		};
 		E10D5CD919FEBF6A00211ED4 /* LitXCTestAdaptor */ = {
 			isa = PBXNativeTarget;
 			buildConfigurationList = E10D5CE219FEBF6A00211ED4 /* Build configuration list for PBXNativeTarget "LitXCTestAdaptor" */;
@@ -1848,6 +1952,7 @@
 			buildRules = (
 			);
 			dependencies = (
+				9D2107C41DF9FBFA00BE26FF /* PBXTargetDependency */,
 				E18043341A00123600662FE7 /* PBXTargetDependency */,
 				E180436E1A001E6C00662FE7 /* PBXTargetDependency */,
 				E1DE1CC01BB9E34700A902C1 /* PBXTargetDependency */,
@@ -2183,6 +2288,10 @@
 				LastUpgradeCheck = 0830;
 				ORGANIZATIONNAME = "Apple Inc.";
 				TargetAttributes = {
+					9DB047A71DF9D43D006CDF52 = {
+						CreatedOnToolsVersion = 8.3;
+						ProvisioningStyle = Automatic;
+					};
 					E10D5CD919FEBF6A00211ED4 = {
 						CreatedOnToolsVersion = 6.3;
 						ProvisioningStyle = Manual;
@@ -2291,6 +2400,7 @@
 				E1A224E519F99C580059043E /* gtest_main */,
 				E147DEFE1BA81D330032D08E /* BasicTests */,
 				E1A224EE19F99D880059043E /* CoreTests */,
+				9DB047A71DF9D43D006CDF52 /* BuildSystemTests */,
 				E1A224F819F99DFD0059043E /* NinjaTests */,
 				E180432F1A00122900662FE7 /* Lit-Tests-Setup */,
 				E17C29ED1B5AC12B00C12DA9 /* Sphinx-Docs */,
@@ -2328,6 +2438,20 @@
 /* End PBXResourcesBuildPhase section */
 
 /* Begin PBXShellScriptBuildPhase section */
+		9D2107C51DFA07D700BE26FF /* Create Target Link */ = {
+			isa = PBXShellScriptBuildPhase;
+			buildActionMask = 2147483647;
+			files = (
+			);
+			inputPaths = (
+			);
+			name = "Create Target Link";
+			outputPaths = (
+			);
+			runOnlyForDeploymentPostprocessing = 0;
+			shellPath = /bin/sh;
+			shellScript = "\"${SRCROOT}/utils/Xcode/create-target-link.sh\"";
+		};
 		E147DF121BA81D330032D08E /* Create Target Link */ = {
 			isa = PBXShellScriptBuildPhase;
 			buildActionMask = 2147483647;
@@ -2454,6 +2578,14 @@
 /* End PBXShellScriptBuildPhase section */
 
 /* Begin PBXSourcesBuildPhase section */
+		9DB047A41DF9D43D006CDF52 /* Sources */ = {
+			isa = PBXSourcesBuildPhase;
+			buildActionMask = 2147483647;
+			files = (
+				9DB047C01DF9F592006CDF52 /* LaneBasedExecutionQueueTest.cpp in Sources */,
+			);
+			runOnlyForDeploymentPostprocessing = 0;
+		};
 		E10D5CD619FEBF6A00211ED4 /* Sources */ = {
 			isa = PBXSourcesBuildPhase;
 			buildActionMask = 2147483647;
@@ -2663,6 +2795,36 @@
 /* End PBXSourcesBuildPhase section */
 
 /* Begin PBXTargetDependency section */
+		9D2107C41DF9FBFA00BE26FF /* PBXTargetDependency */ = {
+			isa = PBXTargetDependency;
+			target = 9DB047A71DF9D43D006CDF52 /* BuildSystemTests */;
+			targetProxy = 9D2107C31DF9FBFA00BE26FF /* PBXContainerItemProxy */;
+		};
+		9DB047B01DF9D464006CDF52 /* PBXTargetDependency */ = {
+			isa = PBXTargetDependency;
+			target = E1A224DC19F99B0E0059043E /* gtest */;
+			targetProxy = 9DB047AF1DF9D464006CDF52 /* PBXContainerItemProxy */;
+		};
+		9DB047B21DF9D469006CDF52 /* PBXTargetDependency */ = {
+			isa = PBXTargetDependency;
+			target = E1A224E519F99C580059043E /* gtest_main */;
+			targetProxy = 9DB047B11DF9D469006CDF52 /* PBXContainerItemProxy */;
+		};
+		9DB047B41DF9D471006CDF52 /* PBXTargetDependency */ = {
+			isa = PBXTargetDependency;
+			target = E1B838981B52E7DE00DB876B /* llvmSupport */;
+			targetProxy = 9DB047B31DF9D471006CDF52 /* PBXContainerItemProxy */;
+		};
+		9DB047B81DF9D483006CDF52 /* PBXTargetDependency */ = {
+			isa = PBXTargetDependency;
+			target = E1B839481B541BFD00DB876B /* llbuildBuildSystem */;
+			targetProxy = 9DB047B71DF9D483006CDF52 /* PBXContainerItemProxy */;
+		};
+		9DB047BF1DF9D4B8006CDF52 /* PBXTargetDependency */ = {
+			isa = PBXTargetDependency;
+			target = 9DB047A71DF9D43D006CDF52 /* BuildSystemTests */;
+			targetProxy = 9DB047BE1DF9D4B8006CDF52 /* PBXContainerItemProxy */;
+		};
 		E104FAF91B655BB2005C68A0 /* PBXTargetDependency */ = {
 			isa = PBXTargetDependency;
 			target = E1B839481B541BFD00DB876B /* llbuildBuildSystem */;
@@ -2926,6 +3088,48 @@
 /* End PBXTargetDependency section */
 
 /* Begin XCBuildConfiguration section */
+		9DB047AD1DF9D43D006CDF52 /* Debug */ = {
+			isa = XCBuildConfiguration;
+			buildSettings = {
+				CLANG_ANALYZER_NONNULL = YES;
+				CLANG_CXX_LANGUAGE_STANDARD = "gnu++0x";
+				CLANG_WARN_DOCUMENTATION_COMMENTS = YES;
+				CODE_SIGN_IDENTITY = "-";
+				DEBUG_INFORMATION_FORMAT = dwarf;
+				GCC_ENABLE_CPP_RTTI = NO;
+				GCC_PREPROCESSOR_DEFINITIONS = (
+					"DEBUG=1",
+					"$(inherited)",
+					"GTEST_HAS_RTTI=0",
+				);
+				GCC_WARN_64_TO_32_BIT_CONVERSION = NO;
+				MACOSX_DEPLOYMENT_TARGET = 10.12;
+				MTL_ENABLE_DEBUG_INFO = YES;
+				PRODUCT_NAME = "$(TARGET_NAME)";
+				USER_HEADER_SEARCH_PATHS = "$(SRCROOT)/include $(SRCROOT)/utils/unittest/googletest/include";
+			};
+			name = Debug;
+		};
+		9DB047AE1DF9D43D006CDF52 /* Release */ = {
+			isa = XCBuildConfiguration;
+			buildSettings = {
+				CLANG_ANALYZER_NONNULL = YES;
+				CLANG_CXX_LANGUAGE_STANDARD = "gnu++0x";
+				CLANG_WARN_DOCUMENTATION_COMMENTS = YES;
+				CODE_SIGN_IDENTITY = "-";
+				GCC_ENABLE_CPP_RTTI = NO;
+				GCC_PREPROCESSOR_DEFINITIONS = (
+					NDEBUG,
+					"GTEST_HAS_RTTI=0",
+				);
+				GCC_WARN_64_TO_32_BIT_CONVERSION = NO;
+				MACOSX_DEPLOYMENT_TARGET = 10.12;
+				MTL_ENABLE_DEBUG_INFO = NO;
+				PRODUCT_NAME = "$(TARGET_NAME)";
+				USER_HEADER_SEARCH_PATHS = "$(SRCROOT)/include $(SRCROOT)/utils/unittest/googletest/include";
+			};
+			name = Release;
+		};
 		E10D5CE019FEBF6A00211ED4 /* Debug */ = {
 			isa = XCBuildConfiguration;
 			buildSettings = {
@@ -3518,6 +3722,15 @@
 /* End XCBuildConfiguration section */
 
 /* Begin XCConfigurationList section */
+		9DB047AC1DF9D43D006CDF52 /* Build configuration list for PBXNativeTarget "BuildSystemTests" */ = {
+			isa = XCConfigurationList;
+			buildConfigurations = (
+				9DB047AD1DF9D43D006CDF52 /* Debug */,
+				9DB047AE1DF9D43D006CDF52 /* Release */,
+			);
+			defaultConfigurationIsVisible = 0;
+			defaultConfigurationName = Release;
+		};
 		E10D5CE219FEBF6A00211ED4 /* Build configuration list for PBXNativeTarget "LitXCTestAdaptor" */ = {
 			isa = XCConfigurationList;
 			buildConfigurations = (
diff --git a/products/libllbuild/BuildSystem-C-API.cpp b/products/libllbuild/BuildSystem-C-API.cpp
index b988964..ffd7e91 100644
--- a/products/libllbuild/BuildSystem-C-API.cpp
+++ b/products/libllbuild/BuildSystem-C-API.cpp
@@ -117,14 +117,13 @@
 class CAPIBuildSystemFrontendDelegate : public BuildSystemFrontendDelegate {
   llb_buildsystem_delegate_t cAPIDelegate;
   CAPIFileSystem fileSystem;
-  std::atomic<bool> isCancelled_;
 
 public:
   CAPIBuildSystemFrontendDelegate(llvm::SourceMgr& sourceMgr,
                                   BuildSystemInvocation& invocation,
                                   llb_buildsystem_delegate_t delegate)
       : BuildSystemFrontendDelegate(sourceMgr, invocation, "basic", 0),
-        cAPIDelegate(delegate), fileSystem(delegate), isCancelled_(false) { }
+        cAPIDelegate(delegate), fileSystem(delegate) { }
 
   virtual basic::FileSystem& getFileSystem() override { return fileSystem; }
   
@@ -142,10 +141,6 @@
     return std::unique_ptr<Tool>((Tool*)tool);
   }
 
-  virtual bool isCancelled() override {
-    return isCancelled_;
-  }
-
   virtual void hadCommandFailure() override {
     // Call the base implementation.
     BuildSystemFrontendDelegate::hadCommandFailure();
@@ -255,18 +250,9 @@
     }
   }
 
-  /// Reset mutable build state before a new build operation.
-  void resetForBuild() {
-    isCancelled_ = false;
-  }
-
   /// Request cancellation of any current build.
-  void cancel() {
-    // FIXME: We need to implement BuildSystem layer support for real
-    // cancellation (including task and subprocess termination).
-
-    // FIXME: We should audit that a build is happening.
-    isCancelled_ = true;
+  void cancel() override {
+    BuildSystemFrontendDelegate::cancel();
   }
 };
 
diff --git a/tests/BuildSystem/Build/Inputs/ignore-sigint b/tests/BuildSystem/Build/Inputs/ignore-sigint
new file mode 100755
index 0000000..210b8fa
--- /dev/null
+++ b/tests/BuildSystem/Build/Inputs/ignore-sigint
@@ -0,0 +1,8 @@
+#!/bin/sh
+
+trap 'echo "Ignoring SIGINT..."' INT
+
+while [ true ]
+do
+  yes >/dev/null
+done
diff --git a/tests/BuildSystem/Build/sigkill-escalation.llbuild b/tests/BuildSystem/Build/sigkill-escalation.llbuild
new file mode 100644
index 0000000..4f4835b
--- /dev/null
+++ b/tests/BuildSystem/Build/sigkill-escalation.llbuild
@@ -0,0 +1,32 @@
+# Check that aborting a build escalates to SIGKILL after a grace period.
+
+# RUN: rm -rf %t.build
+# RUN: mkdir -p %t.build
+# RUN: cp %s %t.build/build.llbuild
+# RUN: cp %S/Inputs/wait-for-file %t.build
+# RUN: cp %S/Inputs/ignore-sigint %t.build
+# RUN: /bin/bash -x -c \
+# RUN:   "%{llbuild} buildsystem build --serial --chdir %t.build --no-db &> %t.out & \
+# RUN:    echo $! >%t.build/llbuild.pid; \
+# RUN:    wait $(cat %t.build/llbuild.pid)" || true
+# RUN: %{FileCheck} --input-file %t.out %s
+#
+# CHECK: SIGNAL
+# CHECK: cancelling build.
+
+client:
+  name: basic
+
+targets:
+  "": ["llbuild.pid"]
+
+commands:  
+  SIGNAL:
+    tool: shell
+    inputs: ["<cleanup>"]
+    outputs: ["llbuild.pid"]
+    description: "SIGNAL"
+    args:
+      ./ignore-sigint &
+      ./wait-for-file llbuild.pid &&
+      cat llbuild.pid | xargs kill -SIGINT
diff --git a/tests/BuildSystem/Build/signal-handling.llbuild b/tests/BuildSystem/Build/signal-handling.llbuild
new file mode 100644
index 0000000..2fd7ab4
--- /dev/null
+++ b/tests/BuildSystem/Build/signal-handling.llbuild
@@ -0,0 +1,30 @@
+# Check that SIGINT handling aborts a build.
+
+# RUN: rm -rf %t.build
+# RUN: mkdir -p %t.build
+# RUN: cp %s %t.build/build.llbuild
+# RUN: cp %S/Inputs/wait-for-file %t.build
+# RUN: /bin/bash -x -c \
+# RUN:   "%{llbuild} buildsystem build --serial --chdir %t.build --no-db &> %t.out & \
+# RUN:    echo $! >%t.build/llbuild.pid; \
+# RUN:    wait $(cat %t.build/llbuild.pid)" || true
+# RUN: %{FileCheck} --input-file %t.out %s
+#
+# CHECK: SIGNAL
+# CHECK: cancelling build.
+
+client:
+  name: basic
+
+targets:
+  "": ["llbuild.pid"]
+
+commands:  
+  SIGNAL:
+    tool: shell
+    inputs: ["<cleanup>"]
+    outputs: ["llbuild.pid"]
+    description: "SIGNAL"
+    args:
+      ./wait-for-file llbuild.pid &&
+      cat llbuild.pid | xargs kill -SIGINT
diff --git a/tests/Ninja/Build/signal-handling.ninja b/tests/Ninja/Build/signal-handling.ninja
index 1cadaec..3378c43 100644
--- a/tests/Ninja/Build/signal-handling.ninja
+++ b/tests/Ninja/Build/signal-handling.ninja
@@ -4,7 +4,7 @@
 # RUN: mkdir -p %t.build
 # RUN: cp %s %t.build/build.ninja
 # RUN: cp %S/Inputs/wait-for-file %t.build
-# RUN: /bin/sh -c \
+# RUN: /bin/bash -c \
 # RUN:   "%{llbuild} ninja build --jobs 1 --chdir %t.build &> %t.out & \
 # RUN:    echo $! > %t.build/llbuild.pid; \
 # RUN:    wait $(cat %t.build/llbuild.pid)" || true
@@ -12,9 +12,6 @@
 #
 # CHECK: [1/{{.*}}] SIGNAL PARENT LLBUILD PROCESS
 # CHECK: cancelling build.
-#
-# FIXME: Disabled on Linux for the time being, it doesn't always pass.
-# REQUIRES: platform=Darwin
 
 rule CAT
   command = cat ${in} > ${out}
diff --git a/unittests/BuildSystem/CMakeLists.txt b/unittests/BuildSystem/CMakeLists.txt
new file mode 100644
index 0000000..6512a3f
--- /dev/null
+++ b/unittests/BuildSystem/CMakeLists.txt
@@ -0,0 +1,5 @@
+add_llbuild_unittest(BuildSystemTests
+  LaneBasedExecutionQueueTest
+  )
+
+target_link_libraries(BuildSystemTests curses llbuildBuildSystem llvmSupport)
diff --git a/unittests/BuildSystem/LaneBasedExecutionQueueTest.cpp b/unittests/BuildSystem/LaneBasedExecutionQueueTest.cpp
new file mode 100644
index 0000000..25cff93
--- /dev/null
+++ b/unittests/BuildSystem/LaneBasedExecutionQueueTest.cpp
@@ -0,0 +1,54 @@
+//===- unittests/BuildSystem/LaneBasedExecutionQueueTest.cpp --------------------------------===//
+//
+// This source file is part of the Swift.org open source project
+//
+// Copyright (c) 2014 - 2016 Apple Inc. and the Swift project authors
+// Licensed under Apache License v2.0 with Runtime Library Exception
+//
+// See http://swift.org/LICENSE.txt for license information
+// See http://swift.org/CONTRIBUTORS.txt for the list of Swift project authors
+//
+//===----------------------------------------------------------------------===//
+
+#include "llbuild/BuildSystem/BuildExecutionQueue.h"
+
+#include "llvm/ADT/ArrayRef.h"
+#include "llvm/ADT/SmallString.h"
+#include "llvm/ADT/StringRef.h"
+
+#include "gtest/gtest.h"
+
+using namespace llbuild;
+using namespace llbuild::buildsystem;
+
+namespace {
+  class DummyDelegate : public BuildExecutionQueueDelegate {
+  public:
+    DummyDelegate() {}
+
+    virtual void commandJobStarted(Command* command) override {}
+    virtual void commandJobFinished(Command* command) override {}
+    virtual void commandProcessStarted(Command* command, ProcessHandle handle) override {}
+    virtual void commandProcessHadError(Command* command, ProcessHandle handle, const Twine& message) override {}
+    virtual void commandProcessHadOutput(Command* command, ProcessHandle handle, StringRef data) override {}
+    virtual void commandProcessFinished(Command* command, ProcessHandle handle, int exitStatus) override {}
+  };
+
+  TEST(LaneBasedExecutionQueueTest, basic) {
+    DummyDelegate delegate;
+    auto queue = std::unique_ptr<BuildExecutionQueue>(createLaneBasedExecutionQueue(delegate, 2));
+
+    auto fn = [&queue](QueueJobContext* context) {
+      std::vector<StringRef> commandLine;
+      commandLine.push_back("/usr/bin/yes");
+      queue->executeProcess(context, commandLine);
+    };
+
+    queue->addJob(QueueJob((Command*)0x1, fn));
+    ::usleep(10); // there's a tiny race, until executeProcess() has called executeCommand()
+
+    queue->cancelAllJobs();
+    queue.reset();
+  }
+    
+}
diff --git a/unittests/CMakeLists.txt b/unittests/CMakeLists.txt
index aa3f724..419d4b4 100644
--- a/unittests/CMakeLists.txt
+++ b/unittests/CMakeLists.txt
@@ -7,4 +7,5 @@
 
 add_subdirectory(Basic)
 add_subdirectory(Core)
+add_subdirectory(BuildSystem)
 add_subdirectory(Ninja)