Merge pull request #187 from ddunbar/basic-tracing-support

Basic tracing support
diff --git a/include/llbuild/Basic/Tracing.h b/include/llbuild/Basic/Tracing.h
new file mode 100644
index 0000000..f344d3b
--- /dev/null
+++ b/include/llbuild/Basic/Tracing.h
@@ -0,0 +1,122 @@
+//===- Tracing.h ------------------------------------------------*- C++ -*-===//
+//
+// This source file is part of the Swift.org open source project
+//
+// Copyright (c) 2017 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
+//
+//===----------------------------------------------------------------------===//
+
+#ifndef LLBUILD_BASIC_TRACING_H
+#define LLBUILD_BASIC_TRACING_H
+
+#include "llvm/ADT/StringRef.h"
+
+namespace llbuild {
+
+/// Tracing Kind Codes
+///
+/// These are currently global across the entire library, please take care to
+/// not unnecessarily cause them to reorder as it will prevent use of prior
+/// tracing data.
+enum class TraceEventKind {
+  // Execution Queue
+    
+  /// An individual job execution interval.
+  ExecutionQueueJob = 0,
+      
+  /// A subprocess launch.
+  ExecutionQueueSubprocess = 1,
+
+  /// A callback from the task, \see EngineTaskCallbackKind.
+  EngineTaskCallback = 2,
+
+  /// An event on the engine processing queue, \see EngineQueueItemKind.
+  EngineQueueItemEvent = 3,
+
+  /// A point event to track the depth of the execution queue
+  ExecutionQueueDepth = 4,
+};
+
+// Engine Task Callbacks
+enum class EngineTaskCallbackKind {
+  Start = 0,
+  ProvidePriorValue,
+  ProvideValue,
+  InputsAvailable,
+};
+
+// Engine Queue Processing
+enum class EngineQueueItemKind {
+  RuleToScan = 0,
+  InputRequest,
+  FinishedInputRequest,
+  ReadyTask,
+  FinishedTask,
+  Waiting,
+};
+
+/// An RAII type to define an individual tracing point.
+struct TracingPoint {
+  const uint32_t kind;
+  const uint32_t arg1;
+  const uint32_t arg2;
+  const uint32_t arg3;
+  const uint32_t arg4;
+
+  TracingPoint(TraceEventKind kind, uint32_t arg1 = 0, uint32_t arg2 = 0,
+                uint32_t arg3 = 0, uint32_t arg4 = 0)
+    : kind(uint32_t(kind)), arg1(arg1), arg2(arg2), arg3(arg3), arg4(arg4)
+  {
+  }
+};
+
+/// An RAII type to define an individual tracing interval.
+///
+/// The client may modify the values of the arguments after initialization, for
+/// example to submit additional metrics for the event kind as part of the
+/// interval completion event.
+struct TracingInterval {
+  const uint32_t kind;
+  uint32_t arg1;
+  uint32_t arg2;
+  uint32_t arg3;
+  uint32_t arg4;
+
+  TracingInterval(TraceEventKind kind, uint32_t arg1 = 0, uint32_t arg2 = 0,
+                  uint32_t arg3 = 0, uint32_t arg4 = 0)
+      : kind(uint32_t(kind)), arg1(arg1), arg2(arg2), arg3(arg3), arg4(arg4)
+  {
+  }
+  ~TracingInterval() {
+  }
+
+  // MARK: Utility Wrappers
+  
+  TracingInterval(EngineTaskCallbackKind arg1)
+    : TracingInterval(TraceEventKind::EngineTaskCallback, uint32_t(arg1)) {}
+
+  TracingInterval(EngineQueueItemKind arg1)
+    : TracingInterval(TraceEventKind::EngineQueueItemEvent, uint32_t(arg1)) {}
+};
+
+/// An RAII type to define a string.
+struct TracingString {
+  const uint32_t kind;
+
+  /// The integer code for the string, which can be provided to a trace point or
+  /// interval.
+  const uint32_t value;
+
+  TracingString(TraceEventKind kind, llvm::StringRef str)
+      : kind(uint32_t(kind)), value(0) {}
+
+  operator uint32_t() const { return value; }
+};
+  
+}
+
+#endif
diff --git a/lib/BuildSystem/LaneBasedExecutionQueue.cpp b/lib/BuildSystem/LaneBasedExecutionQueue.cpp
index f154b45..50197a8 100644
--- a/lib/BuildSystem/LaneBasedExecutionQueue.cpp
+++ b/lib/BuildSystem/LaneBasedExecutionQueue.cpp
@@ -15,6 +15,9 @@
 
 #include "llbuild/Basic/LLVM.h"
 #include "llbuild/Basic/PlatformUtility.h"
