Merge "compact sched_waking: writer (traced_probes)"
diff --git a/src/traced/probes/ftrace/compact_sched.cc b/src/traced/probes/ftrace/compact_sched.cc
index 63b87ff..70168e9 100644
--- a/src/traced/probes/ftrace/compact_sched.cc
+++ b/src/traced/probes/ftrace/compact_sched.cc
@@ -37,12 +37,6 @@
CompactSchedSwitchFormat switch_format;
switch_format.event_id = event.ftrace_event_id;
-
- // We make a compile-time buffer capacity decision based on the expected event
- // size per a set of pages. Check that the assumption holds.
- if (event.size < CompactSchedBundleState::kMinSupportedSchedSwitchSize) {
- return base::nullopt;
- }
switch_format.size = event.size;
bool prev_state_valid = false;
@@ -81,7 +75,7 @@
next_comm_valid =
(field.ftrace_type == kFtraceFixedCString &&
- field.ftrace_size == CompactSchedBundleState::kExpectedCommLength);
+ field.ftrace_size == CommInterner::kExpectedCommLength);
break;
default:
break;
@@ -90,48 +84,112 @@
if (!prev_state_valid || !next_pid_valid || !next_prio_valid ||
!next_comm_valid) {
- PERFETTO_ELOG("unexpected sched_switch format");
return base::nullopt;
}
-
return base::make_optional(switch_format);
}
+// Pre-parse the format of sched_waking, checking if our simplifying
+// assumptions about possible widths/signedness hold, and record the subset
+// of the format that will be used during parsing.
+base::Optional<CompactSchedWakingFormat> ValidateSchedWakingFormat(
+ const Event& event) {
+ using protos::pbzero::SchedWakingFtraceEvent;
+
+ CompactSchedWakingFormat waking_format;
+ waking_format.event_id = event.ftrace_event_id;
+ waking_format.size = event.size;
+
+ bool pid_valid = false;
+ bool target_cpu_valid = false;
+ bool prio_valid = false;
+ bool comm_valid = false;
+ for (const auto& field : event.fields) {
+ switch (field.proto_field_id) {
+ case SchedWakingFtraceEvent::kPidFieldNumber:
+ waking_format.pid_offset = field.ftrace_offset;
+ waking_format.pid_type = field.ftrace_type;
+
+ // kernel type: pid_t
+ pid_valid = (field.ftrace_type == kFtracePid32);
+ break;
+
+ case SchedWakingFtraceEvent::kTargetCpuFieldNumber:
+ waking_format.target_cpu_offset = field.ftrace_offset;
+ waking_format.target_cpu_type = field.ftrace_type;
+
+ // kernel type: int
+ target_cpu_valid = (field.ftrace_type == kFtraceInt32);
+ break;
+
+ case SchedWakingFtraceEvent::kPrioFieldNumber:
+ waking_format.prio_offset = field.ftrace_offset;
+ waking_format.prio_type = field.ftrace_type;
+
+ // kernel type: int
+ prio_valid = (field.ftrace_type == kFtraceInt32);
+ break;
+
+ case SchedWakingFtraceEvent::kCommFieldNumber:
+ waking_format.comm_offset = field.ftrace_offset;
+
+ comm_valid = (field.ftrace_type == kFtraceFixedCString &&
+ field.ftrace_size == CommInterner::kExpectedCommLength);
+ break;
+ default:
+ break;
+ }
+ }
+
+ if (!pid_valid || !target_cpu_valid || !prio_valid || !comm_valid) {
+ return base::nullopt;
+ }
+ return base::make_optional(waking_format);
+}
+
} // namespace
// TODO(rsavitski): could avoid looping over all events if the caller did the
// work to remember the relevant events (translation table construction already
// loops over them).
+// TODO(rsavitski): consider tracking the validity of the formats individually,
+// so that we can e.g. still use compact_sched on a device without
+// compact_waking.
CompactSchedEventFormat ValidateFormatForCompactSched(
const std::vector<Event>& events) {
using protos::pbzero::FtraceEvent;
base::Optional<CompactSchedSwitchFormat> switch_format;
+ base::Optional<CompactSchedWakingFormat> waking_format;
for (const Event& event : events) {
if (event.proto_field_id == FtraceEvent::kSchedSwitchFieldNumber) {
switch_format = ValidateSchedSwitchFormat(event);
}
+ if (event.proto_field_id == FtraceEvent::kSchedWakingFieldNumber) {
+ waking_format = ValidateSchedWakingFormat(event);
+ }
}
- if (switch_format.has_value()) {
- return CompactSchedEventFormat{/*format_valid=*/true,
- switch_format.value()};
+ if (switch_format.has_value() && waking_format.has_value()) {
+ return CompactSchedEventFormat{/*format_valid=*/true, switch_format.value(),
+ waking_format.value()};
} else {
+ PERFETTO_ELOG("Unexpected sched_switch or sched_waking format.");
return CompactSchedEventFormat{/*format_valid=*/false,
- CompactSchedSwitchFormat{}};
+ CompactSchedSwitchFormat{},
+ CompactSchedWakingFormat{}};
}
}
CompactSchedEventFormat InvalidCompactSchedEventFormatForTesting() {
return CompactSchedEventFormat{/*format_valid=*/false,
- CompactSchedSwitchFormat{}};
+ CompactSchedSwitchFormat{},
+ CompactSchedWakingFormat{}};
}
// TODO(rsavitski): find the correct place in the trace for, and method of,
// reporting rejection of compact_sched due to compile-time assumptions not
// holding at runtime.
-// TODO(rsavitski): consider checking if the ftrace config correctly enables
-// sched_switch, for at least an informative print for now?
CompactSchedConfig CreateCompactSchedConfig(
const FtraceConfig& request,
const CompactSchedEventFormat& compact_format) {
@@ -153,37 +211,76 @@
}
// Sanity check size of stack-allocated bundle state.
-static_assert(sizeof(CompactSchedBundleState) <= 1 << 20,
- "CompactSchedBundleState excessively large (used on the stack).");
+static_assert(sizeof(CompactSchedBuffer) <= 1 << 18,
+ "CompactSchedBuffer's on-stack size excessively large.");
-void CompactSchedBundleState::WriteAndReset(
+void CompactSchedSwitchBuffer::Write(
+ protos::pbzero::FtraceEventBundle::CompactSched* compact_out) const {
+ compact_out->set_switch_timestamp(timestamp_);
+ compact_out->set_switch_next_pid(next_pid_);
+ compact_out->set_switch_prev_state(prev_state_);
+ compact_out->set_switch_next_prio(next_prio_);
+ compact_out->set_switch_next_comm_index(next_comm_index_);
+}
+
+void CompactSchedSwitchBuffer::Reset() {
+ last_timestamp_ = 0;
+ timestamp_.Reset();
+ next_pid_.Reset();
+ prev_state_.Reset();
+ next_prio_.Reset();
+ next_comm_index_.Reset();
+}
+
+void CompactSchedWakingBuffer::Write(
+ protos::pbzero::FtraceEventBundle::CompactSched* compact_out) const {
+ compact_out->set_waking_timestamp(timestamp_);
+ compact_out->set_waking_pid(pid_);
+ compact_out->set_waking_target_cpu(target_cpu_);
+ compact_out->set_waking_prio(prio_);
+ compact_out->set_waking_comm_index(comm_index_);
+}
+
+void CompactSchedWakingBuffer::Reset() {
+ last_timestamp_ = 0;
+ timestamp_.Reset();
+ pid_.Reset();
+ target_cpu_.Reset();
+ prio_.Reset();
+ comm_index_.Reset();
+}
+
+void CommInterner::Write(
+ protos::pbzero::FtraceEventBundle::CompactSched* compact_out) const {
+ for (size_t i = 0; i < interned_comms_size_; i++) {
+ compact_out->add_intern_table(interned_comms_[i].data(),
+ interned_comms_[i].size());
+ }
+}
+
+void CommInterner::Reset() {
+ intern_buf_write_pos_ = 0;
+ interned_comms_size_ = 0;
+}
+
+void CompactSchedBuffer::WriteAndReset(
protos::pbzero::FtraceEventBundle* bundle) {
- // If we buffered at least one event (using the interner as a proxy),
- // write the state out.
- if (interned_switch_comms_size_ > 0) {
- auto compact_out = bundle->set_compact_sched();
+ if (switch_.size() > 0 || waking_.size() > 0) {
+ auto* compact_out = bundle->set_compact_sched();
- compact_out->set_switch_timestamp(switch_timestamp_);
- compact_out->set_switch_next_pid(switch_next_pid_);
- compact_out->set_switch_prev_state(switch_prev_state_);
- compact_out->set_switch_next_prio(switch_next_prio_);
+ PERFETTO_DCHECK(interner_.interned_comms_size() > 0);
+ interner_.Write(compact_out);
- for (size_t i = 0; i < interned_switch_comms_size_; i++) {
- compact_out->add_intern_table(interned_switch_comms_[i].data(),
- interned_switch_comms_[i].size());
- }
- compact_out->set_switch_next_comm_index(switch_next_comm_index_);
+ if (switch_.size() > 0)
+ switch_.Write(compact_out);
+
+ if (waking_.size() > 0)
+ waking_.Write(compact_out);
}
- // Reset internal state.
- last_switch_timestamp_ = 0;
- switch_timestamp_.Reset();
- switch_next_pid_.Reset();
- switch_prev_state_.Reset();
- switch_next_prio_.Reset();
- switch_next_comm_index_.Reset();
- intern_buf_write_pos_ = 0;
- interned_switch_comms_size_ = 0;
+ interner_.Reset();
+ switch_.Reset();
+ waking_.Reset();
}
} // namespace perfetto
diff --git a/src/traced/probes/ftrace/compact_sched.h b/src/traced/probes/ftrace/compact_sched.h
index 6dc4692..657d651 100644
--- a/src/traced/probes/ftrace/compact_sched.h
+++ b/src/traced/probes/ftrace/compact_sched.h
@@ -28,7 +28,7 @@
class FtraceConfig;
-// The subset of the sched_switch event's format that is used when parsing &
+// The subset of the sched_switch event's format that is used when parsing and
// encoding into the compact format.
struct CompactSchedSwitchFormat {
uint32_t event_id;
@@ -43,6 +43,21 @@
uint16_t next_comm_offset;
};
+// The subset of the sched_waking event's format that is used when parsing and
+// encoding into the compact format.
+struct CompactSchedWakingFormat {
+ uint32_t event_id;
+ uint16_t size;
+
+ uint16_t pid_offset;
+ FtraceFieldType pid_type;
+ uint16_t target_cpu_offset;
+ FtraceFieldType target_cpu_type;
+ uint16_t prio_offset;
+ FtraceFieldType prio_type;
+ uint16_t comm_offset;
+};
+
// Pre-parsed format of a subset of scheduling events, for use during ftrace
// parsing if compact encoding is enabled. Holds a flag, |format_valid| to
// state whether the compile-time assumptions about the format held at runtime.
@@ -50,7 +65,9 @@
struct CompactSchedEventFormat {
// If false, the rest of the struct is considered invalid.
const bool format_valid;
+
const CompactSchedSwitchFormat sched_switch;
+ const CompactSchedWakingFormat sched_waking;
};
CompactSchedEventFormat ValidateFormatForCompactSched(
@@ -62,8 +79,8 @@
struct CompactSchedConfig {
CompactSchedConfig(bool _enabled) : enabled(_enabled) {}
- // If true, and sched_switch event is enabled, encode it in a compact format
- // instead of the normal form.
+ // If true, and sched_switch and/or sched_waking events are enabled, encode
+ // them in a compact format instead of the normal form.
const bool enabled = false;
};
@@ -74,97 +91,145 @@
CompactSchedConfig EnabledCompactSchedConfigForTesting();
CompactSchedConfig DisabledCompactSchedConfigForTesting();
-// Mutable state for buffering parts of scheduling events, that can later be
-// written out in a compact format with |WriteAndReset|. Used by the ftrace
-// reader, allocated on the stack.
-class CompactSchedBundleState {
+// Collects fields of sched_switch events, allowing them to be written out
+// in a compact encoding.
+class CompactSchedSwitchBuffer {
public:
- // Most of the state is stack-allocated, with a compile-time
- // size. We work in batches of pages (see kParsingBufferSizePages in
- // ftrace_controller.cc), and assume a minimum size of a sched event as
- // written by the kernel (validated at runtime). We therefore can calculate
- // the maximum necessary capacity for a given parsing buffer size (as
- // statically asserted in ftrace_controller.cc).
- // Note: be careful not to align the individual buffers at a multiple of the
- // cache size.
- // TODO(rsavitski): this will need a slight rework once we add sched_waking,
- // as it'll be the min size of the two events.
- static constexpr size_t kMaxElements = 2560;
- static constexpr size_t kMinSupportedSchedSwitchSize = 56;
+ protozero::PackedVarInt& timestamp() { return timestamp_; }
+ protozero::PackedVarInt& prev_state() { return prev_state_; }
+ protozero::PackedVarInt& next_pid() { return next_pid_; }
+ protozero::PackedVarInt& next_prio() { return next_prio_; }
+ protozero::PackedVarInt& next_comm_index() { return next_comm_index_; }
+
+ size_t size() const {
+ // Caller should fill all per-field buffers at the same rate.
+ return timestamp_.size();
+ }
+
+ inline void AppendTimestamp(uint64_t timestamp) {
+ timestamp_.Append(timestamp - last_timestamp_);
+ last_timestamp_ = timestamp;
+ }
+
+ void Write(
+ protos::pbzero::FtraceEventBundle::CompactSched* compact_out) const;
+ void Reset();
+
+ private:
+ // First timestamp in a bundle is absolute. The rest are all delta-encoded,
+ // each relative to the preceding sched_switch timestamp.
+ uint64_t last_timestamp_ = 0;
+
+ protozero::PackedVarInt timestamp_;
+ protozero::PackedVarInt prev_state_;
+ protozero::PackedVarInt next_pid_;
+ protozero::PackedVarInt next_prio_;
+ // Interning indices of the next_comm values. See |CommInterner|.
+ protozero::PackedVarInt next_comm_index_;
+};
+
+// As |CompactSchedSwitchBuffer|, but for sched_waking events.
+class CompactSchedWakingBuffer {
+ public:
+ protozero::PackedVarInt& pid() { return pid_; }
+ protozero::PackedVarInt& target_cpu() { return target_cpu_; }
+ protozero::PackedVarInt& prio() { return prio_; }
+ protozero::PackedVarInt& comm_index() { return comm_index_; }
+
+ size_t size() const {
+ // Caller should fill all per-field buffers at the same rate.
+ return timestamp_.size();
+ }
+
+ inline void AppendTimestamp(uint64_t timestamp) {
+ timestamp_.Append(timestamp - last_timestamp_);
+ last_timestamp_ = timestamp;
+ }
+
+ void Write(
+ protos::pbzero::FtraceEventBundle::CompactSched* compact_out) const;
+ void Reset();
+
+ private:
+ uint64_t last_timestamp_ = 0;
+
+ protozero::PackedVarInt timestamp_;
+ protozero::PackedVarInt pid_;
+ protozero::PackedVarInt target_cpu_;
+ protozero::PackedVarInt prio_;
+ // Interning indices of the comm values. See |CommInterner|.
+ protozero::PackedVarInt comm_index_;
+};
+
+class CommInterner {
+ public:
static constexpr size_t kExpectedCommLength = 16;
- protozero::PackedVarInt* switch_timestamp() { return &switch_timestamp_; }
-
- protozero::PackedVarInt* switch_prev_state() { return &switch_prev_state_; }
-
- protozero::PackedVarInt* switch_next_pid() { return &switch_next_pid_; }
-
- protozero::PackedVarInt* switch_next_prio() { return &switch_next_prio_; }
-
- size_t interned_switch_comms_size() const {
- return interned_switch_comms_size_;
- }
-
- inline void AppendSwitchTimestamp(uint64_t timestamp) {
- switch_timestamp_.Append(timestamp - last_switch_timestamp_);
- last_switch_timestamp_ = timestamp;
- }
-
- // TODO(rsavitski): see if we can use the fact that comms are <16 bytes
- // long when comparing them.
- void InternSwitchNextComm(const char* ptr) {
+ size_t InternComm(const char* ptr) {
// Linearly scan existing string views, ftrace reader will
// make sure this set doesn't grow too large.
base::StringView transient_view(ptr);
- for (size_t i = 0; i < interned_switch_comms_size_; i++) {
- if (transient_view == interned_switch_comms_[i]) {
- switch_next_comm_index_.Append(i);
- return;
+ for (size_t i = 0; i < interned_comms_size_; i++) {
+ if (transient_view == interned_comms_[i]) {
+ return i;
}
}
- // Unique next_comm, intern it. Null byte is not copied over.
+ // Unique comm, intern it. Null byte is not copied over.
char* start = intern_buf_ + intern_buf_write_pos_;
size_t size = transient_view.size();
memcpy(start, ptr, size);
intern_buf_write_pos_ += size;
- switch_next_comm_index_.Append(interned_switch_comms_size_);
+ size_t idx = interned_comms_size_;
base::StringView safe_view(start, size);
- interned_switch_comms_[interned_switch_comms_size_++] = safe_view;
+ interned_comms_[interned_comms_size_++] = safe_view;
PERFETTO_DCHECK(intern_buf_write_pos_ <= sizeof(intern_buf_));
+ PERFETTO_DCHECK(interned_comms_size_ < kMaxElements);
+ return idx;
}
+ size_t interned_comms_size() const { return interned_comms_size_; }
+
+ void Write(
+ protos::pbzero::FtraceEventBundle::CompactSched* compact_out) const;
+ void Reset();
+
+ private:
+ // TODO(rsavitski): Consider making the storage dynamically-expandable instead
+ // to not rely on sizing the buffer for the worst case.
+ static constexpr size_t kMaxElements = 4096;
+
+ char intern_buf_[kMaxElements * (kExpectedCommLength - 1)];
+ size_t intern_buf_write_pos_ = 0;
+
+ // Views into unique interned comm strings. Even if every event carries a
+ // unique comm, the ftrace reader is expected to flush the compact buffer way
+ // before this reaches capacity. This is since the cost of processing each
+ // event grows with every unique interned comm (as the interning needs to
+ // search all existing internings).
+ std::array<base::StringView, kMaxElements> interned_comms_;
+ uint32_t interned_comms_size_ = 0;
+};
+
+// Mutable state for buffering parts of scheduling events, that can later be
+// written out in a compact format with |WriteAndReset|. Used by the ftrace
+// reader.
+class CompactSchedBuffer {
+ public:
+ CompactSchedSwitchBuffer& sched_switch() { return switch_; }
+ CompactSchedWakingBuffer& sched_waking() { return waking_; }
+ CommInterner& interner() { return interner_; }
+
// Writes out the currently buffered events, and starts the next batch
// internally.
void WriteAndReset(protos::pbzero::FtraceEventBundle* bundle);
private:
- // First timestamp in a bundle is absolute. The rest are all delta-encoded,
- // each relative to the preceding sched_switch timestamp.
- uint64_t last_switch_timestamp_ = 0;
-
- protozero::PackedVarInt switch_timestamp_;
- protozero::PackedVarInt switch_prev_state_;
- protozero::PackedVarInt switch_next_pid_;
- protozero::PackedVarInt switch_next_prio_;
-
- // Storage for interned strings (without null bytes).
- char intern_buf_[kMaxElements * (kExpectedCommLength - 1)];
- size_t intern_buf_write_pos_ = 0;
-
- // Views into unique interned next_comm strings. Even if every sched_switch
- // carries a unique next_comm, the ftrace reader is expected to flush the
- // compact buffer way before this reaches capacity. This is since the cost of
- // processing each event grows with every unique interned next_comm (as the
- // interning needs to search all existing internings).
- std::array<base::StringView, kMaxElements> interned_switch_comms_;
- uint32_t interned_switch_comms_size_ = 0;
-
- // One entry per sched_switch event, contains the index of the interned
- // next_comm string view (i.e. array index into |interned_switch_comms|).
- protozero::PackedVarInt switch_next_comm_index_;
+ CommInterner interner_;
+ CompactSchedSwitchBuffer switch_;
+ CompactSchedWakingBuffer waking_;
};
} // namespace perfetto
diff --git a/src/traced/probes/ftrace/cpu_reader.cc b/src/traced/probes/ftrace/cpu_reader.cc
index 96f114a..8299001 100644
--- a/src/traced/probes/ftrace/cpu_reader.cc
+++ b/src/traced/probes/ftrace/cpu_reader.cc
@@ -16,8 +16,8 @@
#include "src/traced/probes/ftrace/cpu_reader.h"
-#include <signal.h>
#include <dirent.h>
+#include <signal.h>
#include <utility>
@@ -279,7 +279,7 @@
const ProtoTranslationTable* table) {
// Begin an FtraceEventBundle, and allocate the buffer for compact scheduler
// events (which will be unused if the compact option isn't enabled).
- CompactSchedBundleState compact_sched;
+ CompactSchedBuffer compact_sched;
auto packet = trace_writer->NewTracePacket();
auto* bundle = packet->set_ftrace_events();
@@ -312,8 +312,9 @@
// a threshold. We need to flush the compact buffer to make the
// interning lookups cheap again.
bool interner_past_threshold =
- compact_sched_enabled && compact_sched.interned_switch_comms_size() >
- kCompactSchedInternerThreshold;
+ compact_sched_enabled &&
+ compact_sched.interner().interned_comms_size() >
+ kCompactSchedInternerThreshold;
if (page_header->lost_events || interner_past_threshold) {
if (compact_sched_enabled)
compact_sched.WriteAndReset(bundle);
@@ -398,14 +399,13 @@
// binary ftrace events. See |ParsePageHeader| for the format of the earlier.
//
// This method is deliberately static so it can be tested independently.
-size_t CpuReader::ParsePagePayload(
- const uint8_t* start_of_payload,
- const PageHeader* page_header,
- const ProtoTranslationTable* table,
- const FtraceDataSourceConfig* ds_config,
- CompactSchedBundleState* compact_sched_buffer,
- FtraceEventBundle* bundle,
- FtraceMetadata* metadata) {
+size_t CpuReader::ParsePagePayload(const uint8_t* start_of_payload,
+ const PageHeader* page_header,
+ const ProtoTranslationTable* table,
+ const FtraceDataSourceConfig* ds_config,
+ CompactSchedBuffer* compact_sched_buffer,
+ FtraceEventBundle* bundle,
+ FtraceMetadata* metadata) {
const uint8_t* ptr = start_of_payload;
const uint8_t* const end = ptr + page_header->size;
@@ -484,7 +484,10 @@
bool compact_sched_enabled = ds_config->compact_sched.enabled;
const CompactSchedSwitchFormat& sched_switch_format =
table->compact_sched_format().sched_switch;
+ const CompactSchedWakingFormat& sched_waking_format =
+ table->compact_sched_format().sched_waking;
+ // compact sched_switch
if (compact_sched_enabled &&
ftrace_event_id == sched_switch_format.event_id) {
if (event_size < sched_switch_format.size)
@@ -492,8 +495,18 @@
ParseSchedSwitchCompact(start, timestamp, &sched_switch_format,
compact_sched_buffer, metadata);
+
+ // compact sched_waking
+ } else if (compact_sched_enabled &&
+ ftrace_event_id == sched_waking_format.event_id) {
+ if (event_size < sched_waking_format.size)
+ return 0;
+
+ ParseSchedWakingCompact(start, timestamp, &sched_waking_format,
+ compact_sched_buffer, metadata);
+
} else {
- // Parse all other types of enabled events.
+ // Common case: parse all other types of enabled events.
protos::pbzero::FtraceEvent* event = bundle->add_event();
event->set_timestamp(timestamp);
if (!ParseEvent(ftrace_event_id, start, next, table, event,
@@ -663,31 +676,56 @@
// |CompactSchedSwitchFormat| for the assumptions made around the format, which
// this code is closely tied to.
// static
-void CpuReader::ParseSchedSwitchCompact(
- const uint8_t* start,
- uint64_t timestamp,
- const CompactSchedSwitchFormat* format,
- CompactSchedBundleState* compact_sched_buffer,
- FtraceMetadata* metadata) {
- compact_sched_buffer->AppendSwitchTimestamp(timestamp);
+void CpuReader::ParseSchedSwitchCompact(const uint8_t* start,
+ uint64_t timestamp,
+ const CompactSchedSwitchFormat* format,
+ CompactSchedBuffer* compact_buf,
+ FtraceMetadata* metadata) {
+ compact_buf->sched_switch().AppendTimestamp(timestamp);
int32_t next_pid = ReadValue<int32_t>(start + format->next_pid_offset);
- compact_sched_buffer->switch_next_pid()->Append(next_pid);
+ compact_buf->sched_switch().next_pid().Append(next_pid);
metadata->AddPid(next_pid);
int32_t next_prio = ReadValue<int32_t>(start + format->next_prio_offset);
- compact_sched_buffer->switch_next_prio()->Append(next_prio);
+ compact_buf->sched_switch().next_prio().Append(next_prio);
// Varint encoding of int32 and int64 is the same, so treat the value as
// int64 after reading.
int64_t prev_state = ReadSignedFtraceValue(start + format->prev_state_offset,
format->prev_state_type);
- compact_sched_buffer->switch_prev_state()->Append(prev_state);
+ compact_buf->sched_switch().prev_state().Append(prev_state);
// next_comm
const char* comm_ptr =
reinterpret_cast<const char*>(start + format->next_comm_offset);
- compact_sched_buffer->InternSwitchNextComm(comm_ptr);
+ size_t iid = compact_buf->interner().InternComm(comm_ptr);
+ compact_buf->sched_switch().next_comm_index().Append(iid);
+}
+
+// static
+void CpuReader::ParseSchedWakingCompact(const uint8_t* start,
+ uint64_t timestamp,
+ const CompactSchedWakingFormat* format,
+ CompactSchedBuffer* compact_buf,
+ FtraceMetadata* metadata) {
+ compact_buf->sched_waking().AppendTimestamp(timestamp);
+
+ int32_t pid = ReadValue<int32_t>(start + format->pid_offset);
+ compact_buf->sched_waking().pid().Append(pid);
+ metadata->AddPid(pid);
+
+ int32_t target_cpu = ReadValue<int32_t>(start + format->target_cpu_offset);
+ compact_buf->sched_waking().target_cpu().Append(target_cpu);
+
+ int32_t prio = ReadValue<int32_t>(start + format->prio_offset);
+ compact_buf->sched_waking().prio().Append(prio);
+
+ // comm
+ const char* comm_ptr =
+ reinterpret_cast<const char*>(start + format->comm_offset);
+ size_t iid = compact_buf->interner().InternComm(comm_ptr);
+ compact_buf->sched_waking().comm_index().Append(iid);
}
} // namespace perfetto
diff --git a/src/traced/probes/ftrace/cpu_reader.h b/src/traced/probes/ftrace/cpu_reader.h
index 139eebe..f329044 100644
--- a/src/traced/probes/ftrace/cpu_reader.h
+++ b/src/traced/probes/ftrace/cpu_reader.h
@@ -169,7 +169,7 @@
const PageHeader* page_header,
const ProtoTranslationTable* table,
const FtraceDataSourceConfig* ds_config,
- CompactSchedBundleState* compact_sched_buffer,
+ CompactSchedBuffer* compact_sched_buffer,
FtraceEventBundle* bundle,
FtraceMetadata* metadata);
@@ -198,7 +198,15 @@
static void ParseSchedSwitchCompact(const uint8_t* start,
uint64_t timestamp,
const CompactSchedSwitchFormat* format,
- CompactSchedBundleState* bundle_state,
+ CompactSchedBuffer* compact_buf,
+ FtraceMetadata* metadata);
+
+ // Parse a sched_waking event according to pre-validated format, and buffer
+ // the individual fields in the given compact encoding batch.
+ static void ParseSchedWakingCompact(const uint8_t* start,
+ uint64_t timestamp,
+ const CompactSchedWakingFormat* format,
+ CompactSchedBuffer* compact_buf,
FtraceMetadata* metadata);
// Parses & encodes the given range of contiguous tracing pages. Called by
diff --git a/src/traced/probes/ftrace/cpu_reader_benchmark.cc b/src/traced/probes/ftrace/cpu_reader_benchmark.cc
index b4d942f..58b4e20 100644
--- a/src/traced/probes/ftrace/cpu_reader_benchmark.cc
+++ b/src/traced/probes/ftrace/cpu_reader_benchmark.cc
@@ -289,7 +289,7 @@
} // namespace
-using perfetto::CompactSchedBundleState;
+using perfetto::CompactSchedBuffer;
using perfetto::CpuReader;
using perfetto::DisabledCompactSchedConfigForTesting;
using perfetto::EventFilter;
@@ -324,7 +324,7 @@
while (state.KeepRunning()) {
writer.Reset(&stream);
- CompactSchedBundleState compact_buffer;
+ CompactSchedBuffer compact_buffer;
const uint8_t* parse_pos = page.get();
perfetto::base::Optional<CpuReader::PageHeader> page_header =
CpuReader::ParsePageHeader(&parse_pos, table->page_header_size_len());
diff --git a/src/traced/probes/ftrace/cpu_reader_unittest.cc b/src/traced/probes/ftrace/cpu_reader_unittest.cc
index 7f17c07..f8b58cf 100644
--- a/src/traced/probes/ftrace/cpu_reader_unittest.cc
+++ b/src/traced/probes/ftrace/cpu_reader_unittest.cc
@@ -381,7 +381,7 @@
table->EventToFtraceId(GroupAndName("ftrace", "print")));
FtraceMetadata metadata{};
- CompactSchedBundleState compact_buffer;
+ CompactSchedBuffer compact_buffer;
const uint8_t* parse_pos = page.get();
base::Optional<CpuReader::PageHeader> page_header =
CpuReader::ParsePageHeader(&parse_pos, table->page_header_size_len());
@@ -510,7 +510,7 @@
table->EventToFtraceId(GroupAndName("ftrace", "print")));
FtraceMetadata metadata{};
- CompactSchedBundleState compact_buffer;
+ CompactSchedBuffer compact_buffer;
const uint8_t* parse_pos = page.get();
base::Optional<CpuReader::PageHeader> page_header =
CpuReader::ParsePageHeader(&parse_pos, table->page_header_size_len());
@@ -560,7 +560,7 @@
table->EventToFtraceId(GroupAndName("ftrace", "print")));
FtraceMetadata metadata{};
- CompactSchedBundleState compact_buffer;
+ CompactSchedBuffer compact_buffer;
const uint8_t* parse_pos = page.get();
base::Optional<CpuReader::PageHeader> page_header =
CpuReader::ParsePageHeader(&parse_pos, table->page_header_size_len());
@@ -599,7 +599,7 @@
DisabledCompactSchedConfigForTesting()};
FtraceMetadata metadata{};
- CompactSchedBundleState compact_buffer;
+ CompactSchedBuffer compact_buffer;
const uint8_t* parse_pos = page.get();
base::Optional<CpuReader::PageHeader> page_header =
CpuReader::ParsePageHeader(&parse_pos, table->page_header_size_len());
@@ -665,7 +665,7 @@
table->EventToFtraceId(GroupAndName("ftrace", "print")));
FtraceMetadata metadata{};
- CompactSchedBundleState compact_buffer;
+ CompactSchedBuffer compact_buffer;
const uint8_t* parse_pos = page.get();
base::Optional<CpuReader::PageHeader> page_header =
CpuReader::ParsePageHeader(&parse_pos, table->page_header_size_len());
@@ -775,7 +775,7 @@
table->EventToFtraceId(GroupAndName("sched", "sched_switch")));
FtraceMetadata metadata{};
- CompactSchedBundleState compact_buffer;
+ CompactSchedBuffer compact_buffer;
const uint8_t* parse_pos = page.get();
base::Optional<CpuReader::PageHeader> page_header =
CpuReader::ParsePageHeader(&parse_pos, table->page_header_size_len());
@@ -822,7 +822,7 @@
table->EventToFtraceId(GroupAndName("sched", "sched_switch")));
FtraceMetadata metadata{};
- CompactSchedBundleState compact_buffer;
+ CompactSchedBuffer compact_buffer;
const uint8_t* parse_pos = page.get();
base::Optional<CpuReader::PageHeader> page_header =
CpuReader::ParsePageHeader(&parse_pos, table->page_header_size_len());
@@ -846,7 +846,8 @@
EXPECT_FALSE(bundle->has_compact_sched());
// Instead, sched switch fields were buffered:
- EXPECT_LT(0u, compact_buffer.interned_switch_comms_size());
+ EXPECT_LT(0u, compact_buffer.sched_switch().size());
+ EXPECT_LT(0u, compact_buffer.interner().interned_comms_size());
// Write the buffer out & check the serialized format:
compact_buffer.WriteAndReset(bundle_provider.writer());
@@ -1620,7 +1621,7 @@
table->EventToFtraceId(GroupAndName("sched", "sched_switch")));
FtraceMetadata metadata{};
- CompactSchedBundleState compact_buffer;
+ CompactSchedBuffer compact_buffer;
const uint8_t* parse_pos = page.get();
base::Optional<CpuReader::PageHeader> page_header =
CpuReader::ParsePageHeader(&parse_pos, table->page_header_size_len());
@@ -2065,7 +2066,7 @@
table->EventToFtraceId(GroupAndName("sched", "sched_switch")));
FtraceMetadata metadata{};
- CompactSchedBundleState compact_buffer;
+ CompactSchedBuffer compact_buffer;
const uint8_t* parse_pos = page.get();
base::Optional<CpuReader::PageHeader> page_header =
CpuReader::ParsePageHeader(&parse_pos, table->page_header_size_len());
diff --git a/src/traced/probes/ftrace/ftrace_config_muxer_unittest.cc b/src/traced/probes/ftrace/ftrace_config_muxer_unittest.cc
index 8901cb5..890e07a 100644
--- a/src/traced/probes/ftrace/ftrace_config_muxer_unittest.cc
+++ b/src/traced/probes/ftrace/ftrace_config_muxer_unittest.cc
@@ -661,8 +661,9 @@
TEST_F(FtraceConfigMuxerTest, CompactSchedConfig) {
// Set scheduling event format as validated. The pre-parsed format itself
// doesn't need to be sensible, as the tests won't use it.
- auto valid_compact_format = CompactSchedEventFormat{
- /*format_valid=*/true, CompactSchedSwitchFormat{}};
+ auto valid_compact_format =
+ CompactSchedEventFormat{/*format_valid=*/true, CompactSchedSwitchFormat{},
+ CompactSchedWakingFormat{}};
NiceMock<MockFtraceProcfs> ftrace;
table_ = CreateFakeTable(valid_compact_format);
diff --git a/src/traced/probes/ftrace/ftrace_controller.cc b/src/traced/probes/ftrace/ftrace_controller.cc
index 0732527..a66ed05 100644
--- a/src/traced/probes/ftrace/ftrace_controller.cc
+++ b/src/traced/probes/ftrace/ftrace_controller.cc
@@ -34,7 +34,6 @@
#include "perfetto/ext/base/file_utils.h"
#include "perfetto/ext/base/metatrace.h"
#include "perfetto/ext/tracing/core/trace_writer.h"
-#include "src/traced/probes/ftrace/compact_sched.h"
#include "src/traced/probes/ftrace/cpu_reader.h"
#include "src/traced/probes/ftrace/cpu_stats_parser.h"
#include "src/traced/probes/ftrace/event_info.h"
@@ -64,21 +63,10 @@
// data, which should fit in a typical L2D cache. Furthermore, the batching
// limits the memory usage of traced_probes.
//
-// Note that the compact scheduler event encoding buffers operate on the same
-// buffer size, and their maximum capacity is set at compile-time as it lives on
-// the stack. So the maximum possible number of events encoded in this many
-// tracing pages must not exceed compact_sched's capacity. See the static_assert
-// below.
// TODO(rsavitski): consider making buffering & parsing page counts independent,
// should be a single counter in the cpu_reader, similar to lost_events case.
constexpr size_t kParsingBufferSizePages = 32;
-static_assert(kParsingBufferSizePages *
- (base::kPageSize /
- CompactSchedBundleState::kMinSupportedSchedSwitchSize) <=
- CompactSchedBundleState::kMaxElements,
- "insufficient compact_sched buffer capacity");
-
uint32_t ClampDrainPeriodMs(uint32_t drain_period_ms) {
if (drain_period_ms == 0) {
return kDefaultDrainPeriodMs;
diff --git a/src/traced/probes/ftrace/proto_translation_table_unittest.cc b/src/traced/probes/ftrace/proto_translation_table_unittest.cc
index f3841d9..8f54e48 100644
--- a/src/traced/probes/ftrace/proto_translation_table_unittest.cc
+++ b/src/traced/probes/ftrace/proto_translation_table_unittest.cc
@@ -259,9 +259,10 @@
INSTANTIATE_TEST_SUITE_P(BySize, TranslationTableCreationTest, Values(4, 8));
-TEST(TranslationTableTest, CompactSchedFormatParsingSeedData) {
+TEST(TranslationTableTest, CompactSchedFormatParsingWalleyeData) {
std::string path =
- "src/traced/probes/ftrace/test/data/android_seed_N2F62_3.10.49/";
+ "src/traced/probes/ftrace/test/data/"
+ "android_walleye_OPM5.171019.017.A1_4.4.88/";
FtraceProcfs ftrace_procfs(path);
auto table = ProtoTranslationTable::Create(
&ftrace_procfs, GetStaticEventInfo(), GetStaticCommonFieldsInfo());
@@ -271,30 +272,7 @@
// Format matches compile-time assumptions.
ASSERT_TRUE(format.format_valid);
- // Check exact format (note: 32 bit long prev_state).
- EXPECT_EQ(68u, format.sched_switch.event_id);
- EXPECT_EQ(60u, format.sched_switch.size);
- EXPECT_EQ(52u, format.sched_switch.next_pid_offset);
- EXPECT_EQ(FtraceFieldType::kFtracePid32, format.sched_switch.next_pid_type);
- EXPECT_EQ(56u, format.sched_switch.next_prio_offset);
- EXPECT_EQ(FtraceFieldType::kFtraceInt32, format.sched_switch.next_prio_type);
- EXPECT_EQ(32u, format.sched_switch.prev_state_offset);
- EXPECT_EQ(FtraceFieldType::kFtraceInt32, format.sched_switch.prev_state_type);
- EXPECT_EQ(36u, format.sched_switch.next_comm_offset);
-}
-
-TEST(TranslationTableTest, CompactSchedFormatParsingSyntheticData) {
- std::string path = "src/traced/probes/ftrace/test/data/synthetic/";
- FtraceProcfs ftrace_procfs(path);
- auto table = ProtoTranslationTable::Create(
- &ftrace_procfs, GetStaticEventInfo(), GetStaticCommonFieldsInfo());
- PERFETTO_CHECK(table);
- const CompactSchedEventFormat& format = table->compact_sched_format();
-
- // Format matches compile-time assumptions.
- ASSERT_TRUE(format.format_valid);
-
- // Check exact format (note: 64 bit long prev_state).
+ // Check exact sched_switch format (note: 64 bit long prev_state).
EXPECT_EQ(47u, format.sched_switch.event_id);
EXPECT_EQ(64u, format.sched_switch.size);
EXPECT_EQ(56u, format.sched_switch.next_pid_offset);
@@ -304,6 +282,33 @@
EXPECT_EQ(32u, format.sched_switch.prev_state_offset);
EXPECT_EQ(FtraceFieldType::kFtraceInt64, format.sched_switch.prev_state_type);
EXPECT_EQ(40u, format.sched_switch.next_comm_offset);
+
+ // Check exact sched_waking format.
+ EXPECT_EQ(44u, format.sched_waking.event_id);
+ EXPECT_EQ(40u, format.sched_waking.size);
+ EXPECT_EQ(24u, format.sched_waking.pid_offset);
+ EXPECT_EQ(FtraceFieldType::kFtracePid32, format.sched_waking.pid_type);
+ EXPECT_EQ(36u, format.sched_waking.target_cpu_offset);
+ EXPECT_EQ(FtraceFieldType::kFtraceInt32, format.sched_waking.target_cpu_type);
+ EXPECT_EQ(28u, format.sched_waking.prio_offset);
+ EXPECT_EQ(FtraceFieldType::kFtraceInt32, format.sched_waking.prio_type);
+ EXPECT_EQ(8u, format.sched_waking.comm_offset);
+}
+
+TEST(TranslationTableTest, CompactSchedFormatParsingSeedData) {
+ std::string path =
+ "src/traced/probes/ftrace/test/data/android_seed_N2F62_3.10.49/";
+ FtraceProcfs ftrace_procfs(path);
+ auto table = ProtoTranslationTable::Create(
+ &ftrace_procfs, GetStaticEventInfo(), GetStaticCommonFieldsInfo());
+ PERFETTO_CHECK(table);
+ const CompactSchedEventFormat& format = table->compact_sched_format();
+
+ // We consider the entire format invalid as there's no sched_waking event
+ // available. This is a simplifying assumption. We could instead look at each
+ // event independently (and in this case, sched_switch does match compile-time
+ // assumptions).
+ ASSERT_FALSE(format.format_valid);
}
TEST(TranslationTableTest, InferFtraceType) {
diff --git a/src/traced/probes/ftrace/test/data/synthetic/available_events b/src/traced/probes/ftrace/test/data/synthetic/available_events
index 0a0ea6f..28798fa 100644
--- a/src/traced/probes/ftrace/test/data/synthetic/available_events
+++ b/src/traced/probes/ftrace/test/data/synthetic/available_events
@@ -1,4 +1,5 @@
sched:sched_switch
+sched:sched_waking
kmem:ion_heap_grow
kmem:ion_heap_shrink
kmem:rss_stat
diff --git a/src/traced/probes/ftrace/test/data/synthetic/events/sched/sched_waking/format b/src/traced/probes/ftrace/test/data/synthetic/events/sched/sched_waking/format
new file mode 100644
index 0000000..57bd43f
--- /dev/null
+++ b/src/traced/probes/ftrace/test/data/synthetic/events/sched/sched_waking/format
@@ -0,0 +1,15 @@
+name: sched_waking
+ID: 44
+format:
+ field:unsigned short common_type; offset:0; size:2; signed:0;
+ field:unsigned char common_flags; offset:2; size:1; signed:0;
+ field:unsigned char common_preempt_count; offset:3; size:1; signed:0;
+ field:int common_pid; offset:4; size:4; signed:1;
+
+ field:char comm[16]; offset:8; size:16; signed:0;
+ field:pid_t pid; offset:24; size:4; signed:1;
+ field:int prio; offset:28; size:4; signed:1;
+ field:int success; offset:32; size:4; signed:1;
+ field:int target_cpu; offset:36; size:4; signed:1;
+
+print fmt: "comm=%s pid=%d prio=%d target_cpu=%03d", REC->comm, REC->pid, REC->prio, REC->target_cpu