[RELAND] tp: move thread state implementation to trace processor

Thread state is pretty complicated to calculate and has a lot of edge
cases and subtlety that's hard to capture in SQL. The other problem is
that we also want to have thread state available for metrics (currently
some metrics have recreated a cut down version of this class).

For this reason, move the thread state computation as a dynamic table in
C++ which allows sharing it with other places. It also has the advantage
of making startup of the UI much faster as it doesn't need to compute
the thread state table.

This is a reland of aosp/1360559

Bug: 163812686
Change-Id: I48ed30e9b9696890fb8d389b0014d65e609feb3d
diff --git a/Android.bp b/Android.bp
index a5a45c5..bd994b3 100644
--- a/Android.bp
+++ b/Android.bp
@@ -6787,6 +6787,7 @@
     "src/trace_processor/dynamic/experimental_flamegraph_generator.cc",
     "src/trace_processor/dynamic/experimental_sched_upid_generator.cc",
     "src/trace_processor/dynamic/experimental_slice_layout_generator.cc",
+    "src/trace_processor/dynamic/thread_state_generator.cc",
     "src/trace_processor/iterator_impl.cc",
     "src/trace_processor/read_trace.cc",
     "src/trace_processor/trace_processor.cc",
@@ -6985,6 +6986,7 @@
   srcs: [
     "src/trace_processor/dynamic/experimental_counter_dur_generator_unittest.cc",
     "src/trace_processor/dynamic/experimental_slice_layout_generator_unittest.cc",
+    "src/trace_processor/dynamic/thread_state_generator_unittest.cc",
     "src/trace_processor/forwarding_trace_parser_unittest.cc",
     "src/trace_processor/importers/ftrace/sched_event_tracker_unittest.cc",
     "src/trace_processor/importers/fuchsia/fuchsia_trace_utils_unittest.cc",
diff --git a/BUILD b/BUILD
index 576dc0e..33db149 100644
--- a/BUILD
+++ b/BUILD
@@ -961,6 +961,8 @@
         "src/trace_processor/dynamic/experimental_sched_upid_generator.h",
         "src/trace_processor/dynamic/experimental_slice_layout_generator.cc",
         "src/trace_processor/dynamic/experimental_slice_layout_generator.h",
+        "src/trace_processor/dynamic/thread_state_generator.cc",
+        "src/trace_processor/dynamic/thread_state_generator.h",
         "src/trace_processor/iterator_impl.cc",
         "src/trace_processor/iterator_impl.h",
         "src/trace_processor/read_trace.cc",
diff --git a/src/trace_processor/BUILD.gn b/src/trace_processor/BUILD.gn
index 1d401a7..69d80d8 100644
--- a/src/trace_processor/BUILD.gn
+++ b/src/trace_processor/BUILD.gn
@@ -291,6 +291,8 @@
       "dynamic/experimental_sched_upid_generator.h",
       "dynamic/experimental_slice_layout_generator.cc",
       "dynamic/experimental_slice_layout_generator.h",
+      "dynamic/thread_state_generator.cc",
+      "dynamic/thread_state_generator.h",
       "iterator_impl.cc",
       "iterator_impl.h",
       "read_trace.cc",
@@ -404,6 +406,7 @@
     sources += [
       "dynamic/experimental_counter_dur_generator_unittest.cc",
       "dynamic/experimental_slice_layout_generator_unittest.cc",
+      "dynamic/thread_state_generator_unittest.cc",
     ]
     deps += [
       ":lib",
diff --git a/src/trace_processor/db/table.h b/src/trace_processor/db/table.h
index 45d96e4..fb8b73b 100644
--- a/src/trace_processor/db/table.h
+++ b/src/trace_processor/db/table.h
@@ -194,13 +194,13 @@
   }
 
   template <typename T>
-  const TypedColumn<T>* GetTypedColumnByName(const char* name) const {
-    return TypedColumn<T>::FromColumn(GetColumnByName(name));
+  const TypedColumn<T>& GetTypedColumnByName(const char* name) const {
+    return *TypedColumn<T>::FromColumn(GetColumnByName(name));
   }
 
   template <typename T>
-  const IdColumn<T>* GetIdColumnByName(const char* name) const {
-    return IdColumn<T>::FromColumn(GetColumnByName(name));
+  const IdColumn<T>& GetIdColumnByName(const char* name) const {
+    return *IdColumn<T>::FromColumn(GetColumnByName(name));
   }
 
   // Returns the number of columns in the Table.
diff --git a/src/trace_processor/dynamic/experimental_slice_layout_generator.cc b/src/trace_processor/dynamic/experimental_slice_layout_generator.cc
index a111016..297b7e0 100644
--- a/src/trace_processor/dynamic/experimental_slice_layout_generator.cc
+++ b/src/trace_processor/dynamic/experimental_slice_layout_generator.cc
@@ -179,13 +179,13 @@
   // Map of id -> root_id
   std::map<tables::SliceTable::Id, tables::SliceTable::Id> id_map;
 
-  const auto& id_col = *table.GetIdColumnByName<tables::SliceTable::Id>("id");
+  const auto& id_col = table.GetIdColumnByName<tables::SliceTable::Id>("id");
   const auto& parent_id_col =
-      *table.GetTypedColumnByName<base::Optional<tables::SliceTable::Id>>(
+      table.GetTypedColumnByName<base::Optional<tables::SliceTable::Id>>(
           "parent_id");
-  const auto& depth_col = *table.GetTypedColumnByName<uint32_t>("depth");
-  const auto& ts_col = *table.GetTypedColumnByName<int64_t>("ts");
-  const auto& dur_col = *table.GetTypedColumnByName<int64_t>("dur");
+  const auto& depth_col = table.GetTypedColumnByName<uint32_t>("depth");
+  const auto& ts_col = table.GetTypedColumnByName<int64_t>("ts");
+  const auto& dur_col = table.GetTypedColumnByName<int64_t>("dur");
 
   // Step 1:
   // Find the bounding box (start ts, end ts, and max depth) for each group
diff --git a/src/trace_processor/dynamic/thread_state_generator.cc b/src/trace_processor/dynamic/thread_state_generator.cc
new file mode 100644
index 0000000..8a1abda
--- /dev/null
+++ b/src/trace_processor/dynamic/thread_state_generator.cc
@@ -0,0 +1,234 @@
+/*
+ * Copyright (C) 2020 The Android Open Source Project
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "src/trace_processor/dynamic/thread_state_generator.h"
+
+#include <memory>
+#include <set>
+
+#include "src/trace_processor/types/trace_processor_context.h"
+
+namespace perfetto {
+namespace trace_processor {
+
+ThreadStateGenerator::ThreadStateGenerator(TraceProcessorContext* context)
+    : running_string_id_(context->storage->InternString("Running")),
+      runnable_string_id_(context->storage->InternString("R")),
+      context_(context) {}
+
+ThreadStateGenerator::~ThreadStateGenerator() = default;
+
+util::Status ThreadStateGenerator::ValidateConstraints(
+    const QueryConstraints&) {
+  return util::OkStatus();
+}
+
+std::unique_ptr<Table> ThreadStateGenerator::ComputeTable(
+    const std::vector<Constraint>&,
+    const std::vector<Order>&) {
+  if (!unsorted_thread_state_table_) {
+    int64_t trace_end_ts =
+        context_->storage->GetTraceTimestampBoundsNs().second;
+
+    unsorted_thread_state_table_ = ComputeThreadStateTable(trace_end_ts);
+
+    // We explicitly sort by ts here as ComputeThreadStateTable does not insert
+    // rows in sorted order but we expect our clients to always want to sort
+    // on ts. Writing ComputeThreadStateTable to insert in sorted order is
+    // more trouble than its worth.
+    sorted_thread_state_table_ = unsorted_thread_state_table_->Sort(
+        {unsorted_thread_state_table_->ts().ascending()});
+  }
+  PERFETTO_CHECK(sorted_thread_state_table_);
+  return std::unique_ptr<Table>(new Table(sorted_thread_state_table_->Copy()));
+}
+
+std::unique_ptr<tables::ThreadStateTable>
+ThreadStateGenerator::ComputeThreadStateTable(int64_t trace_end_ts) {
+  std::unique_ptr<tables::ThreadStateTable> table(new tables::ThreadStateTable(
+      context_->storage->mutable_string_pool(), nullptr));
+
+  const auto& raw_sched = context_->storage->sched_slice_table();
+  const auto& instants = context_->storage->instant_table();
+
+  // In both tables, exclude utid == 0 which represents the idle thread.
+  auto sched = raw_sched.Filter({raw_sched.utid().ne(0)});
+  auto waking = instants.Filter(
+      {instants.name().eq("sched_waking"), instants.ref().ne(0)});
+
+  // We prefer to use waking if at all possible and fall back to wakeup if not
+  // available.
+  if (waking.row_count() == 0) {
+    waking = instants.Filter(
+        {instants.name().eq("sched_wakeup"), instants.ref().ne(0)});
+  }
+
+  const auto& sched_ts = sched.GetTypedColumnByName<int64_t>("ts");
+  const auto& waking_ts = waking.GetTypedColumnByName<int64_t>("ts");
+
+  uint32_t sched_idx = 0;
+  uint32_t waking_idx = 0;
+  std::unordered_map<UniqueTid, uint32_t> state_map;
+  while (sched_idx < sched.row_count() || waking_idx < waking.row_count()) {
+    // We go through both tables, picking the earliest timestamp from either
+    // to process that event.
+    if (waking_idx >= waking.row_count() ||
+        (sched_idx < sched.row_count() &&
+         sched_ts[sched_idx] <= waking_ts[waking_idx])) {
+      AddSchedEvent(sched, sched_idx++, state_map, trace_end_ts, table.get());
+    } else {
+      AddWakingEvent(waking, waking_idx++, state_map, table.get());
+    }
+  }
+  return table;
+}
+
+void ThreadStateGenerator::UpdateDurIfNotRunning(
+    int64_t new_ts,
+    uint32_t row,
+    tables::ThreadStateTable* table) {
+  // The duration should always have been left dangling and the new timestamp
+  // should happen after the older one (as we go through events in ts order).
+  PERFETTO_DCHECK(table->dur()[row] == -1);
+  PERFETTO_DCHECK(new_ts > table->ts()[row]);
+
+  if (table->state()[row] == running_string_id_)
+    return;
+  table->mutable_dur()->Set(row, new_ts - table->ts()[row]);
+}
+
+void ThreadStateGenerator::AddSchedEvent(
+    const Table& sched,
+    uint32_t sched_idx,
+    std::unordered_map<UniqueTid, uint32_t>& state_map,
+    int64_t trace_end_ts,
+    tables::ThreadStateTable* table) {
+  UniqueTid utid = sched.GetTypedColumnByName<uint32_t>("utid")[sched_idx];
+  int64_t ts = sched.GetTypedColumnByName<int64_t>("ts")[sched_idx];
+
+  // Update the duration on the existing event.
+  auto it = state_map.find(utid);
+  if (it != state_map.end()) {
+    // Don't update dur from -1 if we were already running (can happen because
+    // of data loss).
+    UpdateDurIfNotRunning(ts, it->second, table);
+  }
+
+  // Undo the expansion of the final sched slice for each CPU to the end of the
+  // trace by setting the duration back to -1. This counteracts the code in
+  // SchedEventTracker::FlushPendingEvents
+  // TODO(lalitm): remove this hack when we stop expanding the last slice to the
+  // end of the trace.
+  int64_t dur = sched.GetTypedColumnByName<int64_t>("dur")[sched_idx];
+  if (ts + dur == trace_end_ts) {
+    dur = -1;
+  }
+
+  // First, we add the sched slice itself as "Running" with the other fields
+  // unchanged.
+  tables::ThreadStateTable::Row sched_row;
+  sched_row.ts = ts;
+  sched_row.dur = dur;
+  sched_row.cpu = sched.GetTypedColumnByName<uint32_t>("cpu")[sched_idx];
+  sched_row.state = running_string_id_;
+  sched_row.utid = utid;
+  state_map[utid] = table->Insert(sched_row).row;
+
+  // If the sched row had a negative duration, don't add the dangling
+  // descheduled slice as it would be meaningless.
+  if (sched_row.dur == -1) {
+    return;
+  }
+
+  // Next, we add a dangling, slice to represent the descheduled state with the
+  // given end state from the sched event. The duration will be updated by the
+  // next event (sched or waking) that we see.
+  tables::ThreadStateTable::Row row;
+  row.ts = sched_row.ts + sched_row.dur;
+  row.dur = -1;
+  row.state = sched.GetTypedColumnByName<StringId>("end_state")[sched_idx];
+  row.utid = utid;
+  state_map[utid] = table->Insert(row).row;
+}
+
+void ThreadStateGenerator::AddWakingEvent(
+    const Table& waking,
+    uint32_t waking_idx,
+    std::unordered_map<UniqueTid, uint32_t>& state_map,
+    tables::ThreadStateTable* table) {
+  int64_t ts = waking.GetTypedColumnByName<int64_t>("ts")[waking_idx];
+  UniqueTid utid = static_cast<UniqueTid>(
+      waking.GetTypedColumnByName<int64_t>("ref")[waking_idx]);
+
+  auto it = state_map.find(utid);
+  if (it != state_map.end()) {
+    // As counter-intuitive as it seems, occassionally we can get a waking
+    // event for a thread which is currently running.
+    //
+    // There are two cases when this can happen:
+    // 1. The kernel legitimately send a waking event for a "running" thread
+    //    because the thread was woken up before the kernel switched away
+    //    from it. In this case, the waking timestamp will be in the past
+    //    because we added the descheduled slice when we processed the sched
+    //    event).
+    // 2. We're close to the end of the trace or had data-loss and we missed
+    //    the switch out event for a thread but we see a waking after.
+
+    // Case 1 described above. In this situation, we should drop the waking
+    // entirely.
+    if (table->ts()[it->second] >= ts) {
+      return;
+    }
+
+    // Case 2 described above. We still want to set the thread as running but
+    // we don't update the duration.
+    UpdateDurIfNotRunning(ts, it->second, table);
+  }
+
+  // Add a dangling slice to represent that this slice is now available for
+  // running. The duration will be updated by the next event (sched) that we
+  // see.
+  tables::ThreadStateTable::Row row;
+  row.ts = ts;
+  row.dur = -1;
+  row.state = runnable_string_id_;
+  row.utid = utid;
+  state_map[utid] = table->Insert(row).row;
+}
+
+Table::Schema ThreadStateGenerator::CreateSchema() {
+  auto schema = tables::ThreadStateTable::Schema();
+
+  // Because we expect our users to generally want ordered by ts, we set the
+  // ordering for the schema to match our forced sort pass in ComputeTable.
+  auto ts_it = std::find_if(
+      schema.columns.begin(), schema.columns.end(),
+      [](const Table::Schema::Column& col) { return col.name == "ts"; });
+  ts_it->is_sorted = true;
+
+  return schema;
+}
+
+std::string ThreadStateGenerator::TableName() {
+  return "thread_state";
+}
+
+uint32_t ThreadStateGenerator::EstimateRowCount() {
+  return context_->storage->sched_slice_table().row_count();
+}
+
+}  // namespace trace_processor
+}  // namespace perfetto
diff --git a/src/trace_processor/dynamic/thread_state_generator.h b/src/trace_processor/dynamic/thread_state_generator.h
new file mode 100644
index 0000000..aea75ac
--- /dev/null
+++ b/src/trace_processor/dynamic/thread_state_generator.h
@@ -0,0 +1,76 @@
+/*
+ * Copyright (C) 2020 The Android Open Source Project
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef SRC_TRACE_PROCESSOR_DYNAMIC_THREAD_STATE_GENERATOR_H_
+#define SRC_TRACE_PROCESSOR_DYNAMIC_THREAD_STATE_GENERATOR_H_
+
+#include "src/trace_processor/sqlite/db_sqlite_table.h"
+
+#include "src/trace_processor/storage/trace_storage.h"
+
+namespace perfetto {
+namespace trace_processor {
+
+class TraceProcessorContext;
+
+// Dynamic table implementing the thread state table.
+// This table is a basically the same as sched with extra information added
+// about wakeups (obtained from sched_waking/sched_wakeup).
+class ThreadStateGenerator : public DbSqliteTable::DynamicTableGenerator {
+ public:
+  explicit ThreadStateGenerator(TraceProcessorContext* context);
+  ~ThreadStateGenerator() override;
+
+  Table::Schema CreateSchema() override;
+  std::string TableName() override;
+  uint32_t EstimateRowCount() override;
+  util::Status ValidateConstraints(const QueryConstraints&) override;
+  std::unique_ptr<Table> ComputeTable(const std::vector<Constraint>& cs,
+                                      const std::vector<Order>& ob) override;
+
+  // Visible for testing.
+  std::unique_ptr<tables::ThreadStateTable> ComputeThreadStateTable(
+      int64_t trace_end_ts);
+
+ private:
+  void AddSchedEvent(const Table& sched,
+                     uint32_t sched_idx,
+                     std::unordered_map<UniqueTid, uint32_t>& state_map,
+                     int64_t trace_end_ts,
+                     tables::ThreadStateTable* table);
+
+  void AddWakingEvent(const Table& wakeup,
+                      uint32_t wakeup_idx,
+                      std::unordered_map<UniqueTid, uint32_t>& state_map,
+                      tables::ThreadStateTable* table);
+
+  void UpdateDurIfNotRunning(int64_t new_ts,
+                             uint32_t row,
+                             tables::ThreadStateTable* table);
+
+  std::unique_ptr<tables::ThreadStateTable> unsorted_thread_state_table_;
+  base::Optional<Table> sorted_thread_state_table_;
+
+  const StringId running_string_id_;
+  const StringId runnable_string_id_;
+
+  TraceProcessorContext* context_ = nullptr;
+};
+
+}  // namespace trace_processor
+}  // namespace perfetto
+
+#endif  // SRC_TRACE_PROCESSOR_DYNAMIC_THREAD_STATE_GENERATOR_H_
diff --git a/src/trace_processor/dynamic/thread_state_generator_unittest.cc b/src/trace_processor/dynamic/thread_state_generator_unittest.cc
new file mode 100644
index 0000000..3879add
--- /dev/null
+++ b/src/trace_processor/dynamic/thread_state_generator_unittest.cc
@@ -0,0 +1,293 @@
+/*
+ * Copyright (C) 2020 The Android Open Source Project
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "src/trace_processor/dynamic/thread_state_generator.h"
+
+#include <algorithm>
+
+#include "src/trace_processor/types/trace_processor_context.h"
+#include "test/gtest_and_gmock.h"
+
+namespace perfetto {
+namespace trace_processor {
+namespace {
+
+class ThreadStateGeneratorUnittest : public testing::Test {
+ public:
+  struct Ts {
+    int64_t ts;
+  };
+
+  ThreadStateGeneratorUnittest() : idle_thread_(0), thread_a_(1), thread_b_(2) {
+    context_.storage.reset(new TraceStorage());
+    thread_state_generator_.reset(new ThreadStateGenerator(&context_));
+  }
+
+  void ForwardSchedTo(Ts ts) { sched_insert_ts_ = ts.ts; }
+
+  void AddWaking(Ts ts, UniqueTid utid) {
+    tables::InstantTable::Row row;
+    row.ts = ts.ts;
+    row.ref = utid;
+    row.name = context_.storage->InternString("sched_waking");
+    context_.storage->mutable_instant_table()->Insert(row);
+  }
+
+  void AddWakup(Ts ts, UniqueTid utid) {
+    tables::InstantTable::Row row;
+    row.ts = ts.ts;
+    row.ref = utid;
+    row.name = context_.storage->InternString("sched_wakeup");
+    context_.storage->mutable_instant_table()->Insert(row);
+  }
+
+  void AddSched(base::Optional<Ts> end, UniqueTid utid, const char* end_state) {
+    StringId end_state_id = context_.storage->InternString(end_state);
+
+    tables::SchedSliceTable::Row row;
+
+    // cpu is hardcoded because it doesn't matter for the algorithm and is
+    // just passed through unchanged.
+    row.cpu = 0;
+
+    row.ts = sched_insert_ts_;
+    row.dur = end ? end->ts - row.ts : -1;
+    row.utid = utid;
+    row.end_state = end_state_id;
+    context_.storage->mutable_sched_slice_table()->Insert(row);
+
+    sched_insert_ts_ = end ? end->ts : -1;
+  }
+
+  void RunThreadStateComputation(Ts trace_end_ts = Ts{
+                                     std::numeric_limits<int64_t>::max()}) {
+    thread_state_table_ =
+        thread_state_generator_->ComputeThreadStateTable(trace_end_ts.ts);
+  }
+
+  void VerifyThreadState(Ts from,
+                         base::Optional<Ts> to,
+                         UniqueTid utid,
+                         const char* state) {
+    uint32_t row = thread_state_verify_row_;
+
+    ASSERT_LT(row, thread_state_table_->row_count());
+    EXPECT_EQ(thread_state_table_->ts()[row], from.ts);
+    EXPECT_EQ(thread_state_table_->dur()[row], to ? to->ts - from.ts : -1);
+    EXPECT_EQ(thread_state_table_->utid()[row], utid);
+    if (state == kRunning) {
+      EXPECT_EQ(thread_state_table_->cpu()[row], 0u);
+    } else {
+      EXPECT_EQ(thread_state_table_->cpu()[row], base::nullopt);
+    }
+    EXPECT_EQ(thread_state_table_->state().GetString(row),
+              base::StringView(state));
+
+    thread_state_verify_row_++;
+  }
+
+  void VerifyEndOfThreadState() {
+    ASSERT_EQ(thread_state_verify_row_, thread_state_table_->row_count());
+  }
+
+ protected:
+  static constexpr char kRunning[] = "Running";
+
+  const UniqueTid idle_thread_;
+  const UniqueTid thread_a_;
+  const UniqueTid thread_b_;
+
+ private:
+  TraceProcessorContext context_;
+
+  int64_t sched_insert_ts_ = 0;
+
+  uint32_t thread_state_verify_row_ = 0;
+
+  std::unique_ptr<ThreadStateGenerator> thread_state_generator_;
+  std::unique_ptr<tables::ThreadStateTable> thread_state_table_;
+};
+
+constexpr char ThreadStateGeneratorUnittest::kRunning[];
+
+TEST_F(ThreadStateGeneratorUnittest, MultipleThreadWithOnlySched) {
+  ForwardSchedTo(Ts{0});
+  AddSched(Ts{10}, thread_a_, "S");
+  AddSched(Ts{15}, thread_b_, "D");
+  AddSched(Ts{20}, thread_a_, "R");
+
+  RunThreadStateComputation();
+
+  VerifyThreadState(Ts{0}, Ts{10}, thread_a_, kRunning);
+  VerifyThreadState(Ts{10}, Ts{15}, thread_a_, "S");
+
+  VerifyThreadState(Ts{10}, Ts{15}, thread_b_, kRunning);
+  VerifyThreadState(Ts{15}, base::nullopt, thread_b_, "D");
+
+  VerifyThreadState(Ts{15}, Ts{20}, thread_a_, kRunning);
+  VerifyThreadState(Ts{20}, base::nullopt, thread_a_, "R");
+
+  VerifyEndOfThreadState();
+}
+
+TEST_F(ThreadStateGeneratorUnittest, WakingFirst) {
+  AddWaking(Ts{10}, thread_a_);
+
+  ForwardSchedTo(Ts{20});
+  AddSched(Ts{30}, thread_a_, "S");
+
+  RunThreadStateComputation();
+
+  VerifyThreadState(Ts{10}, Ts{20}, thread_a_, "R");
+  VerifyThreadState(Ts{20}, Ts{30}, thread_a_, kRunning);
+  VerifyThreadState(Ts{30}, base::nullopt, thread_a_, "S");
+
+  VerifyEndOfThreadState();
+}
+
+TEST_F(ThreadStateGeneratorUnittest, SchedWithWaking) {
+  ForwardSchedTo(Ts{0});
+  AddSched(Ts{10}, thread_a_, "S");
+
+  AddWaking(Ts{15}, thread_a_);
+
+  ForwardSchedTo(Ts{20});
+  AddSched(Ts{25}, thread_a_, "R");
+
+  RunThreadStateComputation();
+
+  VerifyThreadState(Ts{0}, Ts{10}, thread_a_, kRunning);
+  VerifyThreadState(Ts{10}, Ts{15}, thread_a_, "S");
+  VerifyThreadState(Ts{15}, Ts{20}, thread_a_, "R");
+  VerifyThreadState(Ts{20}, Ts{25}, thread_a_, kRunning);
+  VerifyThreadState(Ts{25}, base::nullopt, thread_a_, "R");
+
+  VerifyEndOfThreadState();
+}
+
+TEST_F(ThreadStateGeneratorUnittest, SchedWithWakeup) {
+  ForwardSchedTo(Ts{0});
+  AddSched(Ts{10}, thread_a_, "S");
+
+  AddWakup(Ts{15}, thread_a_);
+
+  ForwardSchedTo(Ts{20});
+  AddSched(Ts{25}, thread_a_, "R");
+
+  RunThreadStateComputation();
+
+  VerifyThreadState(Ts{0}, Ts{10}, thread_a_, kRunning);
+  VerifyThreadState(Ts{10}, Ts{15}, thread_a_, "S");
+  VerifyThreadState(Ts{15}, Ts{20}, thread_a_, "R");
+  VerifyThreadState(Ts{20}, Ts{25}, thread_a_, kRunning);
+  VerifyThreadState(Ts{25}, base::nullopt, thread_a_, "R");
+
+  VerifyEndOfThreadState();
+}
+
+TEST_F(ThreadStateGeneratorUnittest, SchedIdleIgnored) {
+  ForwardSchedTo(Ts{0});
+  AddSched(Ts{10}, idle_thread_, "R");
+  AddSched(Ts{15}, thread_a_, "R");
+
+  RunThreadStateComputation();
+
+  VerifyThreadState(Ts{10}, Ts{15}, thread_a_, kRunning);
+  VerifyThreadState(Ts{15}, base::nullopt, thread_a_, "R");
+
+  VerifyEndOfThreadState();
+}
+
+TEST_F(ThreadStateGeneratorUnittest, NegativeSchedDuration) {
+  ForwardSchedTo(Ts{0});
+
+  AddSched(Ts{10}, thread_a_, "S");
+
+  AddWaking(Ts{15}, thread_a_);
+
+  ForwardSchedTo(Ts{20});
+  AddSched(base::nullopt, thread_a_, "");
+
+  RunThreadStateComputation();
+
+  VerifyThreadState(Ts{0}, Ts{10}, thread_a_, kRunning);
+  VerifyThreadState(Ts{10}, Ts{15}, thread_a_, "S");
+  VerifyThreadState(Ts{15}, Ts{20}, thread_a_, "R");
+  VerifyThreadState(Ts{20}, base::nullopt, thread_a_, kRunning);
+
+  VerifyEndOfThreadState();
+}
+
+TEST_F(ThreadStateGeneratorUnittest, WakingOnRunningThreadAtEnd) {
+  AddWaking(Ts{5}, thread_a_);
+
+  ForwardSchedTo(Ts{10});
+  AddSched(base::nullopt, thread_a_, "");
+
+  AddWaking(Ts{15}, thread_a_);
+
+  RunThreadStateComputation();
+
+  VerifyThreadState(Ts{5}, Ts{10}, thread_a_, "R");
+  VerifyThreadState(Ts{10}, base::nullopt, thread_a_, kRunning);
+  VerifyThreadState(Ts{15}, base::nullopt, thread_a_, "R");
+
+  VerifyEndOfThreadState();
+}
+
+TEST_F(ThreadStateGeneratorUnittest, SchedDataLoss) {
+  ForwardSchedTo(Ts{10});
+  AddSched(base::nullopt, thread_a_, "");
+  ForwardSchedTo(Ts{30});
+  AddSched(Ts{40}, thread_a_, "D");
+
+  RunThreadStateComputation();
+
+  VerifyThreadState(Ts{10}, base::nullopt, thread_a_, kRunning);
+  VerifyThreadState(Ts{30}, Ts{40}, thread_a_, kRunning);
+  VerifyThreadState(Ts{40}, base::nullopt, thread_a_, "D");
+
+  VerifyEndOfThreadState();
+}
+
+TEST_F(ThreadStateGeneratorUnittest, StrechedSchedIgnored) {
+  ForwardSchedTo(Ts{10});
+  AddSched(Ts{100}, thread_a_, "");
+
+  RunThreadStateComputation(Ts{100});
+
+  VerifyThreadState(Ts{10}, base::nullopt, thread_a_, kRunning);
+
+  VerifyEndOfThreadState();
+}
+
+TEST_F(ThreadStateGeneratorUnittest, WakingAfterStrechedSched) {
+  ForwardSchedTo(Ts{10});
+  AddSched(Ts{100}, thread_a_, "");
+
+  AddWaking(Ts{15}, thread_a_);
+
+  RunThreadStateComputation(Ts{100});
+
+  VerifyThreadState(Ts{10}, base::nullopt, thread_a_, kRunning);
+  VerifyThreadState(Ts{15}, base::nullopt, thread_a_, "R");
+
+  VerifyEndOfThreadState();
+}
+
+}  // namespace
+}  // namespace trace_processor
+}  // namespace perfetto
diff --git a/src/trace_processor/tables/slice_tables.h b/src/trace_processor/tables/slice_tables.h
index 2ba3a0b..cfd9063 100644
--- a/src/trace_processor/tables/slice_tables.h
+++ b/src/trace_processor/tables/slice_tables.h
@@ -71,6 +71,19 @@
 PERFETTO_TP_TABLE(PERFETTO_TP_SCHED_SLICE_TABLE_DEF);
 
 // @tablegroup Events
+// @param utid {@joinable thread.utid}
+#define PERFETTO_TP_THREAD_STATE_TABLE_DEF(NAME, PARENT, C) \
+  NAME(ThreadStateTable, "thread_state")                    \
+  PERFETTO_TP_ROOT_TABLE(PARENT, C)                         \
+  C(int64_t, ts)                                            \
+  C(int64_t, dur)                                           \
+  C(base::Optional<uint32_t>, cpu)                          \
+  C(uint32_t, utid)                                         \
+  C(StringPool::Id, state)
+
+PERFETTO_TP_TABLE(PERFETTO_TP_THREAD_STATE_TABLE_DEF);
+
+// @tablegroup Events
 #define PERFETTO_TP_GPU_SLICES_DEF(NAME, PARENT, C) \
   NAME(GpuSliceTable, "gpu_slice")                  \
   PARENT(PERFETTO_TP_SLICE_TABLE_DEF, C)            \
diff --git a/src/trace_processor/tables/table_destructors.cc b/src/trace_processor/tables/table_destructors.cc
index 9f4d207..1ae783f 100644
--- a/src/trace_processor/tables/table_destructors.cc
+++ b/src/trace_processor/tables/table_destructors.cc
@@ -71,6 +71,7 @@
 GpuSliceTable::~GpuSliceTable() = default;
 GraphicsFrameSliceTable::~GraphicsFrameSliceTable() = default;
 DescribeSliceTable::~DescribeSliceTable() = default;
+ThreadStateTable::~ThreadStateTable() = default;
 
 // track_tables.h
 TrackTable::~TrackTable() = default;
diff --git a/src/trace_processor/trace_processor_impl.cc b/src/trace_processor/trace_processor_impl.cc
index 500ff87..1af9088 100644
--- a/src/trace_processor/trace_processor_impl.cc
+++ b/src/trace_processor/trace_processor_impl.cc
@@ -30,6 +30,7 @@
 #include "src/trace_processor/dynamic/experimental_flamegraph_generator.h"
 #include "src/trace_processor/dynamic/experimental_sched_upid_generator.h"
 #include "src/trace_processor/dynamic/experimental_slice_layout_generator.h"
+#include "src/trace_processor/dynamic/thread_state_generator.h"
 #include "src/trace_processor/export_json.h"
 #include "src/trace_processor/importers/additional_modules.h"
 #include "src/trace_processor/importers/ftrace/sched_event_tracker.h"
@@ -707,6 +708,8 @@
   RegisterDynamicTable(std::unique_ptr<ExperimentalSchedUpidGenerator>(
       new ExperimentalSchedUpidGenerator(storage->sched_slice_table(),
                                          storage->thread_table())));
+  RegisterDynamicTable(std::unique_ptr<ThreadStateGenerator>(
+      new ThreadStateGenerator(&context_)));
 
   // New style db-backed tables.
   RegisterDbTable(storage->arg_table());
diff --git a/ui/src/controller/trace_controller.ts b/ui/src/controller/trace_controller.ts
index a5946db..d12dc66 100644
--- a/ui/src/controller/trace_controller.ts
+++ b/ui/src/controller/trace_controller.ts
@@ -402,67 +402,6 @@
 
   async initialiseHelperViews() {
     const engine = assertExists<Engine>(this.engine);
-    this.updateStatus('Creating views');
-    let event = 'sched_waking';
-    const waking = await engine.query(
-        `select * from instants where name = 'sched_waking' limit 1`);
-    if (waking.numRecords === 0) {
-      // Only use sched_wakeup if sched_waking is not in the trace.
-      event = 'sched_wakeup';
-    }
-    await engine.query(`create view runnable AS
-      select
-        ts,
-        lead(ts, 1, (select end_ts from trace_bounds))
-          OVER(partition by ref order by ts) - ts as dur,
-        ref as utid
-      from instants
-      where name = '${event}'`);
-
-    // Get the first ts for each utid - whether a sched wakeup/waking
-    // or sched event.
-    await engine.query(`create view first_thread as
-      select min(ts) as ts, utid from
-      (select min(ts) as ts, utid from runnable group by utid
-       UNION
-      select min(ts) as ts,utid from sched group by utid)
-      group by utid`);
-
-    // Create an entry from first ts to either the first sched_wakeup/waking
-    // or to the end if there are no sched wakeup/ings. This means we will
-    // show all information we have even with no sched_wakeup/waking events.
-    await engine.query(`create view fill as
-      select first_thread.ts as ts,
-      coalesce(min(runnable.ts), (select end_ts from trace_bounds)) -
-      first_thread.ts as dur,
-      first_thread.utid as utid
-      from first_thread
-      LEFT JOIN runnable using(utid) group by utid`);
-
-    await engine.query(`create view full_runnable as
-        select * from runnable UNION
-        select * from fill`);
-
-    await engine.query(`create virtual table thread_span
-        using span_left_join(
-          full_runnable partitioned utid,
-          sched partitioned utid)`);
-
-    this.updateStatus('Creating thread state table');
-    // For performance reasons we need to create a table here.
-    // Once b/145350531 is fixed this should be able to revert to a
-    // view and we can recover the extra memory use.
-    await engine.query(`create table thread_state as
-      select ts, dur, utid, cpu,
-      case when end_state is not null then 'Running'
-      when lag(end_state) over ordered is not null
-      then lag(end_state) over ordered else 'R'
-      end as state
-      from thread_span window ordered as
-      (partition by utid order by ts)`);
-
-    this.updateStatus('Creating thread state index');
-    await engine.query(`create index utid_index on thread_state(utid)`);
 
     this.updateStatus('Creating annotation counter track table');
     // Create the helper tables for all the annotations related data.