+#include "llbuild/Basic/Tracing.h"
+
+#include "llbuild/BuildSystem/BuildDescription.h"
 
 #include "llvm/ADT/ArrayRef.h"
 #include "llvm/ADT/Hashing.h"
@@ -39,6 +42,7 @@
 #include <unistd.h>
 #include <signal.h>
 #include <spawn.h>
+#include <sys/resource.h>
 #include <sys/wait.h>
 
 using namespace llbuild;
@@ -55,6 +59,8 @@
 namespace {
 
 struct LaneBasedExecutionQueueJobContext {
+  uint32_t laneNumber;
+  
   QueueJob& job;
 };
 
@@ -110,6 +116,7 @@
     while (true) {
       // Take a job from the ready queue.
       QueueJob job{};
+      uint64_t readyJobsCount;
       {
         std::unique_lock<std::mutex> lock(readyJobsMutex);
 
@@ -123,6 +130,7 @@
         // Take an item according to the chosen policy.
         job = readyJobs.front();
         readyJobs.pop_front();
+        readyJobsCount = readyJobs.size();
       }
 
       // If we got an empty job, the queue is shutting down.
@@ -130,10 +138,18 @@
         break;
 
       // Process the job.
-      LaneBasedExecutionQueueJobContext context{ job };
-      getDelegate().commandJobStarted(job.getForCommand());
-      job.execute(reinterpret_cast<QueueJobContext*>(&context));
-      getDelegate().commandJobFinished(job.getForCommand());
+      LaneBasedExecutionQueueJobContext context{ laneNumber, job };
+      {
+        TracingPoint(TraceEventKind::ExecutionQueueDepth, readyJobsCount);
+        TracingString commandNameID(
+            TraceEventKind::ExecutionQueueJob,
+            job.getForCommand()->getName());
+        TracingInterval i(TraceEventKind::ExecutionQueueJob,
+                          context.laneNumber, commandNameID);
+        getDelegate().commandJobStarted(job.getForCommand());
+        job.execute(reinterpret_cast<QueueJobContext*>(&context));
+        getDelegate().commandJobFinished(job.getForCommand());
+      }
     }
   }
 
@@ -199,9 +215,14 @@
   }
 
   virtual void addJob(QueueJob job) override {
-    std::lock_guard<std::mutex> guard(readyJobsMutex);
-    readyJobs.push_back(job);
-    readyJobsCondition.notify_one();
+    uint64_t readyJobsCount;
+    {
+      std::lock_guard<std::mutex> guard(readyJobsMutex);
+      readyJobs.push_back(job);
+      readyJobsCondition.notify_one();
+      readyJobsCount = readyJobs.size();
+    }
+    TracingPoint(TraceEventKind::ExecutionQueueDepth, readyJobsCount);
   }
 
   virtual void cancelAllJobs() override {
@@ -224,6 +245,11 @@
                  ArrayRef<std::pair<StringRef,
                                     StringRef>> environment,
                  bool inheritEnvironment) override {
+    LaneBasedExecutionQueueJobContext& context =
+      *reinterpret_cast<LaneBasedExecutionQueueJobContext*>(opaqueContext);
+    TracingInterval subprocessInterval(TraceEventKind::ExecutionQueueSubprocess,
+                                       context.laneNumber);
+
     {
       std::unique_lock<std::mutex> lock(readyJobsMutex);
       // Do not execute new processes anymore after cancellation.
@@ -240,8 +266,6 @@
     // Whether or not we are capturing output.
     const bool shouldCaptureOutput = true;
 
-    LaneBasedExecutionQueueJobContext& context =
-      *reinterpret_cast<LaneBasedExecutionQueueJobContext*>(opaqueContext);
     getDelegate().commandProcessStarted(context.job.getForCommand(), handle);
     
     // Initialize the spawn attributes.
@@ -459,9 +483,10 @@
     }
     
     // Wait for the command to complete.
-    int status, result = waitpid(pid, &status, 0);
+    struct rusage usage;
+    int status, result = wait4(pid, &status, 0, &usage);
     while (result == -1 && errno == EINTR)
-      result = waitpid(pid, &status, 0);
+      result = wait4(pid, &status, 0, &usage);
 
     // Update the set of spawned processes.
     {
@@ -477,6 +502,19 @@
                                            CommandResult::Failed, -1);
       return CommandResult::Failed;
     }
