[llvm] r341906 - [XRay] Use FDR Records+Visitors for Trace Loading

Dean Michael Berris via llvm-commits llvm-commits at lists.llvm.org
Mon Sep 10 23:45:59 PDT 2018


Author: dberris
Date: Mon Sep 10 23:45:59 2018
New Revision: 341906

URL: http://llvm.org/viewvc/llvm-project?rev=341906&view=rev
Log:
[XRay] Use FDR Records+Visitors for Trace Loading

Summary:
In this change, we overhaul the implementation for loading
`llvm::xray::Trace` objects from files by using the combination of
specific FDR Record types and visitors breaking up the logic to
reconstitute an execution trace from flight-data recorder mode traces.

This change allows us to handle out-of-temporal order blocks as written
in files, and more consistently recreate an execution trace spanning
multiple blocks and threads. To do this, we use the `WallclockRecord`
associated with each block to maintain temporal order of blocks, before
attempting to recreate an execution trace.

The new addition in this change is the `TraceExpander` type which can be
thought of as a decompression/decoding routine. This allows us to
maintain the state of an execution environment (thread+process) and
create `XRayRecord` instances that fit nicely into the `Trace`
container. We don't have a specific unit test for the TraceExpander
type, since the end-to-end tests for the `llvm-xray convert` tools
already cover precisely this codepath.

This change completes the refactoring started with D50441.

Depends on D51911.

Reviewers: mboerger, eizan

Subscribers: mgorny, hiraditya, mgrang, llvm-commits

Differential Revision: https://reviews.llvm.org/D51912

Added:
    llvm/trunk/include/llvm/XRay/FDRTraceExpander.h
    llvm/trunk/lib/XRay/FDRTraceExpander.cpp
Modified:
    llvm/trunk/include/llvm/XRay/BlockIndexer.h
    llvm/trunk/lib/XRay/BlockIndexer.cpp
    llvm/trunk/lib/XRay/CMakeLists.txt
    llvm/trunk/lib/XRay/Trace.cpp

Modified: llvm/trunk/include/llvm/XRay/BlockIndexer.h
URL: http://llvm.org/viewvc/llvm-project/llvm/trunk/include/llvm/XRay/BlockIndexer.h?rev=341906&r1=341905&r2=341906&view=diff
==============================================================================
--- llvm/trunk/include/llvm/XRay/BlockIndexer.h (original)
+++ llvm/trunk/include/llvm/XRay/BlockIndexer.h Mon Sep 10 23:45:59 2018
@@ -29,6 +29,7 @@ public:
   struct Block {
     uint64_t ProcessID;
     int32_t ThreadID;
+    WallclockRecord *WallclockTime;
     std::vector<Record *> Records;
   };
 
@@ -41,7 +42,7 @@ private:
   enum class State : unsigned { SeekExtents, ExtentsFound, ThreadIDFound };
 
   State CurrentState = State::SeekExtents;
-  Block CurrentBlock{0, 0, {}};
+  Block CurrentBlock{0, 0, nullptr, {}};
 
 public:
   explicit BlockIndexer(Index &I) : RecordVisitor(), Indices(I) {}

Added: llvm/trunk/include/llvm/XRay/FDRTraceExpander.h
URL: http://llvm.org/viewvc/llvm-project/llvm/trunk/include/llvm/XRay/FDRTraceExpander.h?rev=341906&view=auto
==============================================================================
--- llvm/trunk/include/llvm/XRay/FDRTraceExpander.h (added)
+++ llvm/trunk/include/llvm/XRay/FDRTraceExpander.h Mon Sep 10 23:45:59 2018
@@ -0,0 +1,61 @@
+//===- FDRTraceExpander.h - XRay FDR Mode Log Expander --------------------===//
+//
+//                     The LLVM Compiler Infrastructure
+//
+// This file is distributed under the University of Illinois Open Source
+// License. See LICENSE.TXT for details.
+//
+//===----------------------------------------------------------------------===//
+//
+// We define an FDR record visitor which can re-constitute XRayRecord instances
+// from a sequence of FDR mode records in arrival order into a collection.
+//
+//===----------------------------------------------------------------------===//
+#ifndef INCLUDE_LLVM_XRAY_FDRTRACEEXPANDER_H_
+#define INCLUDE_LLVM_XRAY_FDRTRACEEXPANDER_H_
+
+#include "llvm/ADT/STLExtras.h"
+#include "llvm/XRay/FDRRecords.h"
+#include "llvm/XRay/XRayRecord.h"
+
+namespace llvm {
+namespace xray {
+
+class TraceExpander : public RecordVisitor {
+  // Type-erased callback for handling individual XRayRecord instances.
+  function_ref<void(const XRayRecord &)> C;
+  int32_t PID = 0;
+  int32_t TID = 0;
+  uint64_t BaseTSC = 0;
+  XRayRecord CurrentRecord{};
+  uint16_t CPUId = 0;
+  uint16_t LogVersion = 0;
+  bool BuildingFunction = false;
+  bool IgnoringRecords = false;
+
+  void resetCurrentRecord();
+
+public:
+  explicit TraceExpander(function_ref<void(const XRayRecord &)> F, uint16_t L)
+      : RecordVisitor(), C(std::move(F)), LogVersion(L) {}
+
+  Error visit(BufferExtents &) override;
+  Error visit(WallclockRecord &) override;
+  Error visit(NewCPUIDRecord &) override;
+  Error visit(TSCWrapRecord &) override;
+  Error visit(CustomEventRecord &) override;
+  Error visit(CallArgRecord &) override;
+  Error visit(PIDRecord &) override;
+  Error visit(NewBufferRecord &) override;
+  Error visit(EndBufferRecord &) override;
+  Error visit(FunctionRecord &) override;
+
+  // Must be called after all the records have been processed, to handle the
+  // most recent record generated.
+  Error flush();
+};
+
+} // namespace xray
+} // namespace llvm
+
+#endif // INCLUDE_LLVM_XRAY_FDRTRACEEXPANDER_H_

