tp: ClockConverter
To fix the bug we had to implement https://observablehq.com/d/2ba5ca5bbdebe9a9. To simplify the conversion, we implemented ClockConverter, which allows conversion using only data from TraceStorage, and not graphs from ClockTracker.
ClockConverter first fetches all of the snapshots related to the clocks used for conversion and then applies the algorithm from above link to convert trace time to clock values.
This enabled:
- deconstruction of ClockTracker with context in DestroyContext() which released some memory
- further simplification of ClockTracker, which no longer needs any logic related to FromTraceTime()
Bug: 271428674
Change-Id: Ie1404a51e0004519753bab1a727a65fb1dea4a33
diff --git a/Android.bp b/Android.bp
index bb3300e..81477a0 100644
--- a/Android.bp
+++ b/Android.bp
@@ -9300,6 +9300,7 @@
"src/trace_processor/importers/common/args_tracker.cc",
"src/trace_processor/importers/common/args_translation_table.cc",
"src/trace_processor/importers/common/async_track_set_tracker.cc",
+ "src/trace_processor/importers/common/clock_converter.cc",
"src/trace_processor/importers/common/clock_tracker.cc",
"src/trace_processor/importers/common/deobfuscation_mapping_table.cc",
"src/trace_processor/importers/common/event_tracker.cc",
@@ -9331,6 +9332,7 @@
srcs: [
"src/trace_processor/importers/common/args_translation_table_unittest.cc",
"src/trace_processor/importers/common/async_track_set_tracker_unittest.cc",
+ "src/trace_processor/importers/common/clock_converter_unittest.cc",
"src/trace_processor/importers/common/clock_tracker_unittest.cc",
"src/trace_processor/importers/common/deobfuscation_mapping_table_unittest.cc",
"src/trace_processor/importers/common/event_tracker_unittest.cc",
diff --git a/BUILD b/BUILD
index 82fd786..05aee8b 100644
--- a/BUILD
+++ b/BUILD
@@ -1117,6 +1117,8 @@
"src/trace_processor/importers/common/async_track_set_tracker.cc",
"src/trace_processor/importers/common/async_track_set_tracker.h",
"src/trace_processor/importers/common/chunked_trace_reader.h",
+ "src/trace_processor/importers/common/clock_converter.cc",
+ "src/trace_processor/importers/common/clock_converter.h",
"src/trace_processor/importers/common/clock_tracker.cc",
"src/trace_processor/importers/common/clock_tracker.h",
"src/trace_processor/importers/common/deobfuscation_mapping_table.cc",
@@ -1767,6 +1769,7 @@
perfetto_filegroup(
name = "src_trace_processor_prelude_functions_functions",
srcs = [
+ "src/trace_processor/prelude/functions/clock_functions.h",
"src/trace_processor/prelude/functions/create_function.cc",
"src/trace_processor/prelude/functions/create_function.h",
"src/trace_processor/prelude/functions/create_function_internal.cc",
diff --git a/src/trace_processor/importers/common/BUILD.gn b/src/trace_processor/importers/common/BUILD.gn
index 7795398..71e28b3 100644
--- a/src/trace_processor/importers/common/BUILD.gn
+++ b/src/trace_processor/importers/common/BUILD.gn
@@ -23,6 +23,8 @@
"async_track_set_tracker.cc",
"async_track_set_tracker.h",
"chunked_trace_reader.h",
+ "clock_converter.cc",
+ "clock_converter.h",
"clock_tracker.cc",
"clock_tracker.h",
"deobfuscation_mapping_table.cc",
@@ -87,6 +89,7 @@
sources = [
"args_translation_table_unittest.cc",
"async_track_set_tracker_unittest.cc",
+ "clock_converter_unittest.cc",
"clock_tracker_unittest.cc",
"deobfuscation_mapping_table_unittest.cc",
"event_tracker_unittest.cc",
diff --git a/src/trace_processor/importers/common/clock_converter.cc b/src/trace_processor/importers/common/clock_converter.cc
new file mode 100644
index 0000000..908340f
--- /dev/null
+++ b/src/trace_processor/importers/common/clock_converter.cc
@@ -0,0 +1,116 @@
+/*
+ * Copyright (C) 2022 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/importers/common/clock_converter.h"
+
+#include <time.h>
+
+#include <algorithm>
+#include <atomic>
+#include <cinttypes>
+#include <queue>
+
+#include "perfetto/base/logging.h"
+#include "perfetto/ext/base/hash.h"
+#include "src/trace_processor/storage/trace_storage.h"
+#include "src/trace_processor/types/trace_processor_context.h"
+
+#include "protos/perfetto/common/builtin_clock.pbzero.h"
+#include "protos/perfetto/trace/clock_snapshot.pbzero.h"
+
+namespace perfetto {
+namespace trace_processor {
+
+ClockConverter::ClockConverter(TraceProcessorContext* context)
+ : context_(context) {}
+
+void ClockConverter::MaybeInitialize() {
+ if (is_initialized)
+ return;
+
+ is_initialized = true;
+ timelines_.Insert(kRealClock, {});
+ timelines_.Insert(kMonoClock, {});
+ for (auto it = context_->storage->clock_snapshot_table().IterateRows(); it;
+ ++it) {
+ if (it.clock_id() == kRealClock || it.clock_id() == kMonoClock)
+ timelines_.Find(it.clock_id())->emplace(it.ts(), it.clock_value());
+ }
+}
+
+base::StatusOr<ClockConverter::Timestamp> ClockConverter::FromTraceTime(
+ ClockId clock_id,
+ Timestamp ts) {
+ MaybeInitialize();
+
+ Timeline* timeline = timelines_.Find(clock_id);
+ if (!timeline) {
+ return base::ErrStatus(
+ "Provided clock has not been found in the converter "
+ "clocks.");
+ }
+
+ if (timeline->empty()) {
+ return base::ErrStatus("Target clock is not in the trace.");
+ }
+
+ auto next_snapshot = timeline->lower_bound(ts);
+
+ // If lower bound was not found, it means that the ts was higher then the last
+ // one. If that's the case we look for thhe last element and return clock
+ // value for this + offset.
+ if (next_snapshot == timeline->end()) {
+ next_snapshot--;
+ return next_snapshot->second + ts - next_snapshot->first;
+ }
+
+ // If there is a snapshot with this ts or lower bound is the first snapshot,
+ // we have no other option then to return the clock value for this snapshot.
+ if (next_snapshot == timeline->begin() || next_snapshot->first == ts)
+ return next_snapshot->second;
+
+ auto prev_snapshot = next_snapshot;
+ prev_snapshot--;
+
+ // The most truthful way to calculate the clock value is to use this formula,
+ // as there is no reason to assume that the clock is monotonistic. This
+ // prevents us from going back in time.
+ return std::min(prev_snapshot->second + ts - prev_snapshot->first,
+ next_snapshot->second);
+}
+
+std::string ClockConverter::TimeToStr(Timestamp ts) {
+ constexpr int64_t one_second_in_ns = 1LL * 1000LL * 1000LL * 1000LL;
+ int64_t s = ts / one_second_in_ns;
+ int64_t ns = ts % one_second_in_ns;
+
+ time_t time_s = static_cast<time_t>(s);
+ struct tm* time_tm = gmtime(&time_s);
+
+ int seconds = time_tm->tm_sec;
+ int minutes = time_tm->tm_min;
+ int hours = time_tm->tm_hour;
+ int day = time_tm->tm_mday;
+ int month = time_tm->tm_mon + 1;
+ int year = time_tm->tm_year + 1900;
+
+ base::StackString<64> buf("%04d-%02d-%02dT%02d:%02d:%02d.%09" PRId64, year,
+ month, day, hours, minutes, seconds, ns);
+ return buf.ToStdString();
+}
+
+} // namespace trace_processor
+} // namespace perfetto
diff --git a/src/trace_processor/importers/common/clock_converter.h b/src/trace_processor/importers/common/clock_converter.h
new file mode 100644
index 0000000..a5662b4
--- /dev/null
+++ b/src/trace_processor/importers/common/clock_converter.h
@@ -0,0 +1,91 @@
+/*
+ * Copyright (C) 2023 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_IMPORTERS_COMMON_CLOCK_CONVERTER_H_
+#define SRC_TRACE_PROCESSOR_IMPORTERS_COMMON_CLOCK_CONVERTER_H_
+
+#include <stdint.h>
+
+#include <array>
+#include <cinttypes>
+#include <map>
+#include <random>
+#include <set>
+#include <vector>
+
+#include "perfetto/base/logging.h"
+#include "perfetto/ext/base/status_or.h"
+#include "perfetto/ext/base/string_utils.h"
+#include "src/trace_processor/storage/trace_storage.h"
+#include "src/trace_processor/types/trace_processor_context.h"
+
+#include "protos/perfetto/common/builtin_clock.pbzero.h"
+#include "protos/perfetto/trace/clock_snapshot.pbzero.h"
+
+namespace perfetto {
+namespace trace_processor {
+
+// Used for conversion to REAL and MONO clocks for provided timestamps. Can only
+// be used after trace parsing. Only works if there has been at least one
+// snapshot with a target clock. Data is based on clock snapshots table.
+class ClockConverter {
+ public:
+ using ClockId = int64_t;
+ using Timestamp = int64_t;
+
+ explicit ClockConverter(TraceProcessorContext*);
+
+ // Converts trace time to REAL clock as string.
+ base::StatusOr<std::string> ToAbsTime(Timestamp ts) {
+ base::StatusOr<Timestamp> real_ts = FromTraceTime(kRealClock, ts);
+ if (!real_ts.ok())
+ return real_ts.status();
+ return TimeToStr(*real_ts);
+ }
+
+ // Converts trace time to MONO clock time.
+ base::StatusOr<Timestamp> ToMonotonic(Timestamp ts) {
+ return FromTraceTime(kMonoClock, ts);
+ }
+
+ private:
+ static constexpr int64_t kRealClock =
+ protos::pbzero::ClockSnapshot::Clock::REALTIME;
+ static constexpr int64_t kMonoClock = protos::pbzero::BUILTIN_CLOCK_MONOTONIC;
+
+ // Timeline uses Trace Time clock as keys and other clocks time as values.
+ using Timeline = std::map<Timestamp, Timestamp>;
+
+ // Reads the clocks snapshots table and fetches the data required for
+ // conversion. We initialize timelines of only selected clocks to minimize
+ // memory usage. Currently those are MONO and REAL clocks.
+ void MaybeInitialize();
+
+ // Converts trace time to provided clock.
+ base::StatusOr<Timestamp> FromTraceTime(ClockId, Timestamp);
+
+ // Converts timestamp to string.
+ std::string TimeToStr(Timestamp);
+
+ TraceProcessorContext* context_;
+ bool is_initialized = false;
+ base::FlatHashMap<ClockId, Timeline> timelines_;
+};
+
+} // namespace trace_processor
+} // namespace perfetto
+
+#endif // SRC_TRACE_PROCESSOR_IMPORTERS_COMMON_CLOCK_CONVERTER_H_
diff --git a/src/trace_processor/importers/common/clock_converter_unittest.cc b/src/trace_processor/importers/common/clock_converter_unittest.cc
new file mode 100644
index 0000000..f88fd13
--- /dev/null
+++ b/src/trace_processor/importers/common/clock_converter_unittest.cc
@@ -0,0 +1,142 @@
+/*
+ * Copyright (C) 2023 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/importers/common/clock_converter.h"
+#include "src/trace_processor/importers/common/clock_tracker.h"
+
+#include <random>
+
+#include "src/trace_processor/importers/common/metadata_tracker.h"
+#include "src/trace_processor/storage/trace_storage.h"
+#include "src/trace_processor/types/trace_processor_context.h"
+#include "test/gtest_and_gmock.h"
+
+namespace perfetto {
+namespace trace_processor {
+
+class ClockConverterTest : public ::testing::Test {
+ public:
+ ClockConverterTest() { context_.storage.reset(new TraceStorage()); }
+
+ TraceProcessorContext context_;
+ ClockConverter cc_{&context_};
+};
+
+namespace {
+
+using ::testing::NiceMock;
+using Clock = protos::pbzero::ClockSnapshot::Clock;
+
+static constexpr int64_t kMonotonic =
+ protos::pbzero::BuiltinClock::BUILTIN_CLOCK_MONOTONIC;
+static constexpr int64_t kReal = protos::pbzero::ClockSnapshot::Clock::REALTIME;
+
+TEST_F(ClockConverterTest, EmptyTable) {
+ EXPECT_FALSE(cc_.ToAbsTime(10).ok());
+ EXPECT_FALSE(cc_.ToMonotonic(10).ok());
+}
+
+TEST_F(ClockConverterTest, TrivialMonotonic) {
+ tables::ClockSnapshotTable::Row row;
+ row.ts = 10;
+ row.clock_id = kMonotonic;
+ row.clock_value = 20;
+ context_.storage->mutable_clock_snapshot_table()->Insert(row);
+
+ EXPECT_TRUE(cc_.ToMonotonic(10).ok());
+ EXPECT_EQ(cc_.ToMonotonic(10).value(), 20);
+}
+
+TEST_F(ClockConverterTest, TrivialToAbsTime) {
+ tables::ClockSnapshotTable::Row row;
+ row.ts = 10;
+ row.clock_id = kReal;
+ row.clock_value = 20;
+ context_.storage->mutable_clock_snapshot_table()->Insert(row);
+
+ EXPECT_TRUE(cc_.ToAbsTime(10).ok());
+ EXPECT_EQ(cc_.ToAbsTime(10).value(), "1970-01-01T00:00:00.000000020");
+}
+
+TEST_F(ClockConverterTest, Monotonic) {
+ {
+ tables::ClockSnapshotTable::Row rows;
+ rows.ts = 10;
+ rows.clock_id = kMonotonic;
+ rows.clock_value = 10;
+ context_.storage->mutable_clock_snapshot_table()->Insert(rows);
+ }
+ {
+ tables::ClockSnapshotTable::Row rows;
+ rows.ts = 20;
+ rows.clock_id = kMonotonic;
+ rows.clock_value = 10;
+ context_.storage->mutable_clock_snapshot_table()->Insert(rows);
+ }
+ {
+ tables::ClockSnapshotTable::Row rows;
+ rows.ts = 30;
+ rows.clock_id = kMonotonic;
+ rows.clock_value = 20;
+ context_.storage->mutable_clock_snapshot_table()->Insert(rows);
+ }
+ {
+ tables::ClockSnapshotTable::Row rows;
+ rows.ts = 40;
+ rows.clock_id = kMonotonic;
+ rows.clock_value = 20;
+ context_.storage->mutable_clock_snapshot_table()->Insert(rows);
+ }
+
+ EXPECT_EQ(cc_.ToMonotonic(15).value(), 10);
+ EXPECT_EQ(cc_.ToMonotonic(25).value(), 15);
+ EXPECT_EQ(cc_.ToMonotonic(35).value(), 20);
+ EXPECT_EQ(cc_.ToMonotonic(45).value(), 25);
+}
+
+TEST_F(ClockConverterTest, AbsTime) {
+ // We will add 3 snapshots for real time clock, and the last snapshot will be
+ // earlier then the second one.
+ {
+ tables::ClockSnapshotTable::Row rows;
+ rows.ts = 10;
+ rows.clock_id = kReal;
+ rows.clock_value = 0;
+ context_.storage->mutable_clock_snapshot_table()->Insert(rows);
+ }
+ {
+ tables::ClockSnapshotTable::Row rows;
+ rows.ts = 20;
+ rows.clock_id = kReal;
+ rows.clock_value = 1652904000000000000;
+ context_.storage->mutable_clock_snapshot_table()->Insert(rows);
+ }
+ {
+ tables::ClockSnapshotTable::Row rows;
+ rows.ts = 30;
+ rows.clock_id = kReal;
+ rows.clock_value = 1652904000000000000 - 5;
+ context_.storage->mutable_clock_snapshot_table()->Insert(rows);
+ }
+
+ EXPECT_EQ(cc_.ToAbsTime(15).value(), "1970-01-01T00:00:00.000000005");
+ EXPECT_EQ(cc_.ToAbsTime(25).value(), "2022-05-18T19:59:59.999999995");
+ EXPECT_EQ(cc_.ToAbsTime(35).value(), "2022-05-18T20:00:00.000000000");
+}
+
+} // namespace
+} // namespace trace_processor
+} // namespace perfetto
diff --git a/src/trace_processor/importers/common/clock_tracker.cc b/src/trace_processor/importers/common/clock_tracker.cc
index 987739c..3dbc8d9 100644
--- a/src/trace_processor/importers/common/clock_tracker.cc
+++ b/src/trace_processor/importers/common/clock_tracker.cc
@@ -42,7 +42,7 @@
ClockTracker::~ClockTracker() = default;
-uint32_t ClockTracker::AddSnapshot(
+base::StatusOr<uint32_t> ClockTracker::AddSnapshot(
const std::vector<ClockTimestamp>& clock_timestamps) {
const auto snapshot_id = cur_snapshot_id_++;
@@ -60,15 +60,16 @@
for (const auto& clock_ts : clock_timestamps) {
ClockId clock_id = clock_ts.clock.id;
ClockDomain& domain = clocks_[clock_id];
+
if (domain.snapshots.empty()) {
if (clock_ts.clock.is_incremental &&
!IsConvertedSequenceClock(clock_id)) {
- PERFETTO_ELOG("Clock sync error: the global clock with id=%" PRIu64
- " cannot use incremental encoding; this is only "
- "supported for sequence-scoped clocks.",
- clock_id);
context_->storage->IncrementStats(stats::invalid_clock_snapshots);
- return snapshot_id;
+ return base::ErrStatus(
+ "Clock sync error: the global clock with id=%" PRIu64
+ " cannot use incremental encoding; this is only "
+ "supported for sequence-scoped clocks.",
+ clock_id);
}
domain.unit_multiplier_ns = clock_ts.clock.unit_multiplier_ns;
domain.is_incremental = clock_ts.clock.is_incremental;
@@ -76,15 +77,15 @@
clock_ts.clock.unit_multiplier_ns ||
domain.is_incremental !=
clock_ts.clock.is_incremental)) {
- PERFETTO_ELOG("Clock sync error: the clock domain with id=%" PRIu64
- " (unit=%" PRIu64
- ", incremental=%d), was previously registered with "
- "different properties (unit=%" PRIu64 ", incremental=%d).",
- clock_id, clock_ts.clock.unit_multiplier_ns,
- clock_ts.clock.is_incremental, domain.unit_multiplier_ns,
- domain.is_incremental);
context_->storage->IncrementStats(stats::invalid_clock_snapshots);
- return snapshot_id;
+ return base::ErrStatus(
+ "Clock sync error: the clock domain with id=%" PRIu64
+ " (unit=%" PRIu64
+ ", incremental=%d), was previously registered with "
+ "different properties (unit=%" PRIu64 ", incremental=%d).",
+ clock_id, clock_ts.clock.unit_multiplier_ns,
+ clock_ts.clock.is_incremental, domain.unit_multiplier_ns,
+ domain.is_incremental);
}
const int64_t timestamp_ns = clock_ts.timestamp * domain.unit_multiplier_ns;
domain.last_timestamp_ns = timestamp_ns;
@@ -92,11 +93,11 @@
ClockSnapshots& vect = domain.snapshots[snapshot_hash];
if (!vect.snapshot_ids.empty() &&
PERFETTO_UNLIKELY(vect.snapshot_ids.back() == snapshot_id)) {
- PERFETTO_ELOG("Clock sync error: duplicate clock domain with id=%" PRIu64
- " at snapshot %" PRIu32 ".",
- clock_id, snapshot_id);
context_->storage->IncrementStats(stats::invalid_clock_snapshots);
- return snapshot_id;
+ return base::ErrStatus(
+ "Clock sync error: duplicate clock domain with id=%" PRIu64
+ " at snapshot %" PRIu32 ".",
+ clock_id, snapshot_id);
}
// Clock ids in the range [64, 128) are sequence-scoped and must be
@@ -113,17 +114,16 @@
// Clock is not monotonic.
if (clock_id == trace_time_clock_id_) {
- // The trace clock cannot be non-monotonic.
- PERFETTO_ELOG("Clock sync error: the trace clock (id=%" PRIu64
- ") is not monotonic at snapshot %" PRIu32 ". %" PRId64
- " not >= %" PRId64 ".",
- clock_id, snapshot_id, timestamp_ns,
- vect.timestamps_ns.back());
context_->storage->IncrementStats(stats::invalid_clock_snapshots);
- return snapshot_id;
+ // The trace clock cannot be non-monotonic.
+ return base::ErrStatus("Clock sync error: the trace clock (id=%" PRIu64
+ ") is not monotonic at snapshot %" PRIu32
+ ". %" PRId64 " not >= %" PRId64 ".",
+ clock_id, snapshot_id, timestamp_ns,
+ vect.timestamps_ns.back());
}
- PERFETTO_DLOG("Detected non-monotonic clock with ID %" PRIu64, clock_id);
+ PERFETTO_DLOG("Detected non-monotonic clock with ID %lu", clock_id);
// For the other clocks the best thing we can do is mark it as
// non-monotonic and refuse to use it as a source clock in the resolution
@@ -164,6 +164,7 @@
graph_.emplace(it2->clock.id, it1->clock.id, snapshot_hash);
}
}
+
return snapshot_id;
}
@@ -214,6 +215,20 @@
return ClockPath(); // invalid path.
}
+std::optional<int64_t> ClockTracker::ToTraceTimeFromSnapshot(
+ const std::vector<ClockTimestamp>& snapshot) {
+ auto maybe_found_trace_time_clock = std::find_if(
+ snapshot.begin(), snapshot.end(),
+ [this](const ClockTimestamp& clock_timestamp) {
+ return clock_timestamp.clock.id == this->trace_time_clock_id_;
+ });
+
+ if (maybe_found_trace_time_clock == snapshot.end())
+ return std::nullopt;
+
+ return maybe_found_trace_time_clock->timestamp;
+}
+
base::StatusOr<int64_t> ClockTracker::ConvertSlowpath(ClockId src_clock_id,
int64_t src_timestamp,
ClockId target_clock_id) {
@@ -223,6 +238,7 @@
context_->storage->IncrementStats(stats::clock_sync_cache_miss);
ClockPath path = FindPath(src_clock_id, target_clock_id);
+
if (!path.valid()) {
// Too many logs maybe emitted when path is invalid.
context_->storage->IncrementStats(stats::clock_sync_failure);
@@ -316,34 +332,5 @@
return ns;
}
-base::StatusOr<std::string> ClockTracker::FromTraceTimeAsISO8601(
- int64_t timestamp) {
- constexpr ClockId unix_epoch_clock =
- protos::pbzero::ClockSnapshot::Clock::REALTIME;
- base::StatusOr<int64_t> opt_ts = FromTraceTime(unix_epoch_clock, timestamp);
- if (!opt_ts.ok()) {
- return opt_ts.status();
- }
- int64_t ts = opt_ts.value();
-
- constexpr int64_t one_second_in_ns = 1LL * 1000LL * 1000LL * 1000LL;
- int64_t s = ts / one_second_in_ns;
- int64_t ns = ts % one_second_in_ns;
-
- time_t time_s = static_cast<time_t>(s);
- struct tm* time_tm = gmtime(&time_s);
-
- int seconds = time_tm->tm_sec;
- int minutes = time_tm->tm_min;
- int hours = time_tm->tm_hour;
- int day = time_tm->tm_mday;
- int month = time_tm->tm_mon + 1;
- int year = time_tm->tm_year + 1900;
-
- base::StackString<64> buf("%04d-%02d-%02dT%02d:%02d:%02d.%09" PRId64, year,
- month, day, hours, minutes, seconds, ns);
- return buf.ToStdString();
-}
-
} // namespace trace_processor
} // namespace perfetto
diff --git a/src/trace_processor/importers/common/clock_tracker.h b/src/trace_processor/importers/common/clock_tracker.h
index a4edd3c..a812593 100644
--- a/src/trace_processor/importers/common/clock_tracker.h
+++ b/src/trace_processor/importers/common/clock_tracker.h
@@ -34,6 +34,9 @@
#include "src/trace_processor/storage/trace_storage.h"
#include "src/trace_processor/types/trace_processor_context.h"
+#include "protos/perfetto/common/builtin_clock.pbzero.h"
+#include "protos/perfetto/trace/clock_snapshot.pbzero.h"
+
namespace perfetto {
namespace trace_processor {
@@ -49,8 +52,6 @@
// been snapshotted at the same time (within technical limits).
// - ToTraceTime(src_clock_id, src_timestamp):
// converts a timestamp between clock domain and TraceTime.
-// - FromTraceTime(target_clock_id, src_timestamp):
-// converts a timestamp between TraceTime and clock domain.
//
// Concepts:
// - Snapshot hash:
@@ -161,7 +162,7 @@
// Appends a new snapshot for the given clock domains.
// This is typically called by the code that reads the ClockSnapshot packet.
// Returns the internal snapshot id of this set of clocks.
- uint32_t AddSnapshot(const std::vector<ClockTimestamp>&);
+ base::StatusOr<uint32_t> AddSnapshot(const std::vector<ClockTimestamp>&);
base::StatusOr<int64_t> ToTraceTime(ClockId clock_id, int64_t timestamp) {
if (PERFETTO_UNLIKELY(!trace_time_clock_id_used_for_conversion_)) {
@@ -176,15 +177,10 @@
return Convert(clock_id, timestamp, trace_time_clock_id_);
}
- base::StatusOr<int64_t> FromTraceTime(ClockId to_clock_id,
- int64_t timestamp) {
- trace_time_clock_id_used_for_conversion_ = true;
- if (to_clock_id == trace_time_clock_id_)
- return timestamp;
- return Convert(trace_time_clock_id_, timestamp, to_clock_id);
- }
-
- base::StatusOr<std::string> FromTraceTimeAsISO8601(int64_t timestamp);
+ // If trace clock and source clock are available in the snapshot will return
+ // the trace clock time in snapshot.
+ std::optional<int64_t> ToTraceTimeFromSnapshot(
+ const std::vector<ClockTimestamp>&);
void SetTraceTimeClock(ClockId clock_id) {
PERFETTO_DCHECK(!IsSequenceClock(clock_id));
diff --git a/src/trace_processor/importers/common/clock_tracker_unittest.cc b/src/trace_processor/importers/common/clock_tracker_unittest.cc
index 74a665f..717592f 100644
--- a/src/trace_processor/importers/common/clock_tracker_unittest.cc
+++ b/src/trace_processor/importers/common/clock_tracker_unittest.cc
@@ -87,6 +87,15 @@
static_cast<int64_t>(100000 - 1000 + 1e6));
}
+TEST_F(ClockTrackerTest, ToTraceTimeFromSnapshot) {
+ EXPECT_FALSE(ct_.ToTraceTime(REALTIME, 0).ok());
+
+ EXPECT_EQ(*ct_.ToTraceTimeFromSnapshot({{REALTIME, 10}, {BOOTTIME, 10010}}),
+ 10010);
+ EXPECT_EQ(ct_.ToTraceTimeFromSnapshot({{MONOTONIC, 10}, {REALTIME, 10010}}),
+ std::nullopt);
+}
+
// When a clock moves backwards conversions *from* that clock are forbidden
// but conversions *to* that clock should still work.
// Think to the case of REALTIME going backwards from 3AM to 2AM during DST day.
@@ -105,16 +114,16 @@
// Now only BOOTIME -> REALTIME conversion should be possible.
EXPECT_FALSE(ct_.ToTraceTime(REALTIME, 11).ok());
- EXPECT_EQ(*ct_.FromTraceTime(REALTIME, 10011), 11);
- EXPECT_EQ(*ct_.FromTraceTime(REALTIME, 10029), 29);
- EXPECT_EQ(*ct_.FromTraceTime(REALTIME, 40030), 30);
- EXPECT_EQ(*ct_.FromTraceTime(REALTIME, 40040), 40);
+ EXPECT_EQ(*Convert(BOOTTIME, 10011, REALTIME), 11);
+ EXPECT_EQ(*Convert(BOOTTIME, 10029, REALTIME), 29);
+ EXPECT_EQ(*Convert(BOOTTIME, 40030, REALTIME), 30);
+ EXPECT_EQ(*Convert(BOOTTIME, 40040, REALTIME), 40);
ct_.AddSnapshot({{BOOTTIME, 50000}, {REALTIME, 50}});
- EXPECT_EQ(*ct_.FromTraceTime(REALTIME, 50005), 55);
+ EXPECT_EQ(*Convert(BOOTTIME, 50005, REALTIME), 55);
ct_.AddSnapshot({{BOOTTIME, 60020}, {REALTIME, 20}});
- EXPECT_EQ(*ct_.FromTraceTime(REALTIME, 60020), 20);
+ EXPECT_EQ(*Convert(BOOTTIME, 60020, REALTIME), 20);
}
// Simulate the following scenario:
@@ -294,16 +303,6 @@
}
}
-TEST_F(ClockTrackerTest, FromTraceTimeAsISO8601) {
- EXPECT_FALSE(ct_.FromTraceTimeAsISO8601(0).ok());
-
- ct_.AddSnapshot({{REALTIME, 1603224822123456789}, {BOOTTIME, 42}});
- ct_.AddSnapshot({{REALTIME, 1641092645000000001}, {BOOTTIME, 43}});
-
- EXPECT_EQ(*ct_.FromTraceTimeAsISO8601(42), "2020-10-20T20:13:42.123456789");
- EXPECT_EQ(*ct_.FromTraceTimeAsISO8601(43), "2022-01-02T03:04:05.000000001");
-}
-
} // namespace
} // namespace trace_processor
} // namespace perfetto
diff --git a/src/trace_processor/importers/proto/proto_trace_reader.cc b/src/trace_processor/importers/proto/proto_trace_reader.cc
index f947c0f..003f06c 100644
--- a/src/trace_processor/importers/proto/proto_trace_reader.cc
+++ b/src/trace_processor/importers/proto/proto_trace_reader.cc
@@ -362,7 +362,15 @@
clk.is_incremental());
}
- uint32_t snapshot_id = context_->clock_tracker->AddSnapshot(clock_timestamps);
+ base::StatusOr<uint32_t> snapshot_id =
+ context_->clock_tracker->AddSnapshot(clock_timestamps);
+ if (!snapshot_id.ok()) {
+ PERFETTO_ELOG("%s", snapshot_id.status().c_message());
+ return base::OkStatus();
+ }
+
+ std::optional<int64_t> trace_time_from_snapshot =
+ context_->clock_tracker->ToTraceTimeFromSnapshot(clock_timestamps);
// Add the all the clock snapshots to the clock snapshot table.
std::optional<int64_t> trace_ts_for_check;
@@ -371,13 +379,19 @@
// |absolute_timestamp|.
int64_t ts_to_convert =
clock_timestamp.clock.is_incremental ? 0 : clock_timestamp.timestamp;
+ // Even if we have trace time from snapshot, we still run ToTraceTime to
+ // optimise future conversions.
base::StatusOr<int64_t> opt_trace_ts = context_->clock_tracker->ToTraceTime(
clock_timestamp.clock.id, ts_to_convert);
+
if (!opt_trace_ts.ok()) {
- // This can happen if |AddSnapshot| failed to resolve this clock. Just
- // ignore this and move on.
- PERFETTO_DLOG("%s", opt_trace_ts.status().c_message());
- continue;
+ // This can happen if |AddSnapshot| failed to resolve this clock, e.g. if
+ // clock is not monotonic. Try to fetch trace time from snapshot.
+ if (!trace_time_from_snapshot) {
+ PERFETTO_DLOG("%s", opt_trace_ts.status().c_message());
+ continue;
+ }
+ opt_trace_ts = *trace_time_from_snapshot;
}
// Double check that all the clocks in this snapshot resolve to the same
@@ -391,7 +405,7 @@
row.clock_id = static_cast<int64_t>(clock_timestamp.clock.id);
row.clock_value = clock_timestamp.timestamp;
row.clock_name = GetBuiltinClockNameOrNull(clock_timestamp.clock.id);
- row.snapshot_id = snapshot_id;
+ row.snapshot_id = *snapshot_id;
context_->storage->mutable_clock_snapshot_table()->Insert(row);
}
diff --git a/src/trace_processor/prelude/functions/BUILD.gn b/src/trace_processor/prelude/functions/BUILD.gn
index 02cb90f..ec4da44 100644
--- a/src/trace_processor/prelude/functions/BUILD.gn
+++ b/src/trace_processor/prelude/functions/BUILD.gn
@@ -18,6 +18,7 @@
source_set("functions") {
sources = [
+ "clock_functions.h",
"create_function.cc",
"create_function.h",
"create_function_internal.cc",
@@ -47,6 +48,7 @@
"../../../../gn:sqlite",
"../../../../include/perfetto/protozero:protozero",
"../../../../protos/perfetto/common:zero",
+ "../../../../protos/perfetto/trace:zero",
"../../../../protos/perfetto/trace/ftrace:zero",
"../../../../protos/perfetto/trace_processor:zero",
"../../../base",
diff --git a/src/trace_processor/prelude/functions/clock_functions.h b/src/trace_processor/prelude/functions/clock_functions.h
new file mode 100644
index 0000000..34f23f2
--- /dev/null
+++ b/src/trace_processor/prelude/functions/clock_functions.h
@@ -0,0 +1,118 @@
+/*
+ * Copyright (C) 2022 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_PRELUDE_FUNCTIONS_CLOCK_FUNCTIONS_H_
+#define SRC_TRACE_PROCESSOR_PRELUDE_FUNCTIONS_CLOCK_FUNCTIONS_H_
+
+#include <sqlite3.h>
+#include <unordered_map>
+#include "perfetto/ext/base/base64.h"
+#include "protos/perfetto/common/builtin_clock.pbzero.h"
+#include "src/trace_processor/importers/common/clock_converter.h"
+#include "src/trace_processor/prelude/functions/create_function_internal.h"
+#include "src/trace_processor/util/status_macros.h"
+
+#include "src/trace_processor/prelude/functions/register_function.h"
+
+namespace perfetto {
+namespace trace_processor {
+
+struct AbsTimeStr : public SqlFunction {
+ using Context = ClockConverter;
+ static base::Status Run(ClockConverter* tracker,
+ size_t argc,
+ sqlite3_value** argv,
+ SqlValue& out,
+ Destructors& destructors);
+};
+
+base::Status AbsTimeStr::Run(ClockConverter* tracker,
+ size_t argc,
+ sqlite3_value** argv,
+ SqlValue& out,
+ Destructors& destructors) {
+ if (argc != 1) {
+ return base::ErrStatus("ABS_TIME_STR: 1 arg required");
+ }
+
+ // If the timestamp is null, just return null as the result.
+ if (sqlite3_value_type(argv[0]) == SQLITE_NULL) {
+ return base::OkStatus();
+ }
+ if (sqlite3_value_type(argv[0]) != SQLITE_INTEGER) {
+ return base::ErrStatus("ABS_TIME_STR: first argument should be timestamp");
+ }
+
+ int64_t ts = sqlite3_value_int64(argv[0]);
+ base::StatusOr<std::string> iso8601 = tracker->ToAbsTime(ts);
+ if (!iso8601.ok()) {
+ // We are returning an OkStatus, because one bad timestamp shouldn't stop
+ // the query.
+ return base::OkStatus();
+ }
+
+ std::unique_ptr<char, base::FreeDeleter> s(
+ static_cast<char*>(malloc(iso8601->size() + 1)));
+ memcpy(s.get(), iso8601->c_str(), iso8601->size() + 1);
+
+ destructors.string_destructor = free;
+ out = SqlValue::String(s.release());
+ return base::OkStatus();
+}
+
+struct ToMonotonic : public SqlFunction {
+ using Context = ClockConverter;
+ static base::Status Run(ClockConverter* tracker,
+ size_t argc,
+ sqlite3_value** argv,
+ SqlValue& out,
+ Destructors& destructors);
+};
+
+base::Status ToMonotonic::Run(ClockConverter* tracker,
+ size_t argc,
+ sqlite3_value** argv,
+ SqlValue& out,
+ Destructors&) {
+ if (argc != 1) {
+ return base::ErrStatus("TO_MONOTONIC: 1 arg required");
+ }
+
+ // If the timestamp is null, just return null as the result.
+ if (sqlite3_value_type(argv[0]) == SQLITE_NULL) {
+ return base::OkStatus();
+ }
+ if (sqlite3_value_type(argv[0]) != SQLITE_INTEGER) {
+ return base::ErrStatus("TO_MONOTONIC: first argument should be timestamp");
+ }
+
+ int64_t ts = sqlite3_value_int64(argv[0]);
+ base::StatusOr<int64_t> monotonic = tracker->ToMonotonic(ts);
+
+ if (!monotonic.ok()) {
+ // We are returning an OkStatus, because one bad timestamp shouldn't stop
+ // the query.
+ return base::OkStatus();
+ }
+
+ out = SqlValue::Long(*monotonic);
+ return base::OkStatus();
+}
+
+} // namespace trace_processor
+} // namespace perfetto
+
+#endif // SRC_TRACE_PROCESSOR_PRELUDE_FUNCTIONS_CLOCK_FUNCTIONS_H_
diff --git a/src/trace_processor/prelude/functions/utils.h b/src/trace_processor/prelude/functions/utils.h
index c4c5392..82b293f 100644
--- a/src/trace_processor/prelude/functions/utils.h
+++ b/src/trace_processor/prelude/functions/utils.h
@@ -294,89 +294,6 @@
PERFETTO_FATAL("For GCC");
}
-struct AbsTimeStr : public SqlFunction {
- using Context = ClockTracker;
- static base::Status Run(ClockTracker* tracker,
- size_t argc,
- sqlite3_value** argv,
- SqlValue& out,
- Destructors& destructors);
-};
-
-base::Status AbsTimeStr::Run(ClockTracker* tracker,
- size_t argc,
- sqlite3_value** argv,
- SqlValue& out,
- Destructors& destructors) {
- if (argc != 1) {
- return base::ErrStatus("ABS_TIME_STR: 1 arg required");
- }
-
- // If the timestamp is null, just return null as the result.
- if (sqlite3_value_type(argv[0]) == SQLITE_NULL) {
- return base::OkStatus();
- }
- if (sqlite3_value_type(argv[0]) != SQLITE_INTEGER) {
- return base::ErrStatus("ABS_TIME_STR: first argument should be timestamp");
- }
-
- int64_t ts = sqlite3_value_int64(argv[0]);
- base::StatusOr<std::string> iso8601 = tracker->FromTraceTimeAsISO8601(ts);
- if (!iso8601.ok()) {
- // We are returning an OkStatus, because one bad timestamp shouldn't stop
- // the query.
- return base::OkStatus();
- }
-
- std::unique_ptr<char, base::FreeDeleter> s(
- static_cast<char*>(malloc(iso8601->size() + 1)));
- memcpy(s.get(), iso8601->c_str(), iso8601->size() + 1);
-
- destructors.string_destructor = free;
- out = SqlValue::String(s.release());
- return base::OkStatus();
-}
-
-struct ToMonotonic : public SqlFunction {
- using Context = ClockTracker;
- static base::Status Run(ClockTracker* tracker,
- size_t argc,
- sqlite3_value** argv,
- SqlValue& out,
- Destructors& destructors);
-};
-
-base::Status ToMonotonic::Run(ClockTracker* tracker,
- size_t argc,
- sqlite3_value** argv,
- SqlValue& out,
- Destructors&) {
- if (argc != 1) {
- return base::ErrStatus("TO_MONOTONIC: 1 arg required");
- }
-
- // If the timestamp is null, just return null as the result.
- if (sqlite3_value_type(argv[0]) == SQLITE_NULL) {
- return base::OkStatus();
- }
- if (sqlite3_value_type(argv[0]) != SQLITE_INTEGER) {
- return base::ErrStatus("TO_MONOTONIC: first argument should be timestamp");
- }
-
- int64_t ts = sqlite3_value_int64(argv[0]);
- base::StatusOr<int64_t> monotonic =
- tracker->FromTraceTime(protos::pbzero::BUILTIN_CLOCK_MONOTONIC, ts);
-
- if (!monotonic.ok()) {
- // We are returning an OkStatus, because one bad timestamp shouldn't stop
- // the query.
- return base::OkStatus();
- }
-
- out = SqlValue::Long(*monotonic);
- return base::OkStatus();
-}
-
struct SourceGeq : public SqlFunction {
static base::Status Run(void*,
size_t,
diff --git a/src/trace_processor/trace_processor_context.cc b/src/trace_processor/trace_processor_context.cc
index b407e19..34da487 100644
--- a/src/trace_processor/trace_processor_context.cc
+++ b/src/trace_processor/trace_processor_context.cc
@@ -21,6 +21,7 @@
#include "src/trace_processor/importers/common/args_translation_table.h"
#include "src/trace_processor/importers/common/async_track_set_tracker.h"
#include "src/trace_processor/importers/common/chunked_trace_reader.h"
+#include "src/trace_processor/importers/common/clock_converter.h"
#include "src/trace_processor/importers/common/clock_tracker.h"
#include "src/trace_processor/importers/common/deobfuscation_mapping_table.h"
#include "src/trace_processor/importers/common/event_tracker.h"
diff --git a/src/trace_processor/trace_processor_impl.cc b/src/trace_processor/trace_processor_impl.cc
index 742d7a2..68fbf51 100644
--- a/src/trace_processor/trace_processor_impl.cc
+++ b/src/trace_processor/trace_processor_impl.cc
@@ -32,6 +32,7 @@
#include "perfetto/ext/base/string_utils.h"
#include "perfetto/trace_processor/basic_types.h"
#include "src/trace_processor/importers/android_bugreport/android_bugreport_parser.h"
+#include "src/trace_processor/importers/common/clock_converter.h"
#include "src/trace_processor/importers/common/clock_tracker.h"
#include "src/trace_processor/importers/common/metadata_tracker.h"
#include "src/trace_processor/importers/ftrace/sched_event_tracker.h"
@@ -46,6 +47,7 @@
#include "src/trace_processor/importers/proto/content_analyzer.h"
#include "src/trace_processor/importers/systrace/systrace_trace_parser.h"
#include "src/trace_processor/iterator_impl.h"
+#include "src/trace_processor/prelude/functions/clock_functions.h"
#include "src/trace_processor/prelude/functions/create_function.h"
#include "src/trace_processor/prelude/functions/create_view_function.h"
#include "src/trace_processor/prelude/functions/import.h"
@@ -703,9 +705,9 @@
false);
RegisterFunction<ExtractArg>(db, "EXTRACT_ARG", 2, context_.storage.get());
RegisterFunction<AbsTimeStr>(db, "ABS_TIME_STR", 1,
- context_.clock_tracker.get());
+ context_.clock_converter.get());
RegisterFunction<ToMonotonic>(db, "TO_MONOTONIC", 1,
- context_.clock_tracker.get());
+ context_.clock_converter.get());
RegisterFunction<CreateFunction>(
db, "CREATE_FUNCTION", 3,
std::unique_ptr<CreateFunction::Context>(
diff --git a/src/trace_processor/trace_processor_storage_impl.cc b/src/trace_processor/trace_processor_storage_impl.cc
index 75031f8..7b79bff 100644
--- a/src/trace_processor/trace_processor_storage_impl.cc
+++ b/src/trace_processor/trace_processor_storage_impl.cc
@@ -22,6 +22,7 @@
#include "src/trace_processor/importers/common/args_tracker.h"
#include "src/trace_processor/importers/common/args_translation_table.h"
#include "src/trace_processor/importers/common/async_track_set_tracker.h"
+#include "src/trace_processor/importers/common/clock_converter.h"
#include "src/trace_processor/importers/common/clock_tracker.h"
#include "src/trace_processor/importers/common/event_tracker.h"
#include "src/trace_processor/importers/common/flow_tracker.h"
@@ -61,6 +62,7 @@
context_.event_tracker.reset(new EventTracker(&context_));
context_.process_tracker.reset(new ProcessTracker(&context_));
context_.clock_tracker.reset(new ClockTracker(&context_));
+ context_.clock_converter.reset(new ClockConverter(&context_));
context_.heap_profile_tracker.reset(new HeapProfileTracker(&context_));
context_.perf_sample_tracker.reset(new PerfSampleTracker(&context_));
context_.global_stack_profile_tracker.reset(new GlobalStackProfileTracker());
@@ -149,7 +151,7 @@
TraceProcessorContext context;
context.storage = std::move(context_.storage);
context.heap_graph_tracker = std::move(context_.heap_graph_tracker);
- context.clock_tracker = std::move(context_.clock_tracker);
+ context.clock_converter = std::move(context_.clock_converter);
// "to_ftrace" textual converter of the "raw" table requires remembering the
// kernel version (inside system_info_tracker) to know how to textualise
// sched_switch.prev_state bitflags.
diff --git a/src/trace_processor/types/trace_processor_context.h b/src/trace_processor/types/trace_processor_context.h
index 12aa16b..de942da 100644
--- a/src/trace_processor/types/trace_processor_context.h
+++ b/src/trace_processor/types/trace_processor_context.h
@@ -44,6 +44,7 @@
class AndroidProbesTracker;
class ChunkedTraceReader;
class ClockTracker;
+class ClockConverter;
class DeobfuscationMappingTable;
class EventTracker;
class ForwardingTraceParser;
@@ -97,6 +98,7 @@
std::unique_ptr<ProcessTracker> process_tracker;
std::unique_ptr<EventTracker> event_tracker;
std::unique_ptr<ClockTracker> clock_tracker;
+ std::unique_ptr<ClockConverter> clock_converter;
std::unique_ptr<HeapProfileTracker> heap_profile_tracker;
std::unique_ptr<PerfSampleTracker> perf_sample_tracker;
std::unique_ptr<GlobalStackProfileTracker> global_stack_profile_tracker;
diff --git a/test/data/ui-screenshots/ui-chrome_rendering_desktop_select_slice_with_flows.png.sha256 b/test/data/ui-screenshots/ui-chrome_rendering_desktop_select_slice_with_flows.png.sha256
index ca4c662..d657179 100644
--- a/test/data/ui-screenshots/ui-chrome_rendering_desktop_select_slice_with_flows.png.sha256
+++ b/test/data/ui-screenshots/ui-chrome_rendering_desktop_select_slice_with_flows.png.sha256
@@ -1 +1 @@
-9c416089e3ae480d5f2f2807d0607949903dc83a01da998c5930c49f66c92859
\ No newline at end of file
+92aa98e5edad0293244bc0a94672a2e9b63e9b6aaa86ab0a58db8b106e912693
\ No newline at end of file
diff --git a/test/trace_processor/diff_tests/dynamic/tests.py b/test/trace_processor/diff_tests/dynamic/tests.py
index 24589ee..408e36b 100644
--- a/test/trace_processor/diff_tests/dynamic/tests.py
+++ b/test/trace_processor/diff_tests/dynamic/tests.py
@@ -113,7 +113,10 @@
ABS_TIME_STR(25) AS t25,
ABS_TIME_STR(35) AS t35;
""",
- out=Path('various_clocks_abs_time_str.out'))
+ out=Csv("""
+ "t15","t25","t35"
+ "1970-01-01T00:00:00.000000005","2022-05-18T19:59:59.999999995","2022-05-18T20:00:00.000000000"
+ """))
def test_empty_abs_time_str(self):
return DiffTestBlueprint(
diff --git a/test/trace_processor/diff_tests/dynamic/various_clocks_abs_time_str.out b/test/trace_processor/diff_tests/dynamic/various_clocks_abs_time_str.out
deleted file mode 100644
index de34644..0000000
--- a/test/trace_processor/diff_tests/dynamic/various_clocks_abs_time_str.out
+++ /dev/null
@@ -1,2 +0,0 @@
-"t15","t25","t35"
-"1970-01-01T00:00:00.000000005","2022-05-18T20:00:00.000000005","2022-05-18T20:00:00.000000000"