+
+    // We report additional info in the tracing interval
+    //   arg2: user time, in us
+    //   arg3: sys time, in us
+    //   arg4: memory usage, in bytes
+    subprocessInterval.arg2 = (uint64_t(usage.ru_utime.tv_sec) * 1000000000 +
+                               uint64_t(usage.ru_utime.tv_usec) * 1000);
+    subprocessInterval.arg3 = (uint64_t(usage.ru_stime.tv_sec) * 1000000000 +
+                               uint64_t(usage.ru_stime.tv_usec) * 1000);
+    subprocessInterval.arg4 = usage.ru_maxrss;
+    
+    // FIXME: We should report a statistic for how much output we read from the
+    // subprocess (probably as a new point sample).
     
     // Notify of the process completion.
     bool cancelled = WIFSIGNALED(status) && (WTERMSIG(status) == SIGINT || WTERMSIG(status) == SIGKILL);
diff --git a/lib/Core/BuildEngine.cpp b/lib/Core/BuildEngine.cpp
index 3b34a3c..7067f46 100644
--- a/lib/Core/BuildEngine.cpp
+++ b/lib/Core/BuildEngine.cpp
@@ -12,6 +12,7 @@
 
 #include "llbuild/Core/BuildEngine.h"
 
+#include "llbuild/Basic/Tracing.h"
 #include "llbuild/Core/BuildDB.h"
 
 #include "llvm/ADT/STLExtras.h"
@@ -483,7 +484,10 @@
     ruleInfo.result.dependencies.clear();
 
     // Inform the task it should start.
-    task->start(buildEngine);
+    {
+      TracingInterval i(EngineTaskCallbackKind::Start);
+      task->start(buildEngine);
+    }
 
     // Provide the task the prior result, if present.
     //
@@ -492,6 +496,7 @@
     // the clients that want it can ask? It's cheap to provide here, so
     // ultimately this is mostly a matter of cleanliness.
     if (ruleInfo.result.builtAt != 0) {
+      TracingInterval i(EngineTaskCallbackKind::ProvidePriorValue);
       task->providePriorValue(buildEngine, ruleInfo.result.value);
     }
 