Modified: llvm/trunk/lib/XRay/BlockIndexer.cpp
URL: http://llvm.org/viewvc/llvm-project/llvm/trunk/lib/XRay/BlockIndexer.cpp?rev=341906&r1=341905&r2=341906&view=diff
==============================================================================
--- llvm/trunk/lib/XRay/BlockIndexer.cpp (original)
+++ llvm/trunk/lib/XRay/BlockIndexer.cpp Mon Sep 10 23:45:59 2018
@@ -22,9 +22,11 @@ Error BlockIndexer::visit(BufferExtents
     std::tie(It, std::ignore) =
         Indices.insert({{CurrentBlock.ProcessID, CurrentBlock.ThreadID}, {}});
     It->second.push_back({CurrentBlock.ProcessID, CurrentBlock.ThreadID,
+                          CurrentBlock.WallclockTime,
                           std::move(CurrentBlock.Records)});
     CurrentBlock.ProcessID = 0;
     CurrentBlock.ThreadID = 0;
+    CurrentBlock.WallclockTime = nullptr;
     CurrentBlock.Records = {};
   }
   CurrentState = State::ExtentsFound;
@@ -33,6 +35,7 @@ Error BlockIndexer::visit(BufferExtents
 
 Error BlockIndexer::visit(WallclockRecord &R) {
   CurrentBlock.Records.push_back(&R);
+  CurrentBlock.WallclockTime = &R;
   return Error::success();
 }
 
@@ -86,6 +89,7 @@ Error BlockIndexer::flush() {
   std::tie(It, std::ignore) =
       Indices.insert({{CurrentBlock.ProcessID, CurrentBlock.ThreadID}, {}});
   It->second.push_back({CurrentBlock.ProcessID, CurrentBlock.ThreadID,
+                        CurrentBlock.WallclockTime,
                         std::move(CurrentBlock.Records)});
   CurrentBlock.ProcessID = 0;
   CurrentBlock.ThreadID = 0;

Modified: llvm/trunk/lib/XRay/CMakeLists.txt
URL: http://llvm.org/viewvc/llvm-project/llvm/trunk/lib/XRay/CMakeLists.txt?rev=341906&r1=341905&r2=341906&view=diff
==============================================================================
--- llvm/trunk/lib/XRay/CMakeLists.txt (original)
+++ llvm/trunk/lib/XRay/CMakeLists.txt Mon Sep 10 23:45:59 2018
@@ -4,6 +4,7 @@ add_llvm_library(LLVMXRay
   BlockVerifier.cpp
   FDRRecordProducer.cpp
   FDRRecords.cpp
+  FDRTraceExpander.cpp
   FDRTraceWriter.cpp
   FileHeaderReader.cpp
   InstrumentationMap.cpp

Added: llvm/trunk/lib/XRay/FDRTraceExpander.cpp
URL: http://llvm.org/viewvc/llvm-project/llvm/trunk/lib/XRay/FDRTraceExpander.cpp?rev=341906&view=auto
==============================================================================
--- llvm/trunk/lib/XRay/FDRTraceExpander.cpp (added)
+++ llvm/trunk/lib/XRay/FDRTraceExpander.cpp Mon Sep 10 23:45:59 2018
@@ -0,0 +1,92 @@
+//===- FDRTraceExpander.cpp -----------------------------------------------===//
+//
+//                     The LLVM Compiler Infrastructure
+//
+// This file is distributed under the University of Illinois Open Source
+// License. See LICENSE.TXT for details.
+//
+//===----------------------------------------------------------------------===//
+#include "llvm/XRay/FDRTraceExpander.h"
+
+namespace llvm {
+namespace xray {
+
+void TraceExpander::resetCurrentRecord() {
+  if (BuildingFunction)
+    C(CurrentRecord);
+  BuildingFunction = false;
+  CurrentRecord.CallArgs.clear();
+}
+
+Error TraceExpander::visit(BufferExtents &) {
+  resetCurrentRecord();
+  return Error::success();
+}
+
+Error TraceExpander::visit(WallclockRecord &) { return Error::success(); }
+
+Error TraceExpander::visit(NewCPUIDRecord &R) {
+  CPUId = R.cpuid();
+  BaseTSC = R.tsc();
+  return Error::success();
+}
+
+Error TraceExpander::visit(TSCWrapRecord &R) {
+  BaseTSC = R.tsc();
+  return Error::success();
+}
+
+Error TraceExpander::visit(CustomEventRecord &) {
+  // TODO: Support custom event records in the future.
+  resetCurrentRecord();
+  return Error::success();
+}
+
+Error TraceExpander::visit(CallArgRecord &R) {
+  CurrentRecord.CallArgs.push_back(R.arg());
+  CurrentRecord.Type = RecordTypes::ENTER_ARG;
+  return Error::success();
+}
+
+Error TraceExpander::visit(PIDRecord &R) {
+  PID = R.pid();
+  return Error::success();
+}
+
+Error TraceExpander::visit(NewBufferRecord &R) {
+  if (IgnoringRecords)
+    IgnoringRecords = false;
+  TID = R.tid();
+  if (LogVersion == 2)
+    PID = R.tid();
+  return Error::success();
+}
+
+Error TraceExpander::visit(EndBufferRecord &) {
+  IgnoringRecords = true;
+  resetCurrentRecord();
+  return Error::success();
+}
+
+Error TraceExpander::visit(FunctionRecord &R) {
+  resetCurrentRecord();
+  if (!IgnoringRecords) {
+    BaseTSC += R.delta();
+    CurrentRecord.Type = R.recordType();
+    CurrentRecord.FuncId = R.functionId();
+    CurrentRecord.TSC = BaseTSC;
+    CurrentRecord.PId = PID;
+    CurrentRecord.TId = TID;
+    CurrentRecord.CPU = CPUId;
+    BuildingFunction = true;
+  }
+  return Error::success();
+}
+
+Error TraceExpander::flush() {
+  resetCurrentRecord();
+  return Error::success();
+}
+
+} // namespace xray
+} // namespace llvm

Modified: llvm/trunk/lib/XRay/Trace.cpp
URL: http://llvm.org/viewvc/llvm-project/llvm/trunk/lib/XRay/Trace.cpp?rev=341906&r1=341905&r2=341906&view=diff
==============================================================================
--- llvm/trunk/lib/XRay/Trace.cpp (original)
+++ llvm/trunk/lib/XRay/Trace.cpp Mon Sep 10 23:45:59 2018
@@ -15,8 +15,16 @@
 #include "llvm/Support/DataExtractor.h"
 #include "llvm/Support/Error.h"
 #include "llvm/Support/FileSystem.h"
+#include "llvm/XRay/BlockIndexer.h"
+#include "llvm/XRay/BlockVerifier.h"
+#include "llvm/XRay/FDRRecordConsumer.h"
+#include "llvm/XRay/FDRRecordProducer.h"
+#include "llvm/XRay/FDRRecords.h"
+#include "llvm/XRay/FDRTraceExpander.h"
 #include "llvm/XRay/FileHeaderReader.h"
 #include "llvm/XRay/YAMLXRayRecord.h"
+#include <memory>
+#include <vector>
 
 using namespace llvm;
 using namespace llvm::xray;
@@ -26,11 +34,6 @@ namespace {
 using XRayRecordStorage =
     std::aligned_storage<sizeof(XRayRecord), alignof(XRayRecord)>::type;
 
-// This is the number of bytes in the "body" of a MetadataRecord in FDR Mode.
-// This already excludes the first byte, which indicates the type of metadata
-// record it is.
-constexpr auto kFDRMetadataBodySize = 15;
-
 Error loadNaiveFormatLog(StringRef Data, bool IsLittleEndian,
                          XRayFileHeader &FileHeader,
                          std::vector<XRayRecord> &Records) {
@@ -201,467 +204,6 @@ Error loadNaiveFormatLog(StringRef Data,
   return Error::success();
 }
 
-/// When reading from a Flight Data Recorder mode log, metadata records are
-/// sparse compared to packed function records, so we must maintain state as we
-/// read through the sequence of entries. This allows the reader to denormalize
-/// the CPUId and Thread Id onto each Function Record and transform delta
-/// encoded TSC values into absolute encodings on each record.
-struct FDRState {
-  uint16_t CPUId;
-  int32_t ThreadId;
-  int32_t ProcessId;
-  uint64_t BaseTSC;
-
-  /// Encode some of the state transitions for the FDR log reader as explicit
-  /// checks. These are expectations for the next Record in the stream.
-  enum class Token {
-    NEW_BUFFER_RECORD_OR_EOF,
-    WALLCLOCK_RECORD,
-    NEW_CPU_ID_RECORD,
-    FUNCTION_SEQUENCE,
-    SCAN_TO_END_OF_THREAD_BUF,
-    CUSTOM_EVENT_DATA,
-    CALL_ARGUMENT,
-    BUFFER_EXTENTS,
-    PID_RECORD,
-  };
-  Token Expects;
-
-  // Each threads buffer may have trailing garbage to scan over, so we track our
-  // progress.
-  uint64_t CurrentBufferSize;
-  uint64_t CurrentBufferConsumed;
-};
-
-const char *fdrStateToTwine(const FDRState::Token &state) {
-  switch (state) {
-  case FDRState::Token::NEW_BUFFER_RECORD_OR_EOF:
-    return "NEW_BUFFER_RECORD_OR_EOF";
-  case FDRState::Token::WALLCLOCK_RECORD:
-    return "WALLCLOCK_RECORD";
-  case FDRState::Token::NEW_CPU_ID_RECORD:
-    return "NEW_CPU_ID_RECORD";
-  case FDRState::Token::FUNCTION_SEQUENCE:
-    return "FUNCTION_SEQUENCE";
-  case FDRState::Token::SCAN_TO_END_OF_THREAD_BUF:
-    return "SCAN_TO_END_OF_THREAD_BUF";
-  case FDRState::Token::CUSTOM_EVENT_DATA:
-    return "CUSTOM_EVENT_DATA";
-  case FDRState::Token::CALL_ARGUMENT:
-    return "CALL_ARGUMENT";
-  case FDRState::Token::BUFFER_EXTENTS:
-    return "BUFFER_EXTENTS";
-  case FDRState::Token::PID_RECORD:
-    return "PID_RECORD";
-  }
-  return "UNKNOWN";
-}
-
-/// State transition when a NewBufferRecord is encountered.
-Error processFDRNewBufferRecord(FDRState &State, DataExtractor &RecordExtractor,
-                                uint32_t &OffsetPtr) {
-  if (State.Expects != FDRState::Token::NEW_BUFFER_RECORD_OR_EOF)
-    return createStringError(
-        std::make_error_code(std::errc::executable_format_error),
-        "Malformed log: Read New Buffer record kind out of sequence; expected: "
-        "%s at offset %d.",
-        fdrStateToTwine(State.Expects), OffsetPtr);
-
-  auto PreReadOffset = OffsetPtr;
-  State.ThreadId = RecordExtractor.getSigned(&OffsetPtr, 4);
-  if (OffsetPtr == PreReadOffset)
-    return createStringError(
-        std::make_error_code(std::errc::executable_format_error),
-        "Failed reading the thread id at offset %d.", OffsetPtr);
-  State.Expects = FDRState::Token::WALLCLOCK_RECORD;
-
-  // Advance the offset pointer by enough bytes representing the remaining
-  // padding in a metadata record.
-  OffsetPtr += kFDRMetadataBodySize - 4;
-  assert(OffsetPtr - PreReadOffset == kFDRMetadataBodySize);
-  return Error::success();
-}
-
-/// State transition when an EndOfBufferRecord is encountered.
-Error processFDREndOfBufferRecord(FDRState &State, uint32_t &OffsetPtr) {
-  if (State.Expects == FDRState::Token::NEW_BUFFER_RECORD_OR_EOF)
-    return createStringError(
-        std::make_error_code(std::errc::executable_format_error),
-        "Malformed log: Received EOB message without current buffer; expected: "
-        "%s at offset %d.",
-        fdrStateToTwine(State.Expects), OffsetPtr);
-
-  State.Expects = FDRState::Token::SCAN_TO_END_OF_THREAD_BUF;
-
-  // Advance the offset pointer by enough bytes representing the remaining
-  // padding in a metadata record.
-  OffsetPtr += kFDRMetadataBodySize;
-  return Error::success();
-}
-
-/// State transition when a NewCPUIdRecord is encountered.
-Error processFDRNewCPUIdRecord(FDRState &State, DataExtractor &RecordExtractor,
-                               uint32_t &OffsetPtr) {
-  if (State.Expects != FDRState::Token::FUNCTION_SEQUENCE &&
-      State.Expects != FDRState::Token::NEW_CPU_ID_RECORD)
-    return make_error<StringError>(
-        Twine("Malformed log. Read NewCPUId record kind out of sequence; "
-              "expected: ") +
-            fdrStateToTwine(State.Expects),
-        std::make_error_code(std::errc::executable_format_error));
-  auto BeginOffset = OffsetPtr;
-  auto PreReadOffset = OffsetPtr;
-  State.CPUId = RecordExtractor.getU16(&OffsetPtr);
-  if (OffsetPtr == PreReadOffset)
-    return createStringError(
-        std::make_error_code(std::errc::executable_format_error),
-        "Failed reading the CPU field at offset %d.", OffsetPtr);
-
-  PreReadOffset = OffsetPtr;
-  State.BaseTSC = RecordExtractor.getU64(&OffsetPtr);
-  if (OffsetPtr == PreReadOffset)
-    return createStringError(
-        std::make_error_code(std::errc::executable_format_error),
-        "Failed reading the base TSC field at offset %d.", OffsetPtr);
-
-  State.Expects = FDRState::Token::FUNCTION_SEQUENCE;
-
-  // Advance the offset pointer by a few bytes, to account for the padding in
-  // CPU ID metadata records that we've already advanced through.
-  OffsetPtr += kFDRMetadataBodySize - (OffsetPtr - BeginOffset);
-  assert(OffsetPtr - BeginOffset == kFDRMetadataBodySize);
-  return Error::success();
-}
-
-/// State transition when a TSCWrapRecord (overflow detection) is encountered.
-Error processFDRTSCWrapRecord(FDRState &State, DataExtractor &RecordExtractor,
-                              uint32_t &OffsetPtr) {
-  if (State.Expects != FDRState::Token::FUNCTION_SEQUENCE)
-    return make_error<StringError>(
-        Twine("Malformed log. Read TSCWrap record kind out of sequence; "
-              "expecting: ") +
-            fdrStateToTwine(State.Expects),
-        std::make_error_code(std::errc::executable_format_error));
-  auto PreReadOffset = OffsetPtr;
-  State.BaseTSC = RecordExtractor.getU64(&OffsetPtr);
-  if (OffsetPtr == PreReadOffset)
-    return createStringError(
-        std::make_error_code(std::errc::executable_format_error),
-        "Failed reading the base TSC field at offset %d.", OffsetPtr);
-
-  // Advance the offset pointer by a few more bytes, accounting for the padding
-  // in the metadata record after reading the base TSC.
-  OffsetPtr += kFDRMetadataBodySize - 8;
-  assert(OffsetPtr - PreReadOffset == kFDRMetadataBodySize);
-  return Error::success();
-}
-
-/// State transition when a WallTimeMarkerRecord is encountered.
-Error processFDRWallTimeRecord(FDRState &State, DataExtractor &RecordExtractor,
-                               uint32_t &OffsetPtr) {
-  if (State.Expects != FDRState::Token::WALLCLOCK_RECORD)
-    return make_error<StringError>(
-        Twine("Malformed log. Read Wallclock record kind out of sequence; "
-              "expecting: ") +
-            fdrStateToTwine(State.Expects),
-        std::make_error_code(std::errc::executable_format_error));
-
-  // Read in the data from the walltime record.
-  auto PreReadOffset = OffsetPtr;
-  auto WallTime = RecordExtractor.getU64(&OffsetPtr);
-  if (OffsetPtr == PreReadOffset)
-    return createStringError(
-        std::make_error_code(std::errc::executable_format_error),
-        "Failed reading the walltime record at offset %d.", OffsetPtr);
-
-  // TODO: Someday, reconcile the TSC ticks to wall clock time for presentation
-  // purposes. For now, we're ignoring these records.
-  (void)WallTime;
-  State.Expects = FDRState::Token::NEW_CPU_ID_RECORD;
-
-  // Advance the offset pointer by a few more bytes, accounting for the padding
-  // in the metadata record after reading in the walltime data.
-  OffsetPtr += kFDRMetadataBodySize - 8;
-  assert(OffsetPtr - PreReadOffset == kFDRMetadataBodySize);
-  return Error::success();
-}
-
-/// State transition when a PidRecord is encountered.
-Error processFDRPidRecord(FDRState &State, DataExtractor &RecordExtractor,
-                          uint32_t &OffsetPtr) {
-  if (State.Expects != FDRState::Token::PID_RECORD)
-    return make_error<StringError>(
-        Twine("Malformed log. Read Pid record kind out of sequence; "
-              "expected: ") +
-            fdrStateToTwine(State.Expects),
-        std::make_error_code(std::errc::executable_format_error));
-  auto PreReadOffset = OffsetPtr;
-  State.ProcessId = RecordExtractor.getSigned(&OffsetPtr, 4);
-  if (OffsetPtr == PreReadOffset)
-    return createStringError(
-        std::make_error_code(std::errc::executable_format_error),
-        "Failed reading the process ID at offset %d.", OffsetPtr);
-  State.Expects = FDRState::Token::NEW_CPU_ID_RECORD;
-
-  // Advance the offset pointer by a few more bytes, accounting for the padding
-  // in the metadata record after reading in the PID.
-  OffsetPtr += kFDRMetadataBodySize - 4;
-  assert(OffsetPtr - PreReadOffset == kFDRMetadataBodySize);
-  return Error::success();
-}
-
-/// State transition when a CustomEventMarker is encountered.
-Error processCustomEventMarker(FDRState &State, DataExtractor &RecordExtractor,
-                               uint32_t &OffsetPtr) {
-  // We can encounter a CustomEventMarker anywhere in the log, so we can handle
-  // it regardless of the expectation. However, we do set the expectation to
-  // read a set number of fixed bytes, as described in the metadata.
-  auto BeginOffset = OffsetPtr;
-  auto PreReadOffset = OffsetPtr;
-  uint32_t DataSize = RecordExtractor.getU32(&OffsetPtr);
-  if (OffsetPtr == PreReadOffset)
-    return createStringError(
-        std::make_error_code(std::errc::executable_format_error),
-        "Failed reading a custom event marker at offset %d.", OffsetPtr);
-
-  PreReadOffset = OffsetPtr;
-  uint64_t TSC = RecordExtractor.getU64(&OffsetPtr);
-  if (OffsetPtr == PreReadOffset)
-    return createStringError(
-        std::make_error_code(std::errc::executable_format_error),
-        "Failed reading the TSC at offset %d.", OffsetPtr);
-
-  // FIXME: Actually represent the record through the API. For now we only
-  // skip through the data.
-  (void)TSC;
-  // Advance the offset ptr by the size of the data associated with the custom
-  // event, as well as the padding associated with the remainder of the metadata
-  // record.
-  OffsetPtr += (kFDRMetadataBodySize - (OffsetPtr - BeginOffset)) + DataSize;
-  if (!RecordExtractor.isValidOffset(OffsetPtr))
-    return createStringError(
-        std::make_error_code(std::errc::executable_format_error),
-        "Reading custom event data moves past addressable trace data (starting "
-        "at offset %d, advancing to offset %d).",
-        BeginOffset, OffsetPtr);
-  return Error::success();
-}
-
-/// State transition when an BufferExtents record is encountered.
-Error processBufferExtents(FDRState &State, DataExtractor &RecordExtractor,
-                           uint32_t &OffsetPtr) {
-  if (State.Expects != FDRState::Token::BUFFER_EXTENTS)
-    return make_error<StringError>(
-        Twine("Malformed log. Buffer Extents unexpected; expected: ") +
-            fdrStateToTwine(State.Expects),
-        std::make_error_code(std::errc::executable_format_error));
-
-  auto PreReadOffset = OffsetPtr;
-  State.CurrentBufferSize = RecordExtractor.getU64(&OffsetPtr);
-  if (OffsetPtr == PreReadOffset)
-    return createStringError(
-        std::make_error_code(std::errc::executable_format_error),
-        "Failed to read current buffer size at offset %d.", OffsetPtr);
-
-  State.Expects = FDRState::Token::NEW_BUFFER_RECORD_OR_EOF;
-
-  // Advance the offset pointer by enough bytes accounting for the padding in a
-  // metadata record, after we read in the buffer extents.
-  OffsetPtr += kFDRMetadataBodySize - 8;
-  return Error::success();
-}
-
-/// State transition when a CallArgumentRecord is encountered.
-Error processFDRCallArgumentRecord(FDRState &State,
-                                   DataExtractor &RecordExtractor,
-                                   std::vector<XRayRecord> &Records,
-                                   uint32_t &OffsetPtr) {
-  auto &Enter = Records.back();
-  if (Enter.Type != RecordTypes::ENTER && Enter.Type != RecordTypes::ENTER_ARG)
-    return make_error<StringError>(
-        "CallArgument needs to be right after a function entry",
-        std::make_error_code(std::errc::executable_format_error));
-
-  auto PreReadOffset = OffsetPtr;
-  auto Arg = RecordExtractor.getU64(&OffsetPtr);
-  if (OffsetPtr == PreReadOffset)
-    return createStringError(
-        std::make_error_code(std::errc::executable_format_error),
-        "Failed to read argument record at offset %d.", OffsetPtr);
-
-  Enter.Type = RecordTypes::ENTER_ARG;
-  Enter.CallArgs.emplace_back(Arg);
-
-  // Advance the offset pointer by enough bytes accounting for the padding in a
-  // metadata record, after reading the payload.
-  OffsetPtr += kFDRMetadataBodySize - 8;
-  return Error::success();
-}
-
-/// Advances the state machine for reading the FDR record type by reading one
-/// Metadata Record and updating the State appropriately based on the kind of
-/// record encountered. The RecordKind is encoded in the first byte of the
-/// Record, which the caller should pass in because they have already read it
-/// to determine that this is a metadata record as opposed to a function record.
-///
-/// Beginning with Version 2 of the FDR log, we do not depend on the size of the
-/// buffer, but rather use the extents to determine how far to read in the log
-/// for this particular buffer.
-///
-/// In Version 3, FDR log now includes a pid metadata record after
-/// WallTimeMarker
-Error processFDRMetadataRecord(FDRState &State, DataExtractor &RecordExtractor,
-                               uint32_t &OffsetPtr,
-                               std::vector<XRayRecord> &Records,
-                               uint16_t Version, uint8_t FirstByte) {
-  // The remaining 7 bits of the first byte are the RecordKind enum for each
-  // Metadata Record.
-  switch (FirstByte >> 1) {
-  case 0: // NewBuffer
-    if (auto E = processFDRNewBufferRecord(State, RecordExtractor, OffsetPtr))
-      return E;
-    break;
-  case 1: // EndOfBuffer
-    if (Version >= 2)
-      return make_error<StringError>(
-          "Since Version 2 of FDR logging, we no longer support EOB records.",
-          std::make_error_code(std::errc::executable_format_error));
-    if (auto E = processFDREndOfBufferRecord(State, OffsetPtr))
-      return E;
-    break;
-  case 2: // NewCPUId
-    if (auto E = processFDRNewCPUIdRecord(State, RecordExtractor, OffsetPtr))
-      return E;
-    break;
-  case 3: // TSCWrap
-    if (auto E = processFDRTSCWrapRecord(State, RecordExtractor, OffsetPtr))
-      return E;
-    break;
-  case 4: // WallTimeMarker
-    if (auto E = processFDRWallTimeRecord(State, RecordExtractor, OffsetPtr))
-      return E;
-    // In Version 3 and and above, a PidRecord is expected after WallTimeRecord
-    if (Version >= 3)
-      State.Expects = FDRState::Token::PID_RECORD;
-    break;
-  case 5: // CustomEventMarker
-    if (auto E = processCustomEventMarker(State, RecordExtractor, OffsetPtr))
-      return E;
-    break;
-  case 6: // CallArgument
-    if (auto E = processFDRCallArgumentRecord(State, RecordExtractor, Records,
-                                              OffsetPtr))
-      return E;
-    break;
-  case 7: // BufferExtents
-    if (auto E = processBufferExtents(State, RecordExtractor, OffsetPtr))
-      return E;
-    break;
-  case 9: // Pid
-    if (auto E = processFDRPidRecord(State, RecordExtractor, OffsetPtr))
-      return E;
-    break;
-  default:
-    return createStringError(
-        std::make_error_code(std::errc::executable_format_error),
-        "Illegal metadata record type: '%d' at offset %d.", FirstByte >> 1,
-        OffsetPtr);
-  }
-  return Error::success();
-}
-
-/// Reads a function record from an FDR format log, appending a new XRayRecord
-/// to the vector being populated and updating the State with a new value
-/// reference value to interpret TSC deltas.
-///
-/// The XRayRecord constructed includes information from the function record
-/// processed here as well as Thread ID and CPU ID formerly extracted into
-/// State.
-Error processFDRFunctionRecord(FDRState &State, DataExtractor &RecordExtractor,
-                               uint32_t &OffsetPtr, uint8_t FirstByte,
-                               std::vector<XRayRecord> &Records) {
-  switch (State.Expects) {
-  case FDRState::Token::NEW_BUFFER_RECORD_OR_EOF:
-    return make_error<StringError>(
-        "Malformed log. Received Function Record before new buffer setup.",
-        std::make_error_code(std::errc::executable_format_error));
-  case FDRState::Token::WALLCLOCK_RECORD:
-    return make_error<StringError>(
-        "Malformed log. Received Function Record when expecting wallclock.",
-        std::make_error_code(std::errc::executable_format_error));
-  case FDRState::Token::PID_RECORD:
-    return make_error<StringError>(
-        "Malformed log. Received Function Record when expecting pid.",
-        std::make_error_code(std::errc::executable_format_error));
-  case FDRState::Token::NEW_CPU_ID_RECORD:
-    return make_error<StringError>(
-        "Malformed log. Received Function Record before first CPU record.",
-        std::make_error_code(std::errc::executable_format_error));
-  default:
-    Records.emplace_back();
-    auto &Record = Records.back();
-    Record.RecordType = 0; // Record is type NORMAL.
-    // Back up one byte to re-read the first byte, which is important for
-    // computing the function id for a record.
-    --OffsetPtr;
-
-    auto PreReadOffset = OffsetPtr;
-    uint32_t FuncIdBitField = RecordExtractor.getU32(&OffsetPtr);
-    if (OffsetPtr == PreReadOffset)
-      return createStringError(
-          std::make_error_code(std::errc::executable_format_error),
-          "Failed reading truncated function id field at offset %d.",
-          OffsetPtr);
-
-    FirstByte = FuncIdBitField & 0xffu;
-    // Strip off record type bit and use the next three bits.
-    auto T = (FirstByte >> 1) & 0x07;
-    switch (T) {
-    case static_cast<decltype(T)>(RecordTypes::ENTER):
-      Record.Type = RecordTypes::ENTER;
-      break;
-    case static_cast<decltype(T)>(RecordTypes::EXIT):
-      Record.Type = RecordTypes::EXIT;
-      break;
-    case static_cast<decltype(T)>(RecordTypes::TAIL_EXIT):
-      Record.Type = RecordTypes::TAIL_EXIT;
-      break;
-    case static_cast<decltype(T)>(RecordTypes::ENTER_ARG):
-      Record.Type = RecordTypes::ENTER_ARG;
-      State.Expects = FDRState::Token::CALL_ARGUMENT;
-      break;
-    default:
-      return createStringError(
-          std::make_error_code(std::errc::executable_format_error),
-          "Illegal function record type '%d' at offset %d.", T, OffsetPtr);
-    }
-    Record.CPU = State.CPUId;
-    Record.TId = State.ThreadId;
-    Record.PId = State.ProcessId;
-
-    // Despite function Id being a signed int on XRayRecord,
-    // when it is written to an FDR format, the top bits are truncated,
-    // so it is effectively an unsigned value. When we shift off the
-    // top four bits, we want the shift to be logical, so we read as
-    // uint32_t.
-    Record.FuncId = FuncIdBitField >> 4;
-
-    // FunctionRecords have a 32 bit delta from the previous absolute TSC
-    // or TSC delta. If this would overflow, we should read a TSCWrap record
-    // with an absolute TSC reading.
-    PreReadOffset = OffsetPtr;
-    uint64_t NewTSC = State.BaseTSC + RecordExtractor.getU32(&OffsetPtr);
-    if (OffsetPtr == PreReadOffset)
-      return createStringError(
-          std::make_error_code(std::errc::executable_format_error),
-          "Failed reading TSC delta at offset %d.", OffsetPtr);
-
-    State.BaseTSC = NewTSC;
-    Record.TSC = NewTSC;
-  }
-  return Error::success();
-}
-
 /// Reads a log in FDR mode for version 1 of this binary format. FDR mode is
 /// defined as part of the compiler-rt project in xray_fdr_logging.h, and such
 /// a log consists of the familiar 32 bit XRayHeader, followed by sequences of
@@ -709,101 +251,84 @@ Error loadFDRLog(StringRef Data, bool Is
                  XRayFileHeader &FileHeader, std::vector<XRayRecord> &Records) {
 
   if (Data.size() < 32)
-    return make_error<StringError>(
-        "Not enough bytes for an XRay log.",
-        std::make_error_code(std::errc::invalid_argument));
+    return createStringError(std::make_error_code(std::errc::invalid_argument),
+                             "Not enough bytes for an XRay FDR log.");
+  DataExtractor DE(Data, IsLittleEndian, 8);
 
-  DataExtractor Reader(Data, IsLittleEndian, 8);
   uint32_t OffsetPtr = 0;
-  auto FileHeaderOrError = readBinaryFormatHeader(Reader, OffsetPtr);
+  auto FileHeaderOrError = readBinaryFormatHeader(DE, OffsetPtr);
   if (!FileHeaderOrError)
     return FileHeaderOrError.takeError();
   FileHeader = std::move(FileHeaderOrError.get());
 
-  uint64_t BufferSize = 0;
+  // First we load the records into memory.
+  std::vector<std::unique_ptr<Record>> FDRRecords;
+
   {
-    StringRef ExtraDataRef(FileHeader.FreeFormData, 16);
-    DataExtractor ExtraDataExtractor(ExtraDataRef, IsLittleEndian, 8);
-    uint32_t ExtraDataOffset = 0;
-    BufferSize = ExtraDataExtractor.getU64(&ExtraDataOffset);
+    FileBasedRecordProducer P(FileHeader, DE, OffsetPtr);
+    LogBuilderConsumer C(FDRRecords);
+    while (DE.isValidOffsetForDataOfSize(OffsetPtr, 1)) {
+      auto R = P.produce();
+      if (!R)
+        return R.takeError();
+      if (auto E = C.consume(std::move(R.get())))
+        return E;
+    }
   }
 
-  FDRState::Token InitialExpectation;
-  switch (FileHeader.Version) {
-  case 1:
-    InitialExpectation = FDRState::Token::NEW_BUFFER_RECORD_OR_EOF;
-    break;
-  case 2:
-  case 3:
-    InitialExpectation = FDRState::Token::BUFFER_EXTENTS;
-    break;
-  default:
-    return make_error<StringError>(
-        Twine("Unsupported version '") + Twine(FileHeader.Version) + "'",
-        std::make_error_code(std::errc::executable_format_error));
+  // Next we index the records into blocks.
+  BlockIndexer::Index Index;
+  {
+    BlockIndexer Indexer(Index);
+    for (auto &R : FDRRecords)
+      if (auto E = R->apply(Indexer))
+        return E;
+    if (auto E = Indexer.flush())
+      return E;
   }
-  FDRState State{0, 0, 0, 0, InitialExpectation, BufferSize, 0};
 
-  // RecordSize will tell the loop how far to seek ahead based on the record
-  // type that we have just read.
-  while (Reader.isValidOffset(OffsetPtr)) {
-    auto BeginOffset = OffsetPtr;
-    if (State.Expects == FDRState::Token::SCAN_TO_END_OF_THREAD_BUF) {
-      OffsetPtr += State.CurrentBufferSize - State.CurrentBufferConsumed;
-      State.CurrentBufferConsumed = 0;
-      State.Expects = FDRState::Token::NEW_BUFFER_RECORD_OR_EOF;
-      continue;
-    }
-    auto PreReadOffset = OffsetPtr;
-    uint8_t BitField = Reader.getU8(&OffsetPtr);
-    if (OffsetPtr == PreReadOffset)
-      return createStringError(
-          std::make_error_code(std::errc::executable_format_error),
-          "Failed reading first byte of record at offset %d.", OffsetPtr);
-    bool isMetadataRecord = BitField & 0x01uL;
-    bool isBufferExtents =
-        (BitField >> 1) == 7; // BufferExtents record kind == 7
-    if (isMetadataRecord) {
-      if (auto E = processFDRMetadataRecord(State, Reader, OffsetPtr, Records,
-                                            FileHeader.Version, BitField))
-        return E;
-    } else { // Process Function Record
-      if (auto E = processFDRFunctionRecord(State, Reader, OffsetPtr, BitField,
-                                            Records))
-        return E;
+  // Then we verify the consistency of the blocks.
+  {
+    BlockVerifier Verifier;
+    for (auto &PTB : Index) {
+      auto &Blocks = PTB.second;
+      for (auto &B : Blocks) {
+        for (auto *R : B.Records)
+          if (auto E = R->apply(Verifier))
+            return E;
+        if (auto E = Verifier.verify())
+          return E;
+        Verifier.reset();
+      }
     }
+  }
 
-    // The BufferExtents record is technically not part of the buffer, so we
-    // don't count the size of that record against the buffer's actual size.
-    if (!isBufferExtents)
-      State.CurrentBufferConsumed += OffsetPtr - BeginOffset;
-
-    assert(State.CurrentBufferConsumed <= State.CurrentBufferSize);
-
-    if ((FileHeader.Version == 2 || FileHeader.Version == 3) &&
-        State.CurrentBufferSize == State.CurrentBufferConsumed) {
-      // In Version 2 of the log, we don't need to scan to the end of the thread
-      // buffer if we've already consumed all the bytes we need to.
-      State.Expects = FDRState::Token::BUFFER_EXTENTS;
-      State.CurrentBufferSize = BufferSize;
-      State.CurrentBufferConsumed = 0;
+  // This is now the meat of the algorithm. Here we sort the blocks according to
+  // the Walltime record in each of the blocks for the same thread. This allows
+  // us to more consistently recreate the execution trace in temporal order.
+  // After the sort, we then reconstitute `Trace` records using a stateful
+  // visitor associated with a single process+thread pair.
+  {
+    for (auto &PTB : Index) {
+      auto &Blocks = PTB.second;
+      llvm::sort(
+          Blocks.begin(), Blocks.end(),
+          [](const BlockIndexer::Block &L, const BlockIndexer::Block &R) {
+            return (L.WallclockTime->seconds() < R.WallclockTime->seconds() &&
+                    L.WallclockTime->nanos() < R.WallclockTime->nanos());
+          });
+      TraceExpander Expander([&](const XRayRecord &R) { Records.push_back(R); },
+                             FileHeader.Version);
+      for (auto &B : Blocks) {
+        for (auto *R : B.Records)
+          if (auto E = R->apply(Expander))
+            return E;
+      }
+      if (auto E = Expander.flush())
+        return E;
     }
   }
 
-  // Having iterated over everything we've been given, we've either consumed
-  // everything and ended up in the end state, or were told to skip the rest.
-  bool Finished = State.Expects == FDRState::Token::SCAN_TO_END_OF_THREAD_BUF &&
-                  State.CurrentBufferSize == State.CurrentBufferConsumed;
-  if ((State.Expects != FDRState::Token::NEW_BUFFER_RECORD_OR_EOF &&
-       State.Expects != FDRState::Token::BUFFER_EXTENTS) &&
-      !Finished)
-    return make_error<StringError>(
-        Twine("Encountered EOF with unexpected state expectation ") +
-            fdrStateToTwine(State.Expects) +
-            ". Remaining expected bytes in thread buffer total " +
-            Twine(State.CurrentBufferSize - State.CurrentBufferConsumed),
-        std::make_error_code(std::errc::executable_format_error));
-
   return Error::success();
 }
 




More information about the llvm-commits mailing list