Trace Redaction - Add synth threads to process tree
In order to support thread merging, the new threads must be added to
the process tree. These ensure the threads appear in the UI correctly.
To support this, the process tree primitive has been re-implemented
using the same strategy pattern used elsewhere. There are two
extensions points, allowing:
1. filter process
2. filter thread
3. add processes
4. add threads
Bug: 336807771
Change-Id: I1291fde00ce0c15f89ed78e7425a0c9f9bc40555
diff --git a/Android.bp b/Android.bp
index d807b27..487e4dc 100644
--- a/Android.bp
+++ b/Android.bp
@@ -13618,11 +13618,11 @@
"src/trace_redaction/prune_package_list.cc",
"src/trace_redaction/redact_ftrace_event.cc",
"src/trace_redaction/redact_process_events.cc",
+ "src/trace_redaction/redact_process_trees.cc",
"src/trace_redaction/redact_sched_events.cc",
"src/trace_redaction/remap_scheduling_events.cc",
"src/trace_redaction/scrub_ftrace_events.cc",
"src/trace_redaction/scrub_process_stats.cc",
- "src/trace_redaction/scrub_process_trees.cc",
"src/trace_redaction/scrub_trace_packet.cc",
"src/trace_redaction/suspend_resume.cc",
"src/trace_redaction/trace_redaction_framework.cc",
diff --git a/src/trace_redaction/BUILD.gn b/src/trace_redaction/BUILD.gn
index fdff971..35b571f 100644
--- a/src/trace_redaction/BUILD.gn
+++ b/src/trace_redaction/BUILD.gn
@@ -57,6 +57,8 @@
"redact_ftrace_event.h",
"redact_process_events.cc",
"redact_process_events.h",
+ "redact_process_trees.cc",
+ "redact_process_trees.h",
"redact_sched_events.cc",
"redact_sched_events.h",
"remap_scheduling_events.cc",
@@ -65,8 +67,6 @@
"scrub_ftrace_events.h",
"scrub_process_stats.cc",
"scrub_process_stats.h",
- "scrub_process_trees.cc",
- "scrub_process_trees.h",
"scrub_trace_packet.cc",
"scrub_trace_packet.h",
"suspend_resume.cc",
@@ -100,11 +100,11 @@
"filter_sched_waking_events_integrationtest.cc",
"filter_task_rename_integrationtest.cc",
"prune_package_list_integrationtest.cc",
+ "redact_process_trees_integrationtest.cc",
"redact_sched_events_integrationtest.cc",
"remap_scheduling_events_integrationtest.cc",
"scrub_ftrace_events_integrationtest.cc",
"scrub_process_stats_integrationtest.cc",
- "scrub_process_trees_integrationtest.cc",
"trace_redaction_integration_fixture.cc",
"trace_redaction_integration_fixture.h",
"verify_integrity_integrationtest.cc",
diff --git a/src/trace_redaction/main.cc b/src/trace_redaction/main.cc
index 3707be1..851b58f 100644
--- a/src/trace_redaction/main.cc
+++ b/src/trace_redaction/main.cc
@@ -27,11 +27,11 @@
#include "src/trace_redaction/prune_package_list.h"
#include "src/trace_redaction/redact_ftrace_event.h"
#include "src/trace_redaction/redact_process_events.h"
+#include "src/trace_redaction/redact_process_trees.h"
#include "src/trace_redaction/redact_sched_events.h"
#include "src/trace_redaction/remap_scheduling_events.h"
#include "src/trace_redaction/scrub_ftrace_events.h"
#include "src/trace_redaction/scrub_process_stats.h"
-#include "src/trace_redaction/scrub_process_trees.h"
#include "src/trace_redaction/scrub_trace_packet.h"
#include "src/trace_redaction/suspend_resume.h"
#include "src/trace_redaction/trace_redaction_framework.h"
@@ -77,7 +77,6 @@
// Scrub packets and ftrace events first as they will remove the largest
// chucks of data from the trace. This will reduce the amount of data that the
// other primitives need to operate on.
- redactor.emplace_transform<ScrubProcessTrees>();
redactor.emplace_transform<PrunePackageList>();
redactor.emplace_transform<ScrubProcessStats>();
@@ -105,6 +104,15 @@
//
// Add these primitives back one-by-one to find the issue.
+ // Configure the primitive to remove processes and threads that don't belong
+ // to the target package and adds a process and threads for the synth thread
+ // group and threads.
+ {
+ auto* primitive = redactor.emplace_transform<RedactProcessTrees>();
+ primitive->emplace_modifier<ProcessTreeCreateSynthThreads>();
+ primitive->emplace_filter<ProcessTreeFilterConnectedToPackage>();
+ }
+
Context context;
context.package_name = package_name;
diff --git a/src/trace_redaction/redact_process_trees.cc b/src/trace_redaction/redact_process_trees.cc
new file mode 100644
index 0000000..c72cdc6
--- /dev/null
+++ b/src/trace_redaction/redact_process_trees.cc
@@ -0,0 +1,208 @@
+/*
+ * Copyright (C) 2024 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_redaction/redact_process_trees.h"
+
+#include <string>
+
+#include "perfetto/base/status.h"
+#include "perfetto/protozero/field.h"
+#include "perfetto/protozero/scattered_heap_buffer.h"
+#include "src/trace_processor/util/status_macros.h"
+#include "src/trace_redaction/proto_util.h"
+#include "src/trace_redaction/trace_redaction_framework.h"
+
+#include "protos/perfetto/trace/ps/process_tree.pbzero.h"
+#include "protos/perfetto/trace/trace_packet.pbzero.h"
+
+namespace perfetto::trace_redaction {
+
+ProcessTreeFilter::~ProcessTreeFilter() = default;
+
+ProcessTreeModifier::~ProcessTreeModifier() = default;
+
+bool ProcessTreeFilterAllowAll::Filter(const Context&,
+ uint64_t,
+ int32_t) const {
+ return true;
+}
+
+bool ProcessTreeFilterConnectedToPackage::Filter(const Context& context,
+ uint64_t ts,
+ int32_t pid) const {
+ PERFETTO_DCHECK(context.timeline);
+ PERFETTO_DCHECK(context.package_uid.has_value());
+
+ // Checking the uid directly is an option, but the timeline handles multiple
+ // instances of the same app.
+ return context.timeline->PidConnectsToUid(ts, pid, *context.package_uid);
+}
+
+base::Status ProcessTreeDoNothing::Modify(const Context&,
+ protos::pbzero::ProcessTree*) const {
+ return base::OkStatus();
+}
+
+base::Status ProcessTreeCreateSynthThreads::Modify(
+ const Context& context,
+ protos::pbzero::ProcessTree* message) const {
+ PERFETTO_DCHECK(message);
+
+ if (!context.synthetic_threads.has_value()) {
+ return base::ErrStatus(
+ "ProcessTreeCreateSynthThreads: missing synthetic thread group");
+ }
+
+ // There should be one thread per cpu. If there are no threads, it means there
+ // were no cpus. That wrong.
+ if (context.synthetic_threads->tids.empty()) {
+ return base::ErrStatus(
+ "ProcessTreeCreateSynthThreads: missing synthetic threads");
+ }
+
+ // uid 0 and ppid 2 means it is a system process.
+ auto* process = message->add_processes();
+ process->set_uid(0);
+ process->set_ppid(2);
+ process->set_pid(context.synthetic_threads->tgid);
+ process->add_cmdline("MergedThreads");
+
+ for (auto tid : context.synthetic_threads->tids) {
+ auto* thread = message->add_threads();
+ thread->set_tgid(context.synthetic_threads->tgid);
+ thread->set_tid(tid);
+ thread->set_name("CPU");
+ }
+
+ return base::OkStatus();
+}
+
+base::Status RedactProcessTrees::Transform(const Context& context,
+ std::string* packet) const {
+ PERFETTO_DCHECK(packet);
+
+ if (!context.package_uid.has_value()) {
+ return base::ErrStatus("RedactProcessTrees: missing package uid.");
+ }
+
+ if (!context.timeline) {
+ return base::ErrStatus("RedactProcessTrees: missing timeline.");
+ }
+
+ if (!context.synthetic_threads.has_value()) {
+ return base::ErrStatus("RedactProcessTrees: missing synthentic threads.");
+ }
+
+ protozero::ProtoDecoder decoder(*packet);
+
+ auto tree =
+ decoder.FindField(protos::pbzero::TracePacket::kProcessTreeFieldNumber);
+
+ if (!tree.valid()) {
+ return base::OkStatus();
+ }
+
+ // This has been verified by the verify primitive.
+ auto timestamp =
+ decoder.FindField(protos::pbzero::TracePacket::kTimestampFieldNumber);
+
+ protozero::HeapBuffered<protos::pbzero::TracePacket> message;
+
+ for (auto it = decoder.ReadField(); it.valid(); it = decoder.ReadField()) {
+ if (it.id() == tree.id()) {
+ RETURN_IF_ERROR(OnProcessTree(context, timestamp.as_uint64(),
+ it.as_bytes(),
+ message->set_process_tree()));
+ } else {
+ proto_util::AppendField(it, message.get());
+ }
+ }
+
+ packet->assign(message.SerializeAsString());
+
+ return base::OkStatus();
+}
+
+base::Status RedactProcessTrees::OnProcessTree(
+ const Context& context,
+ uint64_t ts,
+ protozero::ConstBytes bytes,
+ protos::pbzero::ProcessTree* message) const {
+ protozero::ProtoDecoder decoder(bytes);
+
+ for (auto it = decoder.ReadField(); it.valid(); it = decoder.ReadField()) {
+ switch (it.id()) {
+ case protos::pbzero::ProcessTree::kProcessesFieldNumber:
+ RETURN_IF_ERROR(OnProcess(context, ts, it, message));
+ break;
+ case protos::pbzero::ProcessTree::kThreadsFieldNumber:
+ RETURN_IF_ERROR(OnThread(context, ts, it, message));
+ break;
+ default:
+ proto_util::AppendField(it, message);
+ break;
+ }
+ }
+
+ PERFETTO_DCHECK(modifier_);
+ return modifier_->Modify(context, message);
+}
+
+base::Status RedactProcessTrees::OnProcess(
+ const Context& context,
+ uint64_t ts,
+ protozero::Field field,
+ protos::pbzero::ProcessTree* message) const {
+ protozero::ProtoDecoder decoder(field.as_bytes());
+
+ auto pid =
+ decoder.FindField(protos::pbzero::ProcessTree::Process::kPidFieldNumber);
+ if (!pid.valid()) {
+ return base::ErrStatus("RedactProcessTrees: process with no pid");
+ }
+
+ PERFETTO_DCHECK(filter_);
+
+ if (filter_->Filter(context, ts, pid.as_int32())) {
+ proto_util::AppendField(field, message);
+ }
+
+ return base::OkStatus();
+}
+
+base::Status RedactProcessTrees::OnThread(
+ const Context& context,
+ uint64_t ts,
+ protozero::Field field,
+ protos::pbzero::ProcessTree* message) const {
+ protozero::ProtoDecoder decoder(field.as_bytes());
+
+ auto tid =
+ decoder.FindField(protos::pbzero::ProcessTree::Thread::kTidFieldNumber);
+ if (!tid.valid()) {
+ return base::ErrStatus("RedactProcessTrees: thread with no tid");
+ }
+
+ PERFETTO_DCHECK(filter_);
+
+ if (filter_->Filter(context, ts, tid.as_int32())) {
+ proto_util::AppendField(field, message);
+ }
+
+ return base::OkStatus();
+}
+
+} // namespace perfetto::trace_redaction
diff --git a/src/trace_redaction/redact_process_trees.h b/src/trace_redaction/redact_process_trees.h
new file mode 100644
index 0000000..4bbd32f
--- /dev/null
+++ b/src/trace_redaction/redact_process_trees.h
@@ -0,0 +1,107 @@
+/*
+ * Copyright (C) 2024 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_REDACTION_REDACT_PROCESS_TREES_H_
+#define SRC_TRACE_REDACTION_REDACT_PROCESS_TREES_H_
+
+#include "perfetto/base/status.h"
+#include "perfetto/protozero/field.h"
+#include "src/trace_redaction/trace_redaction_framework.h"
+
+#include "protos/perfetto/trace/ps/process_tree.pbzero.h"
+
+namespace perfetto::trace_redaction {
+
+class ProcessTreeFilter {
+ public:
+ virtual ~ProcessTreeFilter();
+ virtual bool Filter(const Context& context,
+ uint64_t ts,
+ int32_t pid) const = 0;
+};
+
+class ProcessTreeModifier {
+ public:
+ virtual ~ProcessTreeModifier();
+ virtual base::Status Modify(const Context& context,
+ protos::pbzero::ProcessTree* message) const = 0;
+};
+
+class ProcessTreeFilterAllowAll : public ProcessTreeFilter {
+ public:
+ bool Filter(const Context& context, uint64_t ts, int32_t pid) const override;
+};
+
+class ProcessTreeFilterConnectedToPackage : public ProcessTreeFilter {
+ public:
+ bool Filter(const Context& context, uint64_t ts, int32_t pid) const override;
+};
+
+class ProcessTreeDoNothing : public ProcessTreeModifier {
+ public:
+ base::Status Modify(const Context& context,
+ protos::pbzero::ProcessTree* message) const override;
+};
+
+class ProcessTreeCreateSynthThreads : public ProcessTreeModifier {
+ public:
+ base::Status Modify(const Context& context,
+ protos::pbzero::ProcessTree* message) const override;
+};
+
+// Removes threads and processes from the process tree based on whether or not
+// they are connected to the target package.
+class RedactProcessTrees : public TransformPrimitive {
+ public:
+ base::Status Transform(const Context& context,
+ std::string* packet) const override;
+
+ template <class Filter>
+ void emplace_filter() {
+ filter_ = std::make_unique<Filter>();
+ }
+
+ template <class Builder>
+ void emplace_modifier() {
+ modifier_ = std::make_unique<Builder>();
+ }
+
+ private:
+ base::Status OnProcessTree(const Context& context,
+ uint64_t ts,
+ protozero::ConstBytes bytes,
+ protos::pbzero::ProcessTree* message) const;
+
+ base::Status OnProcess(const Context& context,
+ uint64_t ts,
+ protozero::Field field,
+ protos::pbzero::ProcessTree* message) const;
+
+ base::Status OnThread(const Context& context,
+ uint64_t ts,
+ protozero::Field field,
+ protos::pbzero::ProcessTree* message) const;
+
+ base::Status AppendSynthThreads(const Context& context,
+ protos::pbzero::ProcessTree* message) const;
+
+ std::unique_ptr<ProcessTreeFilter> filter_;
+ std::unique_ptr<ProcessTreeModifier> modifier_;
+};
+
+} // namespace perfetto::trace_redaction
+
+#endif // SRC_TRACE_REDACTION_REDACT_PROCESS_TREES_H_
diff --git a/src/trace_redaction/redact_process_trees_integrationtest.cc b/src/trace_redaction/redact_process_trees_integrationtest.cc
new file mode 100644
index 0000000..2b517d0
--- /dev/null
+++ b/src/trace_redaction/redact_process_trees_integrationtest.cc
@@ -0,0 +1,241 @@
+/*
+ * Copyright (C) 2024 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 <string>
+#include <string_view>
+#include <vector>
+
+#include "src/base/test/status_matchers.h"
+#include "src/trace_redaction/collect_system_info.h"
+#include "src/trace_redaction/collect_timeline_events.h"
+#include "src/trace_redaction/find_package_uid.h"
+#include "src/trace_redaction/redact_process_trees.h"
+#include "src/trace_redaction/trace_redaction_framework.h"
+#include "src/trace_redaction/trace_redaction_integration_fixture.h"
+#include "src/trace_redaction/trace_redactor.h"
+#include "test/gtest_and_gmock.h"
+
+#include "protos/perfetto/trace/ps/process_tree.pbzero.h"
+#include "protos/perfetto/trace/trace.pbzero.h"
+
+namespace perfetto::trace_redaction {
+
+namespace {
+
+constexpr std::string_view kProcessName =
+ "com.Unity.com.unity.multiplayer.samples.coop";
+
+} // namespace
+
+class RedactProcessTreesIntegrationTest
+ : public testing::Test,
+ protected TraceRedactionIntegrationFixure {
+ protected:
+ void SetUp() override {
+ trace_redactor()->emplace_collect<CollectSystemInfo>();
+ trace_redactor()->emplace_build<BuildSyntheticThreads>();
+
+ trace_redactor()->emplace_collect<FindPackageUid>();
+ trace_redactor()->emplace_collect<CollectTimelineEvents>();
+
+ // Filter the process tree based on whether or not a process is part of the
+ // target package.
+ auto* process_tree =
+ trace_redactor()->emplace_transform<RedactProcessTrees>();
+ process_tree->emplace_modifier<ProcessTreeDoNothing>();
+ process_tree->emplace_filter<ProcessTreeFilterConnectedToPackage>();
+
+ // In this case, the process and package have the same name.
+ context()->package_name = kProcessName;
+ }
+
+ std::unordered_set<int32_t> GetPids(const std::string& bytes) const {
+ std::unordered_set<int32_t> pids;
+
+ protos::pbzero::Trace::Decoder decoder(bytes);
+
+ for (auto it = decoder.packet(); it; ++it) {
+ protos::pbzero::TracePacket::Decoder packet(*it);
+
+ if (packet.has_process_tree()) {
+ GetPids(packet.process_tree(), &pids);
+ }
+ }
+
+ return pids;
+ }
+
+ std::unordered_set<int32_t> GetTids(const std::string& bytes) const {
+ std::unordered_set<int32_t> tids;
+
+ protos::pbzero::Trace::Decoder decoder(bytes);
+
+ for (auto it = decoder.packet(); it; ++it) {
+ protos::pbzero::TracePacket::Decoder packet(*it);
+
+ if (packet.has_process_tree()) {
+ GetTids(packet.process_tree(), &tids);
+ }
+ }
+
+ return tids;
+ }
+
+ private:
+ void GetPids(protozero::ConstBytes bytes,
+ std::unordered_set<int32_t>* pids) const {
+ protos::pbzero::ProcessTree::Decoder process_tree(bytes);
+
+ for (auto it = process_tree.processes(); it; ++it) {
+ protos::pbzero::ProcessTree::Process::Decoder process(*it);
+ pids->insert(process.pid());
+ }
+ }
+
+ void GetTids(protozero::ConstBytes bytes,
+ std::unordered_set<int32_t>* tids) const {
+ protos::pbzero::ProcessTree::Decoder process_tree(bytes);
+
+ for (auto it = process_tree.threads(); it; ++it) {
+ protos::pbzero::ProcessTree::Thread::Decoder thread(*it);
+ tids->insert(thread.tid());
+ }
+ }
+};
+
+TEST_F(RedactProcessTreesIntegrationTest, FilterProcesses) {
+ ASSERT_OK(Redact());
+
+ auto original_trace_str = LoadOriginal();
+ ASSERT_OK(original_trace_str);
+
+ auto redacted_trace_str = LoadRedacted();
+ ASSERT_OK(redacted_trace_str);
+
+ auto original_pids = GetPids(*original_trace_str);
+ auto redacted_pids = GetPids(*redacted_trace_str);
+
+ // There are 902 unique pids across all process trees:
+ // grep 'processes {' -A 1 src.pftrace.txt | grep 'pid: ' | grep -Po "\d+"
+ // | sort | uniq | wc -l
+ //
+ // But if ppids are included, there are 903 pids in the process tree:
+ // grep 'processes {' -A 2 src.pftrace.txt | grep 'pid: ' | grep -Po "\d+"
+ // | sort | uniq | wc -l
+ //
+ // The above grep statements use a stringified version of the trace. Using "-A
+ // 1" will return the pid line. Using "-A 2" will include both pid and ppid.
+ //
+ // The original process count aligns with trace processor. However, the
+ // redacted count does not. The final tree has one process but trace processor
+ // reports 4 processes.
+ ASSERT_EQ(original_pids.size(), 902u);
+ ASSERT_EQ(redacted_pids.size(), 1u);
+
+ ASSERT_TRUE(redacted_pids.count(7105));
+}
+
+TEST_F(RedactProcessTreesIntegrationTest, FilterThreads) {
+ ASSERT_OK(Redact());
+
+ auto original_trace_str = LoadOriginal();
+ ASSERT_OK(original_trace_str);
+
+ auto redacted_trace_str = LoadRedacted();
+ ASSERT_OK(redacted_trace_str);
+
+ auto original_tids = GetTids(*original_trace_str);
+ auto redacted_tids = GetTids(*redacted_trace_str);
+
+ // There are 2761 unique tids across all process trees:
+ // grep 'threads {' -A 1 src.pftrace.txt | grep 'tid: ' | grep -Po "\d+" |
+ // sort | uniq | wc -l
+ //
+ // There are 2896 unique tids/tgis across all process trees:
+ // grep 'threads {' -A 2 src.pftrace.txt | grep -P '(tid|tgid): ' | grep
+ // -Po '\d+' | sort | uniq | wc -l
+ //
+ // The original tid count does NOT align with what trace processor returns.
+ // Trace processor reports 3666 threads. The assumption is trace processor is
+ // fulling thread information from additional.
+ //
+ // The redacted tid+tgid count does NOT align with what trace processor
+ // returns. Trace processor reports 199 tids where are there are only 63 tids
+ // found in process tree. This suggests that trace processor is pulling tid
+ // data from other locations.
+ ASSERT_EQ(original_tids.size(), 2761u);
+ ASSERT_EQ(redacted_tids.size(), 63u);
+}
+
+TEST_F(RedactProcessTreesIntegrationTest, AddSynthProcess) {
+ // Append another primitive that won't filter, but will add new threads. This
+ // will be compatible with the other instanced in SetUp().
+ auto* process_tree =
+ trace_redactor()->emplace_transform<RedactProcessTrees>();
+ process_tree->emplace_modifier<ProcessTreeCreateSynthThreads>();
+ process_tree->emplace_filter<ProcessTreeFilterAllowAll>();
+
+ ASSERT_OK(Redact());
+
+ auto redacted_trace_str = LoadRedacted();
+ ASSERT_OK(redacted_trace_str);
+
+ auto redacted_pids = GetPids(*redacted_trace_str);
+
+ const auto& synth_threads = context()->synthetic_threads;
+ ASSERT_TRUE(synth_threads.has_value());
+
+ ASSERT_NE(std::find(redacted_pids.begin(), redacted_pids.end(),
+ synth_threads->tgid),
+ redacted_pids.end());
+}
+
+TEST_F(RedactProcessTreesIntegrationTest, AddSynthThreads) {
+ // Append another primitive that won't filter, but will add new threads. This
+ // will be compatible with the other instanced in SetUp().
+ auto* process_tree =
+ trace_redactor()->emplace_transform<RedactProcessTrees>();
+ process_tree->emplace_modifier<ProcessTreeCreateSynthThreads>();
+ process_tree->emplace_filter<ProcessTreeFilterAllowAll>();
+
+ ASSERT_OK(Redact());
+
+ const auto& synth_threads = context()->synthetic_threads;
+ ASSERT_TRUE(synth_threads.has_value());
+ ASSERT_FALSE(synth_threads->tids.empty());
+
+ auto original_trace_str = LoadOriginal();
+ ASSERT_OK(original_trace_str);
+
+ auto original_tids = GetTids(*original_trace_str);
+
+ // The synth threads should not be found in the original trace.
+ for (auto tid : synth_threads->tids) {
+ ASSERT_FALSE(original_tids.count(tid));
+ }
+
+ auto redacted_trace_str = LoadRedacted();
+ ASSERT_OK(redacted_trace_str);
+
+ auto redacted_tids = GetTids(*redacted_trace_str);
+
+ // The synth threads should be found in the redacted trace.
+ for (auto tid : synth_threads->tids) {
+ ASSERT_TRUE(redacted_tids.count(tid));
+ }
+}
+
+} // namespace perfetto::trace_redaction
diff --git a/src/trace_redaction/scrub_process_trees.cc b/src/trace_redaction/scrub_process_trees.cc
deleted file mode 100644
index a3f1f3b..0000000
--- a/src/trace_redaction/scrub_process_trees.cc
+++ /dev/null
@@ -1,111 +0,0 @@
-/*
- * Copyright (C) 2024 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_redaction/scrub_process_trees.h"
-
-#include "perfetto/base/status.h"
-#include "perfetto/protozero/field.h"
-#include "perfetto/protozero/scattered_heap_buffer.h"
-#include "src/trace_redaction/proto_util.h"
-#include "src/trace_redaction/trace_redaction_framework.h"
-
-#include "protos/perfetto/trace/ps/process_tree.pbzero.h"
-#include "protos/perfetto/trace/trace_packet.pbzero.h"
-
-namespace perfetto::trace_redaction {
-
-namespace {
-
-// Appends a value to the message if (and only if) the pid belongs to the target
-// package.
-void TryAppendPid(const Context& context,
- const protozero::Field& timestamp,
- const protozero::Field& pid,
- const protozero::Field& value,
- protozero::Message* message) {
- // All valid processes with have a time and pid/tid values. However, if
- // they're missing values, the trace is corrupt. To avoid making this work by
- // dropping too much data, drop the cmdline for all processes.
- if (!timestamp.valid() || !pid.valid()) {
- return;
- }
-
- if (context.timeline->PidConnectsToUid(timestamp.as_uint64(), pid.as_int32(),
- *context.package_uid)) {
- proto_util::AppendField(value, message);
- }
-}
-
-} // namespace
-
-base::Status ScrubProcessTrees::VerifyContext(const Context& context) const {
- if (!context.package_uid.has_value()) {
- return base::ErrStatus("ScrubProcessTrees: missing package uid.");
- }
-
- if (!context.timeline) {
- return base::ErrStatus("ScrubProcessTrees: missing timeline.");
- }
-
- return base::OkStatus();
-}
-
-void ScrubProcessTrees::TransformProcess(
- const Context& context,
- const protozero::Field& timestamp,
- const protozero::Field& process,
- protos::pbzero::ProcessTree* process_tree) const {
- protozero::ProtoDecoder decoder(process.as_bytes());
-
- auto pid =
- decoder.FindField(protos::pbzero::ProcessTree::Process::kPidFieldNumber);
-
- auto* process_message = process_tree->add_processes();
-
- for (auto field = decoder.ReadField(); field.valid();
- field = decoder.ReadField()) {
- if (field.id() ==
- protos::pbzero::ProcessTree::Process::kCmdlineFieldNumber) {
- TryAppendPid(context, timestamp, pid, field, process_message);
- } else {
- proto_util::AppendField(field, process_message);
- }
- }
-}
-
-void ScrubProcessTrees::TransformThread(
- const Context& context,
- const protozero::Field& timestamp,
- const protozero::Field& thread,
- protos::pbzero::ProcessTree* process_tree) const {
- protozero::ProtoDecoder decoder(thread.as_bytes());
-
- auto tid =
- decoder.FindField(protos::pbzero::ProcessTree::Thread::kTidFieldNumber);
-
- auto* thread_message = process_tree->add_threads();
-
- for (auto field = decoder.ReadField(); field.valid();
- field = decoder.ReadField()) {
- if (field.id() == protos::pbzero::ProcessTree::Thread::kNameFieldNumber) {
- TryAppendPid(context, timestamp, tid, field, thread_message);
- } else {
- proto_util::AppendField(field, thread_message);
- }
- }
-}
-
-} // namespace perfetto::trace_redaction
diff --git a/src/trace_redaction/scrub_process_trees.h b/src/trace_redaction/scrub_process_trees.h
deleted file mode 100644
index 7c2b07a..0000000
--- a/src/trace_redaction/scrub_process_trees.h
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Copyright (C) 2024 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_REDACTION_SCRUB_PROCESS_TREES_H_
-#define SRC_TRACE_REDACTION_SCRUB_PROCESS_TREES_H_
-
-#include "perfetto/base/status.h"
-#include "perfetto/protozero/field.h"
-#include "src/trace_redaction/modify_process_trees.h"
-#include "src/trace_redaction/trace_redaction_framework.h"
-
-#include "protos/perfetto/trace/ps/process_tree.pbzero.h"
-
-namespace perfetto::trace_redaction {
-
-// Removes process names and thread names from process_trees if their pids/tids
-// are not connected to the target package.
-class ScrubProcessTrees : public ModifyProcessTree {
- protected:
- base::Status VerifyContext(const Context& context) const override;
-
- void TransformProcess(
- const Context& context,
- const protozero::Field& timestamp,
- const protozero::Field& process,
- protos::pbzero::ProcessTree* process_trees) const override;
-
- void TransformThread(
- const Context& context,
- const protozero::Field& timestamp,
- const protozero::Field& thread,
- protos::pbzero::ProcessTree* process_tree) const override;
-};
-
-} // namespace perfetto::trace_redaction
-
-#endif // SRC_TRACE_REDACTION_SCRUB_PROCESS_TREES_H_
diff --git a/src/trace_redaction/scrub_process_trees_integrationtest.cc b/src/trace_redaction/scrub_process_trees_integrationtest.cc
deleted file mode 100644
index a001a36..0000000
--- a/src/trace_redaction/scrub_process_trees_integrationtest.cc
+++ /dev/null
@@ -1,103 +0,0 @@
-/*
- * Copyright (C) 2024 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 <string>
-#include <string_view>
-#include <vector>
-
-#include "src/base/test/status_matchers.h"
-#include "src/trace_redaction/collect_timeline_events.h"
-#include "src/trace_redaction/find_package_uid.h"
-#include "src/trace_redaction/scrub_process_trees.h"
-#include "src/trace_redaction/trace_redaction_framework.h"
-#include "src/trace_redaction/trace_redaction_integration_fixture.h"
-#include "src/trace_redaction/trace_redactor.h"
-#include "test/gtest_and_gmock.h"
-
-#include "protos/perfetto/trace/ps/process_tree.pbzero.h"
-#include "protos/perfetto/trace/trace.pbzero.h"
-
-namespace perfetto::trace_redaction {
-
-namespace {
-
-constexpr std::string_view kProcessName =
- "com.Unity.com.unity.multiplayer.samples.coop";
-
-} // namespace
-
-class ScrubProcessTreesIntegrationTest
- : public testing::Test,
- protected TraceRedactionIntegrationFixure {
- protected:
- void SetUp() override {
- trace_redactor()->emplace_collect<FindPackageUid>();
- trace_redactor()->emplace_collect<CollectTimelineEvents>();
- trace_redactor()->emplace_transform<ScrubProcessTrees>();
-
- // In this case, the process and package have the same name.
- context()->package_name = kProcessName;
- }
-
- std::vector<std::string> CollectProcessNames(
- protos::pbzero::Trace::Decoder trace) const {
- std::vector<std::string> names;
-
- for (auto packet_it = trace.packet(); packet_it; ++packet_it) {
- protos::pbzero::TracePacket::Decoder packet(*packet_it);
-
- if (!packet.has_process_tree()) {
- continue;
- }
-
- protos::pbzero::ProcessTree::Decoder process_tree(packet.process_tree());
-
- for (auto process_it = process_tree.processes(); process_it;
- ++process_it) {
- protos::pbzero::ProcessTree::Process::Decoder process(*process_it);
-
- if (process.has_cmdline()) {
- names.push_back(process.cmdline()->as_std_string());
- }
- }
- }
-
- return names;
- }
-};
-
-TEST_F(ScrubProcessTreesIntegrationTest, RemovesProcessNamesFromProcessTrees) {
- ASSERT_OK(Redact());
-
- auto original_trace_str = LoadOriginal();
- ASSERT_OK(original_trace_str);
-
- auto redacted_trace_str = LoadRedacted();
- ASSERT_OK(redacted_trace_str);
-
- protos::pbzero::Trace::Decoder original_trace(original_trace_str.value());
- auto original_processes = CollectProcessNames(std::move(original_trace));
-
- ASSERT_GT(original_processes.size(), 1u);
-
- protos::pbzero::Trace::Decoder redacted_trace(redacted_trace_str.value());
- auto redacted_processes = CollectProcessNames(std::move(redacted_trace));
-
- ASSERT_EQ(redacted_processes.size(), 1u);
- ASSERT_EQ(redacted_processes.at(0), kProcessName);
-}
-
-} // namespace perfetto::trace_redaction