@@ -631,6 +636,8 @@
       // FIXME: We don't want to process all of these requests, this amounts to
       // doing all of the dependency scanning up-front.
       while (!ruleInfosToScan.empty()) {
+        TracingInterval i(EngineQueueItemKind::RuleToScan);
+        
         didWork = true;
 
         auto request = ruleInfosToScan.back();
@@ -641,6 +648,8 @@
 
       // Process all of the pending input requests.
       while (!inputRequests.empty()) {
+        TracingInterval i(EngineQueueItemKind::InputRequest);
+        
         didWork = true;
 
         auto request = inputRequests.back();
@@ -695,6 +704,8 @@
 
       // Process all of the finished inputs.
       while (!finishedInputRequests.empty()) {
+        TracingInterval i(EngineQueueItemKind::FinishedInputRequest);
+        
         didWork = true;
 
         auto request = finishedInputRequests.back();
@@ -736,8 +747,11 @@
         // FIXME: Should we provide the input key here? We have it available
         // cheaply.
         assert(request.inputRuleInfo->isComplete(this));
-        request.taskInfo->task->provideValue(
-          buildEngine, request.inputID, request.inputRuleInfo->result.value);
+        {
+          TracingInterval i(EngineTaskCallbackKind::ProvideValue);
+          request.taskInfo->task->provideValue(
+              buildEngine, request.inputID, request.inputRuleInfo->result.value);
+        }
 
         // Decrement the wait count, and move to finish queue if necessary.
         decrementTaskWaitCount(request.taskInfo);
@@ -745,6 +759,8 @@
 
       // Process all of the ready to run tasks.
       while (!readyTaskInfos.empty()) {
+        TracingInterval i(EngineQueueItemKind::ReadyTask);
+        
         didWork = true;
 
         TaskInfo* taskInfo = readyTaskInfos.back();
@@ -764,7 +780,10 @@
         //
         // FIXME: We need to track this state, and generate an error if this
         // task ever requests additional inputs.
-        taskInfo->task->inputsAvailable(buildEngine);
+        {
+          TracingInterval i(EngineTaskCallbackKind::InputsAvailable);
+          taskInfo->task->inputsAvailable(buildEngine);
+        }
 
         // Increment our count of outstanding tasks.
         ++numOutstandingUnfinishedTasks;
@@ -772,6 +791,8 @@
 
       // Process all of the finished tasks.
       while (true) {
+        TracingInterval i(EngineQueueItemKind::FinishedTask);
+        
         // Try to take a task from the finished queue.
         TaskInfo* taskInfo = nullptr;
         {
@@ -871,6 +892,8 @@
       // If we haven't done any other work at this point but we have pending
       // tasks, we need to wait for a task to complete.
       if (!didWork && numOutstandingUnfinishedTasks != 0) {
+        TracingInterval i(EngineQueueItemKind::Waiting);
+        
         // Wait for our condition variable.
         std::unique_lock<std::mutex> lock(finishedTaskInfosMutex);
 
diff --git a/unittests/BuildSystem/LaneBasedExecutionQueueTest.cpp b/unittests/BuildSystem/LaneBasedExecutionQueueTest.cpp
index f707b37..4dcd155 100644
--- a/unittests/BuildSystem/LaneBasedExecutionQueueTest.cpp
+++ b/unittests/BuildSystem/LaneBasedExecutionQueueTest.cpp
@@ -1,4 +1,4 @@
-//===- unittests/BuildSystem/LaneBasedExecutionQueueTest.cpp --------------------------------===//
+//===- unittests/BuildSystem/LaneBasedExecutionQueueTest.cpp --------------===//
 //
 // This source file is part of the Swift.org open source project
 //
@@ -14,6 +14,9 @@
 #include "llbuild/BuildSystem/BuildExecutionQueue.h"
 #include "TempDir.h"
 
+#include "llbuild/BuildSystem/BuildDescription.h"
+#include "llbuild/BuildSystem/BuildValue.h"
+
 #include "llvm/ADT/ArrayRef.h"
 #include "llvm/ADT/SmallString.h"
 #include "llvm/ADT/StringRef.h"
@@ -48,6 +51,49 @@
                                         int exitStatus) override {}
   };
 
+  class DummyCommand : public Command {
+  public:
+    DummyCommand() : Command("") {}
+
+    virtual void getShortDescription(SmallVectorImpl<char> &result) {}
+    virtual void getVerboseDescription(SmallVectorImpl<char> &result) {}
+    virtual void configureDescription(const ConfigureContext&,
+                                      StringRef description) {};
+    virtual void configureInputs(const ConfigureContext&,
+                                 const std::vector<Node*>& inputs) {}
+    virtual void configureOutputs(const ConfigureContext&,
+                                const std::vector<Node*>& outputs) {}
+    virtual bool configureAttribute(const ConfigureContext&, StringRef name,
+                                    StringRef value) {
+      return false;
+    }
+    virtual bool configureAttribute(const ConfigureContext&, StringRef name,
+                                    ArrayRef<StringRef> values) {
+      return false;
+    }
+    virtual bool configureAttribute(
+        const ConfigureContext&, StringRef name,
+        ArrayRef<std::pair<StringRef, StringRef>> values) {
+      return false;
+    }
+    virtual BuildValue getResultForOutput(Node* node,
+                                          const BuildValue& value) {
+      return BuildValue::makeInvalid();
+    }
+    virtual bool isResultValid(BuildSystem& system, const BuildValue& value) {
+      return true;
+    }
+    virtual void start(BuildSystemCommandInterface&, core::Task*) {}
+    virtual void providePriorValue(BuildSystemCommandInterface&, core::Task*,
+                                   const BuildValue& value) {}
+    virtual void provideValue(BuildSystemCommandInterface&, core::Task*,
+                              uintptr_t inputID, const BuildValue& value) {}
+    virtual BuildValue execute(BuildSystemCommandInterface&, core::Task*,
+                               QueueJobContext* context) {
+      return BuildValue::makeInvalid();
+    }
+  };
+
   TEST(LaneBasedExecutionQueueTest, basic) {
     DummyDelegate delegate;
     std::unique_ptr<FileSystem> fs = createLocalFileSystem();
@@ -60,7 +106,8 @@
       queue->executeShellCommand(context, "yes >" + outputFile);
     };
 
-    queue->addJob(QueueJob((Command*)0x1, fn));
+    DummyCommand dummyCommand;
+    queue->addJob(QueueJob(&dummyCommand, fn));
 
     // Busy wait until `outputFile` appears which indicates that `yes` is
     // running.
@@ -97,8 +144,10 @@
       buildStartedCondition.notify_all();
     };
 
-    queue->addJob(QueueJob((Command*)0x1, fn));
-    queue->addJob(QueueJob((Command*)0x1, fn));
+    DummyCommand dummyCommand1;
+    queue->addJob(QueueJob(&dummyCommand1, fn));
+    DummyCommand dummyCommand2;
+    queue->addJob(QueueJob(&dummyCommand2, fn));
 
     {
       std::unique_lock<std::mutex> lock(buildStartedMutex);