aboutsummaryrefslogtreecommitdiff
path: root/src/traced/probes/ftrace/cpu_reader.cc
diff options
context:
space:
mode:
Diffstat (limited to 'src/traced/probes/ftrace/cpu_reader.cc')
-rw-r--r--src/traced/probes/ftrace/cpu_reader.cc196
1 files changed, 47 insertions, 149 deletions
diff --git a/src/traced/probes/ftrace/cpu_reader.cc b/src/traced/probes/ftrace/cpu_reader.cc
index 9b1d4f878..6010e903d 100644
--- a/src/traced/probes/ftrace/cpu_reader.cc
+++ b/src/traced/probes/ftrace/cpu_reader.cc
@@ -17,10 +17,8 @@
#include "src/traced/probes/ftrace/cpu_reader.h"
#include <dirent.h>
-#include <fcntl.h>
#include <signal.h>
-#include <algorithm>
#include <utility>
#include "perfetto/base/build_config.h"
@@ -29,19 +27,14 @@
#include "perfetto/ext/base/optional.h"
#include "perfetto/ext/base/utils.h"
#include "perfetto/ext/tracing/core/trace_writer.h"
-#include "src/kallsyms/kernel_symbol_map.h"
-#include "src/kallsyms/lazy_kernel_symbolizer.h"
-#include "src/traced/probes/ftrace/ftrace_config_muxer.h"
-#include "src/traced/probes/ftrace/ftrace_controller.h"
-#include "src/traced/probes/ftrace/ftrace_data_source.h"
-#include "src/traced/probes/ftrace/proto_translation_table.h"
-
#include "protos/perfetto/trace/ftrace/ftrace_event.pbzero.h"
#include "protos/perfetto/trace/ftrace/ftrace_event_bundle.pbzero.h"
#include "protos/perfetto/trace/ftrace/generic.pbzero.h"
-#include "protos/perfetto/trace/interned_data/interned_data.pbzero.h"
-#include "protos/perfetto/trace/profiling/profile_common.pbzero.h"
#include "protos/perfetto/trace/trace_packet.pbzero.h"
+#include "src/traced/probes/ftrace/ftrace_config_muxer.h"
+#include "src/traced/probes/ftrace/ftrace_controller.h"
+#include "src/traced/probes/ftrace/ftrace_data_source.h"
+#include "src/traced/probes/ftrace/proto_translation_table.h"
namespace perfetto {
namespace {
@@ -147,12 +140,8 @@ using protos::pbzero::GenericFtraceEvent;
CpuReader::CpuReader(size_t cpu,
const ProtoTranslationTable* table,
- LazyKernelSymbolizer* symbolizer,
base::ScopedFile trace_fd)
- : cpu_(cpu),
- table_(table),
- symbolizer_(symbolizer),
- trace_fd_(std::move(trace_fd)) {
+ : cpu_(cpu), table_(table), trace_fd_(std::move(trace_fd)) {
PERFETTO_CHECK(trace_fd_);
PERFETTO_CHECK(SetBlocking(*trace_fd_, false));
}
@@ -212,11 +201,8 @@ size_t CpuReader::ReadAndProcessBatch(
// Expected errors:
// EAGAIN: no data (since we're in non-blocking mode).
// ENONMEM, EBUSY: temporary ftrace failures (they happen).
- // ENODEV: the cpu is offline (b/145583318).
- if (errno != EAGAIN && errno != ENOMEM && errno != EBUSY &&
- errno != ENODEV) {
+ if (errno != EAGAIN && errno != ENOMEM && errno != EBUSY)
PERFETTO_PLOG("Unexpected error on raw ftrace read");
- }
break; // stop reading regardless of errno
}
@@ -275,8 +261,7 @@ size_t CpuReader::ReadAndProcessBatch(
for (FtraceDataSource* data_source : started_data_sources) {
bool success = ProcessPagesForDataSource(
data_source->trace_writer(), data_source->mutable_metadata(), cpu_,
- data_source->parsing_config(), parsing_buf, pages_read, table_,
- symbolizer_);
+ data_source->parsing_config(), parsing_buf, pages_read, table_);
PERFETTO_CHECK(success);
}
@@ -291,98 +276,20 @@ bool CpuReader::ProcessPagesForDataSource(
const FtraceDataSourceConfig* ds_config,
const uint8_t* parsing_buf,
const size_t pages_read,
- const ProtoTranslationTable* table,
- LazyKernelSymbolizer* symbolizer) {
- // Allocate the buffer for compact scheduler events (which will be unused if
- // the compact option isn't enabled).
+ 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).
CompactSchedBuffer compact_sched;
- bool compact_sched_enabled = ds_config->compact_sched.enabled;
+ auto packet = trace_writer->NewTracePacket();
+ auto* bundle = packet->set_ftrace_events();
- TraceWriter::TracePacketHandle packet;
- protos::pbzero::FtraceEventBundle* bundle = nullptr;
-
- // This function is called after the contents of a FtraceBundle are written.
- auto finalize_cur_packet = [&] {
- PERFETTO_DCHECK(packet);
- if (compact_sched_enabled)
- compact_sched.WriteAndReset(bundle);
-
- bundle->Finalize();
- bundle = nullptr;
-
- // Write the kernel symbol index (mangled address) -> name table.
- // |metadata| is shared across all cpus, is distinct per |data_source| (i.e.
- // tracing session) and is cleared after each FtraceController::ReadTick().
- // const size_t kaddrs_size = metadata->kernel_addrs.size();
- if (ds_config->symbolize_ksyms) {
- // Symbol indexes are assigned mononically as |kernel_addrs.size()|,
- // starting from index 1 (no symbol has index 0). Here we remember the
- // size() (which is also == the highest value in |kernel_addrs|) at the
- // beginning and only write newer indexes bigger than that.
- uint32_t max_index_at_start = metadata->last_kernel_addr_index_written;
- PERFETTO_DCHECK(max_index_at_start <= metadata->kernel_addrs.size());
- protos::pbzero::InternedData* interned_data = nullptr;
- auto* ksyms_map = symbolizer->GetOrCreateKernelSymbolMap();
- bool wrote_at_least_one_symbol = false;
- for (const FtraceMetadata::KernelAddr& kaddr : metadata->kernel_addrs) {
- if (kaddr.index <= max_index_at_start)
- continue;
- std::string sym_name = ksyms_map->Lookup(kaddr.addr);
- if (sym_name.empty()) {
- // Lookup failed. This can genuinely happen in many occasions. E.g.,
- // workqueue_execute_start has two pointers: one is a pointer to a
- // function (which we expect to be symbolized), the other (|work|) is
- // a pointer to a heap struct, which is unsymbolizable, even when
- // using the textual ftrace endpoint.
- continue;
- }
-
- if (!interned_data) {
- // If this is the very first write, clear the start of the sequence
- // so the trace processor knows that all previous indexes can be
- // discarded and that the mapping is restarting.
- // In most cases this occurs with cpu==0. But if cpu0 is idle, this
- // will happen with the first CPU that has any ftrace data.
- if (max_index_at_start == 0) {
- packet->set_sequence_flags(
- protos::pbzero::TracePacket::SEQ_INCREMENTAL_STATE_CLEARED);
- }
- interned_data = packet->set_interned_data();
- }
- auto* interned_sym = interned_data->add_kernel_symbols();
- interned_sym->set_iid(kaddr.index);
- interned_sym->set_str(sym_name);
- wrote_at_least_one_symbol = true;
- }
+ bool compact_sched_enabled = ds_config->compact_sched.enabled;
- auto max_it_at_end = static_cast<uint32_t>(metadata->kernel_addrs.size());
+ // Note: The fastpath in proto_trace_parser.cc speculates on the fact
+ // that the cpu field is the first field of the proto message. If this
+ // changes, change proto_trace_parser.cc accordingly.
+ bundle->set_cpu(static_cast<uint32_t>(cpu));
- // Rationale for the if (wrote_at_least_one_symbol) check: in rare cases,
- // all symbols seen in a ProcessPagesForDataSource() call can fail the
- // ksyms_map->Lookup(). If that happens we don't want to bump the
- // last_kernel_addr_index_written watermark, as that would cause the next
- // call to NOT emit the SEQ_INCREMENTAL_STATE_CLEARED.
- if (wrote_at_least_one_symbol)
- metadata->last_kernel_addr_index_written = max_it_at_end;
- }
-
- packet->Finalize();
- }; // finalize_cur_packet().
-
- auto start_new_packet = [&](bool lost_events) {
- if (packet)
- finalize_cur_packet();
- packet = trace_writer->NewTracePacket();
- bundle = packet->set_ftrace_events();
- // Note: The fastpath in proto_trace_parser.cc speculates on the fact
- // that the cpu field is the first field of the proto message. If this
- // changes, change proto_trace_parser.cc accordingly.
- bundle->set_cpu(static_cast<uint32_t>(cpu));
- if (lost_events)
- bundle->set_lost_events(true);
- };
-
- start_new_packet(/*lost_events=*/false);
for (size_t i = 0; i < pages_read; i++) {
const uint8_t* curr_page = parsing_buf + (i * base::kPageSize);
const uint8_t* curr_page_end = curr_page + base::kPageSize;
@@ -408,9 +315,17 @@ bool CpuReader::ProcessPagesForDataSource(
compact_sched_enabled &&
compact_sched.interner().interned_comms_size() >
kCompactSchedInternerThreshold;
-
- if (page_header->lost_events || interner_past_threshold)
- start_new_packet(page_header->lost_events);
+ if (page_header->lost_events || interner_past_threshold) {
+ if (compact_sched_enabled)
+ compact_sched.WriteAndReset(bundle);
+ packet->Finalize();
+
+ packet = trace_writer->NewTracePacket();
+ bundle = packet->set_ftrace_events();
+ bundle->set_cpu(static_cast<uint32_t>(cpu));
+ if (page_header->lost_events)
+ bundle->set_lost_events(true);
+ }
size_t evt_size =
ParsePagePayload(parse_pos, &page_header.value(), table, ds_config,
@@ -420,7 +335,9 @@ bool CpuReader::ProcessPagesForDataSource(
// (FtraceMetadata -> FtraceStats in trace).
PERFETTO_DCHECK(evt_size == page_header->size);
}
- finalize_cur_packet();
+
+ if (compact_sched_enabled)
+ compact_sched.WriteAndReset(bundle);
return true;
}
@@ -509,7 +426,7 @@ size_t CpuReader::ParsePagePayload(const uint8_t* start_of_payload,
PERFETTO_DFATAL("Empty padding event.");
return 0;
}
- uint32_t length = 0;
+ uint32_t length;
if (!ReadAndAdvance<uint32_t>(&ptr, end, &length))
return 0;
// length includes itself (4 bytes)
@@ -520,23 +437,20 @@ size_t CpuReader::ParsePagePayload(const uint8_t* start_of_payload,
}
case kTypeTimeExtend: {
// Extend the time delta.
- uint32_t time_delta_ext = 0;
+ uint32_t time_delta_ext;
if (!ReadAndAdvance<uint32_t>(&ptr, end, &time_delta_ext))
return 0;
+ // See https://goo.gl/CFBu5x
timestamp += (static_cast<uint64_t>(time_delta_ext)) << 27;
break;
}
case kTypeTimeStamp: {
- // Absolute timestamp. This was historically partially implemented, but
- // not written. Kernels 4.17+ reimplemented this record, changing its
- // size in the process. We assume the newer layout. Parsed the same as
- // kTypeTimeExtend, except that the timestamp is interpreted as an
- // absolute, instead of a delta on top of the previous state.
- timestamp = event_header.time_delta;
- uint32_t time_delta_ext = 0;
- if (!ReadAndAdvance<uint32_t>(&ptr, end, &time_delta_ext))
+ // Sync time stamp with external clock.
+ TimeStamp time_stamp;
+ if (!ReadAndAdvance<TimeStamp>(&ptr, end, &time_stamp))
return 0;
- timestamp += (static_cast<uint64_t>(time_delta_ext)) << 27;
+ // Not implemented in the kernel, nothing should generate this.
+ PERFETTO_DFATAL("Unimplemented in kernel. Should be unreachable.");
break;
}
// Data record:
@@ -546,7 +460,7 @@ size_t CpuReader::ParsePagePayload(const uint8_t* start_of_payload,
// record. if == 0, this is an extended record and the size of the
// record is stored in the first uint32_t word in the payload. See
// Kernel's include/linux/ring_buffer.h
- uint32_t event_size = 0;
+ uint32_t event_size;
if (event_header.type_or_length == 0) {
if (!ReadAndAdvance<uint32_t>(&ptr, end, &event_size))
return 0;
@@ -635,7 +549,7 @@ bool CpuReader::ParseEvent(uint16_t ftrace_event_id,
bool success = true;
for (const Field& field : table->common_fields())
- success &= ParseField(field, start, end, table, message, metadata);
+ success &= ParseField(field, start, end, message, metadata);
protozero::Message* nested =
message->BeginNestedMessage<protozero::Message>(info.proto_field_id);
@@ -647,14 +561,14 @@ bool CpuReader::ParseEvent(uint16_t ftrace_event_id,
for (const Field& field : info.fields) {
auto generic_field = nested->BeginNestedMessage<protozero::Message>(
GenericFtraceEvent::kFieldFieldNumber);
- // TODO(hjd): Avoid outputting field names every time.
+ // TODO(taylori): Avoid outputting field names every time.
generic_field->AppendString(GenericFtraceEvent::Field::kNameFieldNumber,
field.ftrace_name);
- success &= ParseField(field, start, end, table, generic_field, metadata);
+ success &= ParseField(field, start, end, generic_field, metadata);
}
} else { // Parse all other events.
for (const Field& field : info.fields) {
- success &= ParseField(field, start, end, table, nested, metadata);
+ success &= ParseField(field, start, end, nested, metadata);
}
}
@@ -681,7 +595,6 @@ bool CpuReader::ParseEvent(uint16_t ftrace_event_id,
bool CpuReader::ParseField(const Field& field,
const uint8_t* start,
const uint8_t* end,
- const ProtoTranslationTable* table,
protozero::Message* message,
FtraceMetadata* metadata) {
PERFETTO_DCHECK(start + field.ftrace_offset + field.ftrace_size <= end);
@@ -725,22 +638,10 @@ bool CpuReader::ParseField(const Field& field,
field_id, message);
case kCStringToString:
// TODO(hjd): Kernel-dive to check this how size:0 char fields work.
- return ReadIntoString(field_start, end, field_id, message);
- case kStringPtrToString: {
- uint64_t n = 0;
- // The ftrace field may be 8 or 4 bytes and we need to copy it into the
- // bottom of n. In the unlikely case where the field is >8 bytes we
- // should avoid making things worse by corrupting the stack but we
- // don't need to handle it correctly.
- size_t size = std::min<size_t>(field.ftrace_size, sizeof(n));
- memcpy(base::AssumeLittleEndian(&n),
- reinterpret_cast<const void*>(field_start), size);
- // Look up the adddress in the printk format map and write it into the
- // proto.
- base::StringView name = table->LookupTraceString(n);
- message->AppendBytes(field_id, name.begin(), name.size());
+ return ReadIntoString(field_start, end, field.proto_field_id, message);
+ case kStringPtrToString:
+ // TODO(hjd): Figure out how to read these.
return true;
- }
case kDataLocToString:
return ReadDataLoc(start, field_start, end, field, message);
case kBoolToUint32:
@@ -767,9 +668,6 @@ bool CpuReader::ParseField(const Field& field,
case kDevId64ToUint64:
ReadDevId<uint64_t>(field_start, field_id, message, metadata);
return true;
- case kFtraceSymAddr64ToUint64:
- ReadSymbolAddr<uint64_t>(field_start, field_id, message, metadata);
- return true;
case kInvalidTranslationStrategy:
break;
}