aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorPrimiano Tucci <primiano@google.com>2018-04-30 14:41:25 +0100
committerPrimiano Tucci <primiano@google.com>2018-05-01 10:11:14 +0100
commite81a3eda4818e0b6fce302c89eaaeb99459afa2a (patch)
tree8b31ab217ef7682d222f2da12016bb8beb1a2a76
parent9ce66e8c365b9ed3ddd88a168f9d19f075820788 (diff)
downloadperfetto-e81a3eda4818e0b6fce302c89eaaeb99459afa2a.tar.gz
Add stats about dropped events to the trace
Bug: 78765090 Change-Id: I673ce7bd6bb671ffa73faaf7f72a651b7e160901 Merged-In: I673ce7bd6bb671ffa73faaf7f72a651b7e160901 (cherry picked from commit 5e33cad06edd950934f5a1e925ed6b07125c3a47)
-rw-r--r--Android.bp8
-rw-r--r--protos/perfetto/trace/BUILD.gn1
-rw-r--r--protos/perfetto/trace/trace_packet.proto3
-rw-r--r--protos/perfetto/trace/trace_stats.proto85
-rw-r--r--protos/perfetto/trace/trusted_packet.proto6
-rw-r--r--src/tracing/core/packet_stream_validator.cc4
-rw-r--r--src/tracing/core/service_impl.cc55
-rw-r--r--src/tracing/core/service_impl.h5
-rw-r--r--src/tracing/core/trace_buffer.cc29
-rw-r--r--src/tracing/core/trace_buffer.h27
-rw-r--r--src/tracing/core/trace_packet_unittest.cc4
-rw-r--r--src/tracing/test/tracing_integration_test.cc110
-rw-r--r--test/test_helper.cc4
13 files changed, 276 insertions, 65 deletions
diff --git a/Android.bp b/Android.bp
index f1d42cb0a..34b35eba0 100644
--- a/Android.bp
+++ b/Android.bp
@@ -2931,6 +2931,7 @@ genrule {
name: "perfetto_protos_perfetto_trace_minimal_lite_gen",
srcs: [
"protos/perfetto/trace/clock_snapshot.proto",
+ "protos/perfetto/trace/trace_stats.proto",
"protos/perfetto/trace/trusted_packet.proto",
],
tools: [
@@ -2939,6 +2940,7 @@ genrule {
cmd: "mkdir -p $(genDir)/external/perfetto/protos && $(location aprotoc) --cpp_out=$(genDir)/external/perfetto/protos --proto_path=external/perfetto/protos $(in)",
out: [
"external/perfetto/protos/perfetto/trace/clock_snapshot.pb.cc",
+ "external/perfetto/protos/perfetto/trace/trace_stats.pb.cc",
"external/perfetto/protos/perfetto/trace/trusted_packet.pb.cc",
],
}
@@ -2948,6 +2950,7 @@ genrule {
name: "perfetto_protos_perfetto_trace_minimal_lite_gen_headers",
srcs: [
"protos/perfetto/trace/clock_snapshot.proto",
+ "protos/perfetto/trace/trace_stats.proto",
"protos/perfetto/trace/trusted_packet.proto",
],
tools: [
@@ -2956,6 +2959,7 @@ genrule {
cmd: "mkdir -p $(genDir)/external/perfetto/protos && $(location aprotoc) --cpp_out=$(genDir)/external/perfetto/protos --proto_path=external/perfetto/protos $(in)",
out: [
"external/perfetto/protos/perfetto/trace/clock_snapshot.pb.h",
+ "external/perfetto/protos/perfetto/trace/trace_stats.pb.h",
"external/perfetto/protos/perfetto/trace/trusted_packet.pb.h",
],
export_include_dirs: [
@@ -3039,6 +3043,7 @@ genrule {
"protos/perfetto/trace/test_event.proto",
"protos/perfetto/trace/trace.proto",
"protos/perfetto/trace/trace_packet.proto",
+ "protos/perfetto/trace/trace_stats.proto",
"protos/perfetto/trace/trusted_packet.proto",
],
tools: [
@@ -3051,6 +3056,7 @@ genrule {
"external/perfetto/protos/perfetto/trace/test_event.pbzero.cc",
"external/perfetto/protos/perfetto/trace/trace.pbzero.cc",
"external/perfetto/protos/perfetto/trace/trace_packet.pbzero.cc",
+ "external/perfetto/protos/perfetto/trace/trace_stats.pbzero.cc",
"external/perfetto/protos/perfetto/trace/trusted_packet.pbzero.cc",
],
}
@@ -3063,6 +3069,7 @@ genrule {
"protos/perfetto/trace/test_event.proto",
"protos/perfetto/trace/trace.proto",
"protos/perfetto/trace/trace_packet.proto",
+ "protos/perfetto/trace/trace_stats.proto",
"protos/perfetto/trace/trusted_packet.proto",
],
tools: [
@@ -3075,6 +3082,7 @@ genrule {
"external/perfetto/protos/perfetto/trace/test_event.pbzero.h",
"external/perfetto/protos/perfetto/trace/trace.pbzero.h",
"external/perfetto/protos/perfetto/trace/trace_packet.pbzero.h",
+ "external/perfetto/protos/perfetto/trace/trace_stats.pbzero.h",
"external/perfetto/protos/perfetto/trace/trusted_packet.pbzero.h",
],
export_include_dirs: [
diff --git a/protos/perfetto/trace/BUILD.gn b/protos/perfetto/trace/BUILD.gn
index 814de076e..095813dfa 100644
--- a/protos/perfetto/trace/BUILD.gn
+++ b/protos/perfetto/trace/BUILD.gn
@@ -20,6 +20,7 @@ import("../../../gn/proto_library.gni")
# the generic ":lite" target
proto_sources_minimal = [
"clock_snapshot.proto",
+ "trace_stats.proto",
"trusted_packet.proto",
]
diff --git a/protos/perfetto/trace/trace_packet.proto b/protos/perfetto/trace/trace_packet.proto
index 6fcee5ac5..9a5f4948d 100644
--- a/protos/perfetto/trace/trace_packet.proto
+++ b/protos/perfetto/trace/trace_packet.proto
@@ -25,6 +25,7 @@ import "perfetto/trace/ftrace/ftrace_event_bundle.proto";
import "perfetto/trace/ftrace/ftrace_stats.proto";
import "perfetto/trace/ps/process_tree.proto";
import "perfetto/trace/test_event.proto";
+import "perfetto/trace/trace_stats.proto";
package perfetto.protos;
@@ -42,8 +43,10 @@ message TracePacket {
// IDs up to 32 are reserved for events that are quite frequent because they
// take only one byte to encode their preamble.
+
TraceConfig trace_config = 33;
FtraceStats ftrace_stats = 34;
+ TraceStats trace_stats = 35;
// This field is only used for testing.
TestEvent for_testing = 536870911; // 2^29 - 1, max field id for protos.
diff --git a/protos/perfetto/trace/trace_stats.proto b/protos/perfetto/trace/trace_stats.proto
new file mode 100644
index 000000000..40cb59988
--- /dev/null
+++ b/protos/perfetto/trace/trace_stats.proto
@@ -0,0 +1,85 @@
+/*
+ * Copyright (C) 2018 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.
+ */
+
+syntax = "proto2";
+option optimize_for = LITE_RUNTIME;
+
+package perfetto.protos;
+
+// Statistics for the internals of the tracing service.
+message TraceStats {
+ // From TraceBuffer::Stats.
+ message BufferStats {
+ // Num. bytes written into the circular buffer.
+ optional uint64 bytes_written = 1;
+
+ // Num. chunks (!= packets) written into the buffer.
+ optional uint64 chunks_written = 2;
+
+ // Num. chunks overwritten before they have been read (i.e. loss of data).
+ optional uint64 chunks_overwritten = 3;
+
+ // Num. times the ring buffer wrapped around.
+ optional uint64 write_wrap_count = 4;
+
+ // Num. out-of-band (OOB) patches that succeeded.
+ optional uint64 patches_succeeded = 5;
+
+ // Num. OOB patches that failed (e.g., the chunk to patch was gone).
+ optional uint64 patches_failed = 6;
+
+ // Num. readaheads (for large multi-chunk packet reads) that ended up in a
+ // successful packet read.
+ optional uint64 readaheads_succeeded = 7;
+
+ // Num. readaheads aborted because of missing chunks in the sequence stream.
+ // Note that a small number > 0 is totally expected: occasionally, when
+ // issuing a read, the very last packet in a sequence might be incomplete
+ // (because the producer is still writing it while we read). The read will
+ // stop at that point, for that sequence, increasing this counter.
+ optional uint64 readaheads_failed = 8;
+
+ // Num. of violations of the SharedMemoryABI found while writing or reading
+ // the buffer. This is an indication of either a bug in the producer(s) or
+ // malicious producer(s).
+ optional uint64 abi_violations = 9;
+ }
+
+ // Stats for the TraceBuffer(s) of the current trace session.
+ repeated BufferStats buffer_stats = 1;
+
+ // Num. producers connected (whether they are involved in the current tracing
+ // session or not).
+ optional uint32 producers_connected = 2;
+
+ // Num. producers ever seen for all trace sessions since startup (it's a good
+ // proxy for inferring num. producers crashed / killed).
+ optional uint64 producers_seen = 3;
+
+ // Num. data sources registered for all trace sessions.
+ optional uint32 data_sources_registered = 4;
+
+ // Num. data sources ever seen for all trace sessions since startupb
+ optional uint64 data_sources_seen = 5;
+
+ // Num. concurrently active tracing sessions.
+ optional uint32 tracing_sessions = 6;
+
+ // Num. buffers for all tracing session (not just the current one). This will
+ // be >= buffer_stats.size(), because the latter is only about the current
+ // session.
+ optional uint32 total_buffers = 7;
+}
diff --git a/protos/perfetto/trace/trusted_packet.proto b/protos/perfetto/trace/trusted_packet.proto
index 072cfd426..36262c81c 100644
--- a/protos/perfetto/trace/trusted_packet.proto
+++ b/protos/perfetto/trace/trusted_packet.proto
@@ -28,6 +28,7 @@ option optimize_for = LITE_RUNTIME;
import "perfetto/config/trace_config.proto";
import "perfetto/trace/clock_snapshot.proto";
+import "perfetto/trace/trace_stats.proto";
package perfetto.protos;
@@ -45,8 +46,7 @@ message TrustedPacket {
// uid == 0 and uid not set (the writer uses proto2).
oneof optional_trusted_uid { int32 trusted_uid = 3; };
- // The original trace config.
- TraceConfig trace_config = 33;
-
ClockSnapshot clock_snapshot = 6;
+ TraceConfig trace_config = 33;
+ TraceStats trace_stats = 35;
}
diff --git a/src/tracing/core/packet_stream_validator.cc b/src/tracing/core/packet_stream_validator.cc
index 8f6f21edf..536899943 100644
--- a/src/tracing/core/packet_stream_validator.cc
+++ b/src/tracing/core/packet_stream_validator.cc
@@ -46,6 +46,10 @@ bool PacketStreamValidator::Validate(const Slices& slices) {
if (packet.has_trace_config())
return false;
+ // Only the service is allowed to fill in the TraceStats.
+ if (packet.has_trace_stats())
+ return false;
+
// We are deliberately not checking for clock_snapshot for the moment. It's
// unclear if we want to allow producers to snapshot their clocks. Ideally we
// want a security model where producers can only snapshot their own clocks
diff --git a/src/tracing/core/service_impl.cc b/src/tracing/core/service_impl.cc
index 3bb107f22..7697e863a 100644
--- a/src/tracing/core/service_impl.cc
+++ b/src/tracing/core/service_impl.cc
@@ -53,6 +53,7 @@ namespace {
constexpr size_t kDefaultShmPageSize = base::kPageSize;
constexpr int kMaxBuffersPerConsumer = 128;
constexpr base::TimeMillis kClockSnapshotInterval(10 * 1000);
+constexpr base::TimeMillis kStatsSnapshotInterval(10 * 1000);
constexpr int kMinWriteIntoFilePeriodMs = 100;
constexpr int kDefaultWriteIntoFilePeriodMs = 5000;
constexpr int kFlushTimeoutMs = 1000;
@@ -80,6 +81,7 @@ ServiceImpl::ServiceImpl(std::unique_ptr<SharedMemory::Factory> shm_factory,
base::TaskRunner* task_runner)
: task_runner_(task_runner),
shm_factory_(std::move(shm_factory)),
+ uid_(getuid()),
buffer_ids_(kMaxTraceBufferID),
weak_ptr_factory_(this) {
PERFETTO_DCHECK(task_runner_);
@@ -520,7 +522,9 @@ void ServiceImpl::ReadBuffers(TracingSessionID tsid,
}
std::vector<TracePacket> packets;
+ packets.reserve(1024); // Just an educated guess to avoid trivial expansions.
MaybeSnapshotClocks(tracing_session, &packets);
+ MaybeSnapshotStats(tracing_session, &packets);
MaybeEmitTraceConfig(tracing_session, &packets);
size_t packets_bytes = 0; // SUM(slice.size() for each slice in |packets|).
@@ -798,7 +802,7 @@ void ServiceImpl::CreateDataSourceInstance(
PERFETTO_DCHECK(producer);
// An existing producer that is not ftrace could have registered itself as
// ftrace, we must not enable it in that case.
- if (lockdown_mode_ && producer->uid_ != getuid()) {
+ if (lockdown_mode_ && producer->uid_ != uid_) {
PERFETTO_DLOG("Lockdown mode: not enabling producer %hu", producer->id_);
return;
}
@@ -1053,7 +1057,52 @@ void ServiceImpl::MaybeSnapshotClocks(TracingSession* tracing_session,
c->set_timestamp(static_cast<uint64_t>(base::GetWallTimeNs().count()));
#endif // !PERFETTO_BUILDFLAG(PERFETTO_OS_MACOSX)
- packet.set_trusted_uid(static_cast<int32_t>(getuid()));
+ packet.set_trusted_uid(static_cast<int32_t>(uid_));
+ Slice slice = Slice::Allocate(static_cast<size_t>(packet.ByteSize()));
+ PERFETTO_CHECK(packet.SerializeWithCachedSizesToArray(slice.own_data()));
+ packets->emplace_back();
+ packets->back().AddSlice(std::move(slice));
+}
+
+void ServiceImpl::MaybeSnapshotStats(TracingSession* tracing_session,
+ std::vector<TracePacket>* packets) {
+ base::TimeMillis now = base::GetWallTimeMs();
+ if (now < tracing_session->last_stats_snapshot + kStatsSnapshotInterval)
+ return;
+ tracing_session->last_stats_snapshot = now;
+
+ protos::TrustedPacket packet;
+ packet.set_trusted_uid(static_cast<int32_t>(uid_));
+
+ protos::TraceStats* trace_stats = packet.mutable_trace_stats();
+ trace_stats->set_producers_connected(
+ static_cast<uint32_t>(producers_.size()));
+ trace_stats->set_producers_seen(last_producer_id_);
+ trace_stats->set_data_sources_registered(
+ static_cast<uint32_t>(data_sources_.size()));
+ trace_stats->set_data_sources_seen(last_data_source_instance_id_);
+ trace_stats->set_tracing_sessions(
+ static_cast<uint32_t>(tracing_sessions_.size()));
+ trace_stats->set_total_buffers(static_cast<uint32_t>(buffers_.size()));
+
+ for (BufferID buf_id : tracing_session->buffers_index) {
+ TraceBuffer* buf = GetBufferByID(buf_id);
+ if (!buf) {
+ PERFETTO_DCHECK(false);
+ continue;
+ }
+ auto* buf_stats_proto = trace_stats->add_buffer_stats();
+ const TraceBuffer::Stats& buf_stats = buf->stats();
+ buf_stats_proto->set_bytes_written(buf_stats.bytes_written);
+ buf_stats_proto->set_chunks_written(buf_stats.chunks_written);
+ buf_stats_proto->set_chunks_overwritten(buf_stats.chunks_overwritten);
+ buf_stats_proto->set_write_wrap_count(buf_stats.write_wrap_count);
+ buf_stats_proto->set_patches_succeeded(buf_stats.patches_succeeded);
+ buf_stats_proto->set_patches_failed(buf_stats.patches_failed);
+ buf_stats_proto->set_readaheads_succeeded(buf_stats.readaheads_succeeded);
+ buf_stats_proto->set_readaheads_failed(buf_stats.readaheads_failed);
+ buf_stats_proto->set_abi_violations(buf_stats.abi_violations);
+ } // for (buf in session).
Slice slice = Slice::Allocate(static_cast<size_t>(packet.ByteSize()));
PERFETTO_CHECK(packet.SerializeWithCachedSizesToArray(slice.own_data()));
packets->emplace_back();
@@ -1067,7 +1116,7 @@ void ServiceImpl::MaybeEmitTraceConfig(TracingSession* tracing_session,
tracing_session->did_emit_config = true;
protos::TrustedPacket packet;
tracing_session->config.ToProto(packet.mutable_trace_config());
- packet.set_trusted_uid(static_cast<int32_t>(getuid()));
+ packet.set_trusted_uid(static_cast<int32_t>(uid_));
Slice slice = Slice::Allocate(static_cast<size_t>(packet.ByteSize()));
PERFETTO_CHECK(packet.SerializeWithCachedSizesToArray(slice.own_data()));
packets->emplace_back();
diff --git a/src/tracing/core/service_impl.h b/src/tracing/core/service_impl.h
index c6f79368a..b657a4ab4 100644
--- a/src/tracing/core/service_impl.h
+++ b/src/tracing/core/service_impl.h
@@ -240,6 +240,9 @@ class ServiceImpl : public Service {
// When the last clock snapshot was emitted into the output stream.
base::TimeMillis last_clock_snapshot = {};
+ // When the last TraceStats snapshot was emitted into the output stream.
+ base::TimeMillis last_stats_snapshot = {};
+
// Whether we mirrored the trace config back to the trace output yet.
bool did_emit_config = false;
@@ -276,6 +279,7 @@ class ServiceImpl : public Service {
void MaybeSnapshotClocks(TracingSession*, std::vector<TracePacket>*);
void MaybeEmitTraceConfig(TracingSession*, std::vector<TracePacket>*);
+ void MaybeSnapshotStats(TracingSession*, std::vector<TracePacket>*);
void OnFlushTimeout(TracingSessionID, FlushRequestID);
TraceBuffer* GetBufferByID(BufferID);
@@ -285,6 +289,7 @@ class ServiceImpl : public Service {
DataSourceInstanceID last_data_source_instance_id_ = 0;
TracingSessionID last_tracing_session_id_ = 0;
FlushRequestID last_flush_request_id_ = 0;
+ uid_t uid_ = 0;
// Buffer IDs are global across all consumers (because a Producer can produce
// data for more than one trace session, hence more than one consumer).
diff --git a/src/tracing/core/trace_buffer.cc b/src/tracing/core/trace_buffer.cc
index 8ec272fad..167b6b561 100644
--- a/src/tracing/core/trace_buffer.cc
+++ b/src/tracing/core/trace_buffer.cc
@@ -114,6 +114,7 @@ void TraceBuffer::CopyChunkUntrusted(ProducerID producer_id_trusted,
const size_t record_size =
base::AlignUp<sizeof(ChunkRecord)>(size + sizeof(ChunkRecord));
if (PERFETTO_UNLIKELY(record_size > max_chunk_size_)) {
+ stats_.abi_violations++;
PERFETTO_DCHECK(suppress_sanity_dchecks_for_testing_);
return;
}
@@ -167,11 +168,14 @@ void TraceBuffer::CopyChunkUntrusted(ProducerID producer_id_trusted,
// Now first insert the new chunk. At the end, if necessary, add the padding.
ChunkMeta::Key key(record);
+ stats_.chunks_written++;
+ stats_.bytes_written += size;
auto it_and_inserted =
index_.emplace(key, ChunkMeta(GetChunkRecordAt(wptr_), num_fragments,
chunk_flags, producer_uid_trusted));
if (PERFETTO_UNLIKELY(!it_and_inserted.second)) {
// More likely a producer bug, but could also be a malicious producer.
+ stats_.abi_violations++;
PERFETTO_DCHECK(suppress_sanity_dchecks_for_testing_);
index_.erase(it_and_inserted.first);
index_.emplace(key, ChunkMeta(GetChunkRecordAt(wptr_), num_fragments,
@@ -226,7 +230,15 @@ size_t TraceBuffer::DeleteNextChunksFor(size_t bytes_to_clear) {
// records are not part of the index).
if (PERFETTO_LIKELY(!next_chunk.is_padding)) {
ChunkMeta::Key key(next_chunk);
- const size_t removed = index_.erase(key);
+ auto it = index_.find(key);
+ bool removed = false;
+ if (PERFETTO_LIKELY(it != index_.end())) {
+ const ChunkMeta& meta = it->second;
+ if (PERFETTO_UNLIKELY(meta.num_fragments_read < meta.num_fragments))
+ stats_.chunks_overwritten++;
+ index_.erase(it);
+ removed = true;
+ }
TRACE_BUFFER_DLOG(" del index {%" PRIu32 ",%" PRIu32
",%u} @ [%lu - %lu] %zu",
key.producer_id, key.writer_id, key.chunk_id,
@@ -266,7 +278,7 @@ bool TraceBuffer::TryPatchChunkContents(ProducerID producer_id,
ChunkMeta::Key key(producer_id, writer_id, chunk_id);
auto it = index_.find(key);
if (it == index_.end()) {
- stats_.failed_patches++;
+ stats_.patches_failed++;
return false;
}
ChunkMeta& chunk_meta = it->second;
@@ -296,7 +308,7 @@ bool TraceBuffer::TryPatchChunkContents(ProducerID producer_id,
ptr > chunk_end - Patch::kSize) {
// Either the IPC was so slow and in the meantime the writer managed to
// wrap over |chunk_id| or the producer sent a malicious IPC.
- stats_.failed_patches++;
+ stats_.patches_failed++;
return false;
}
@@ -312,7 +324,7 @@ bool TraceBuffer::TryPatchChunkContents(ProducerID producer_id,
"Chunk raw (after patch): %s",
HexDump(chunk_begin, chunk_meta.chunk_record->size).c_str());
- stats_.succeeded_patches += patches_size;
+ stats_.patches_succeeded += patches_size;
if (!other_patches_pending) {
chunk_meta.flags &= ~kChunkNeedsPatching;
chunk_meta.chunk_record->flags = chunk_meta.flags;
@@ -488,6 +500,7 @@ bool TraceBuffer::ReadNextTracePacket(TracePacket* packet,
// In extremely rare cases (producer bugged / malicious) the chunk might
// contain an invalid fragment. In such case we don't want to stall the
// sequence but just skip the chunk and move on.
+ stats_.abi_violations++;
PERFETTO_DCHECK(suppress_sanity_dchecks_for_testing_);
break;
}
@@ -495,7 +508,7 @@ bool TraceBuffer::ReadNextTracePacket(TracePacket* packet,
PERFETTO_DCHECK(action == kTryReadAhead);
ReadAheadResult ra_res = ReadAhead(packet);
if (ra_res == ReadAheadResult::kSucceededReturnSlices) {
- stats_.fragment_readahead_successes++;
+ stats_.readaheads_succeeded++;
*producer_uid = trusted_uid;
return true;
}
@@ -503,7 +516,7 @@ bool TraceBuffer::ReadNextTracePacket(TracePacket* packet,
if (ra_res == ReadAheadResult::kFailedMoveToNextSequence) {
// readahead didn't find a contigous packet sequence. We'll try again
// on the next ReadPacket() call.
- stats_.fragment_readahead_failures++;
+ stats_.readaheads_failed++;
// TODO(primiano): optimization: this MoveToEnd() is the reason why
// MoveNext() (that is called in the outer for(;;MoveNext)) needs to
@@ -593,6 +606,7 @@ TraceBuffer::ReadAheadResult TraceBuffer::ReadAhead(TracePacket* packet) {
PERFETTO_DCHECK(read_iter_.cur == it.cur);
if (PERFETTO_UNLIKELY(packet_corruption)) {
+ stats_.abi_violations++;
PERFETTO_DCHECK(suppress_sanity_dchecks_for_testing_);
*packet = TracePacket(); // clear.
return ReadAheadResult::kFailedStayOnSameSequence;
@@ -618,6 +632,7 @@ bool TraceBuffer::ReadNextPacketInChunk(ChunkMeta* chunk_meta,
packet_begin >= record_end)) {
// The producer has a bug or is malicious and did declare that the chunk
// contains more packets beyond its boundaries.
+ stats_.abi_violations++;
PERFETTO_DCHECK(suppress_sanity_dchecks_for_testing_);
return false;
}
@@ -635,6 +650,7 @@ bool TraceBuffer::ReadNextPacketInChunk(ChunkMeta* chunk_meta,
const uint8_t* next_packet = packet_data + packet_size;
if (PERFETTO_UNLIKELY(next_packet <= packet_begin ||
next_packet > record_end)) {
+ stats_.abi_violations++;
PERFETTO_DCHECK(suppress_sanity_dchecks_for_testing_);
chunk_meta->cur_fragment_offset = 0;
chunk_meta->num_fragments_read = chunk_meta->num_fragments;
@@ -645,6 +661,7 @@ bool TraceBuffer::ReadNextPacketInChunk(ChunkMeta* chunk_meta,
chunk_meta->num_fragments_read++;
if (PERFETTO_UNLIKELY(packet_size == 0)) {
+ stats_.abi_violations++;
PERFETTO_DCHECK(suppress_sanity_dchecks_for_testing_);
return false;
}
diff --git a/src/tracing/core/trace_buffer.h b/src/tracing/core/trace_buffer.h
index 1998be51c..445c7a8b9 100644
--- a/src/tracing/core/trace_buffer.h
+++ b/src/tracing/core/trace_buffer.h
@@ -112,7 +112,7 @@ class TracePacket;
// -----------------------
// This class supports one reader only (the consumer). Reads are NOT idempotent
// as they move the read cursors around. Reading back the buffer is the most
-// conceptually complex part. The ReadPacket() method, in fact, operates with
+// conceptually complex part. The ReadNextTracePacket() method operates with
// whole packet granularity. Packets are returned only when all their fragments
// are available.
// This class takes care of:
@@ -122,19 +122,24 @@ class TracePacket;
// - Detecting holes in packet fragments (because of loss of chunks).
// Reads guarantee that packets for the same sequence are read in FIFO order
// (according to their ChunkID), but don't give any guarantee about the read
-// order of packets from different sequences (see ReadPacket() comments below).
+// order of packets from different sequences, see comments in
+// ReadNextTracePacket() below.
class TraceBuffer {
public:
static const size_t InlineChunkHeaderSize; // For test/fake_packet.{cc,h}.
+ // Maintain these fields consistent with trace_stats.proto. See comments in
+ // the .proto for the semantic of these fields.
struct Stats {
- size_t failed_patches = 0;
- size_t succeeded_patches = 0;
- size_t fragment_readahead_successes = 0;
- size_t fragment_readahead_failures = 0;
- size_t write_wrap_count = 0;
- // TODO(primiano): add packets_{read,written}.
- // TODO(primiano): add bytes_{read,written}.
+ uint64_t bytes_written = 0;
+ uint64_t chunks_written = 0;
+ uint64_t chunks_overwritten = 0;
+ uint64_t write_wrap_count = 0;
+ uint64_t patches_succeeded = 0;
+ uint64_t patches_failed = 0;
+ uint64_t readaheads_succeeded = 0;
+ uint64_t readaheads_failed = 0;
+ uint64_t abi_violations = 0;
// TODO(primiano): add bytes_lost_for_padding.
};
@@ -336,8 +341,8 @@ class TraceBuffer {
// Allows to iterate over a sub-sequence of |index_| for all keys belonging to
// the same {ProducerID,WriterID}. Furthermore takes into account the wrapping
- // of ChunkID. Instances are valid only as long as the |index_| is not
- // altered (can be used safely only between adjacent ReadPacket() calls).
+ // of ChunkID. Instances are valid only as long as the |index_| is not altered
+ // (can be used safely only between adjacent ReadNextTracePacket() calls).
// The order of the iteration will proceed in the following order:
// |wrapping_id| + 1 -> |seq_end|, |seq_begin| -> |wrapping_id|.
// Practical example:
diff --git a/src/tracing/core/trace_packet_unittest.cc b/src/tracing/core/trace_packet_unittest.cc
index 8aab213ed..61bc0018b 100644
--- a/src/tracing/core/trace_packet_unittest.cc
+++ b/src/tracing/core/trace_packet_unittest.cc
@@ -39,6 +39,10 @@ static_assert(protos::TracePacket::kTraceConfigFieldNumber ==
protos::TrustedPacket::kTraceConfigFieldNumber,
"trace_config field id mismatch");
+static_assert(protos::TracePacket::kTraceStatsFieldNumber ==
+ protos::TrustedPacket::kTraceStatsFieldNumber,
+ "trace_stats field id mismatch");
+
static_assert(protos::TracePacket::kClockSnapshotFieldNumber ==
protos::TrustedPacket::kClockSnapshotFieldNumber,
"clock_snapshot field id mismatch");
diff --git a/src/tracing/test/tracing_integration_test.cc b/src/tracing/test/tracing_integration_test.cc
index 3b7f90609..f90c1b1e6 100644
--- a/src/tracing/test/tracing_integration_test.cc
+++ b/src/tracing/test/tracing_integration_test.cc
@@ -81,6 +81,25 @@ class MockConsumer : public Consumer {
}
};
+void CheckTraceStats(const protos::TracePacket& packet) {
+ EXPECT_TRUE(packet.has_trace_stats());
+ EXPECT_GE(packet.trace_stats().producers_seen(), 1);
+ EXPECT_EQ(1, packet.trace_stats().producers_connected());
+ EXPECT_EQ(1, packet.trace_stats().data_sources_registered());
+ EXPECT_EQ(1, packet.trace_stats().tracing_sessions());
+ EXPECT_EQ(1, packet.trace_stats().total_buffers());
+ EXPECT_EQ(1, packet.trace_stats().buffer_stats_size());
+
+ const auto& buf_stats = packet.trace_stats().buffer_stats(0);
+ EXPECT_GT(buf_stats.bytes_written(), 0);
+ EXPECT_GT(buf_stats.chunks_written(), 0);
+ EXPECT_EQ(0, buf_stats.chunks_overwritten());
+ EXPECT_EQ(0, buf_stats.write_wrap_count());
+ EXPECT_EQ(0, buf_stats.patches_failed());
+ EXPECT_EQ(0, buf_stats.readaheads_failed());
+ EXPECT_EQ(0, buf_stats.abi_violations());
+}
+
class TracingIntegrationTest : public ::testing::Test {
public:
void SetUp() override {
@@ -206,57 +225,61 @@ TEST_F(TracingIntegrationTest, WithIPCTransport) {
size_t num_pack_rx = 0;
bool saw_clock_snapshot = false;
bool saw_trace_config = false;
+ bool saw_trace_stats = false;
auto all_packets_rx = task_runner_->CreateCheckpoint("all_packets_rx");
EXPECT_CALL(consumer_, OnTracePackets(_, _))
- .WillRepeatedly(Invoke([&num_pack_rx, all_packets_rx, &trace_config,
- &saw_clock_snapshot, &saw_trace_config](
- std::vector<TracePacket>* packets,
- bool has_more) {
+ .WillRepeatedly(
+ Invoke([&num_pack_rx, all_packets_rx, &trace_config,
+ &saw_clock_snapshot, &saw_trace_config, &saw_trace_stats](
+ std::vector<TracePacket>* packets, bool has_more) {
#if PERFETTO_BUILDFLAG(PERFETTO_OS_MACOSX)
- const int kExpectedMinNumberOfClocks = 1;
+ const int kExpectedMinNumberOfClocks = 1;
#else
- const int kExpectedMinNumberOfClocks = 6;
+ const int kExpectedMinNumberOfClocks = 6;
#endif
- for (auto& encoded_packet : *packets) {
- protos::TracePacket packet;
- ASSERT_TRUE(encoded_packet.Decode(&packet));
- if (packet.has_for_testing()) {
- char buf[8];
- sprintf(buf, "evt_%zu", num_pack_rx++);
- EXPECT_EQ(std::string(buf), packet.for_testing().str());
- } else if (packet.has_clock_snapshot()) {
- EXPECT_GE(packet.clock_snapshot().clocks_size(),
- kExpectedMinNumberOfClocks);
- saw_clock_snapshot = true;
- } else if (packet.has_trace_config()) {
- protos::TraceConfig config_proto;
- trace_config.ToProto(&config_proto);
- Slice expected_slice =
- Slice::Allocate(static_cast<size_t>(config_proto.ByteSize()));
- config_proto.SerializeWithCachedSizesToArray(
- expected_slice.own_data());
- Slice actual_slice = Slice::Allocate(
- static_cast<size_t>(packet.trace_config().ByteSize()));
- packet.trace_config().SerializeWithCachedSizesToArray(
- actual_slice.own_data());
- EXPECT_EQ(
- std::string(
- reinterpret_cast<const char*>(expected_slice.own_data()),
- expected_slice.size),
- std::string(
- reinterpret_cast<const char*>(actual_slice.own_data()),
- actual_slice.size));
- saw_trace_config = true;
- }
- }
- if (!has_more)
- all_packets_rx();
- }));
+ for (auto& encoded_packet : *packets) {
+ protos::TracePacket packet;
+ ASSERT_TRUE(encoded_packet.Decode(&packet));
+ if (packet.has_for_testing()) {
+ char buf[8];
+ sprintf(buf, "evt_%zu", num_pack_rx++);
+ EXPECT_EQ(std::string(buf), packet.for_testing().str());
+ } else if (packet.has_clock_snapshot()) {
+ EXPECT_GE(packet.clock_snapshot().clocks_size(),
+ kExpectedMinNumberOfClocks);
+ saw_clock_snapshot = true;
+ } else if (packet.has_trace_config()) {
+ protos::TraceConfig config_proto;
+ trace_config.ToProto(&config_proto);
+ Slice expected_slice = Slice::Allocate(
+ static_cast<size_t>(config_proto.ByteSize()));
+ config_proto.SerializeWithCachedSizesToArray(
+ expected_slice.own_data());
+ Slice actual_slice = Slice::Allocate(
+ static_cast<size_t>(packet.trace_config().ByteSize()));
+ packet.trace_config().SerializeWithCachedSizesToArray(
+ actual_slice.own_data());
+ EXPECT_EQ(std::string(reinterpret_cast<const char*>(
+ expected_slice.own_data()),
+ expected_slice.size),
+ std::string(reinterpret_cast<const char*>(
+ actual_slice.own_data()),
+ actual_slice.size));
+ saw_trace_config = true;
+ } else if (packet.has_trace_stats()) {
+ saw_trace_stats = true;
+ CheckTraceStats(packet);
+ }
+ }
+ if (!has_more)
+ all_packets_rx();
+ }));
task_runner_->RunUntilCheckpoint("all_packets_rx");
ASSERT_EQ(kNumPackets, num_pack_rx);
EXPECT_TRUE(saw_clock_snapshot);
EXPECT_TRUE(saw_trace_config);
+ EXPECT_TRUE(saw_trace_stats);
// Disable tracing.
consumer_endpoint_->DisableTracing();
@@ -327,13 +350,18 @@ TEST_F(TracingIntegrationTest, WriteIntoFile) {
protos::Trace tmp_trace;
ASSERT_TRUE(tmp_trace.ParseFromArray(tmp_buf, static_cast<int>(rsize)));
size_t num_test_packet = 0;
+ bool saw_trace_stats = false;
for (int i = 0; i < tmp_trace.packet_size(); i++) {
const protos::TracePacket& packet = tmp_trace.packet(i);
if (packet.has_for_testing()) {
ASSERT_EQ("evt_" + std::to_string(num_test_packet++),
packet.for_testing().str());
+ } else if (packet.has_trace_stats()) {
+ saw_trace_stats = true;
+ CheckTraceStats(packet);
}
}
+ ASSERT_TRUE(saw_trace_stats);
}
// TODO(primiano): add tests to cover:
diff --git a/test/test_helper.cc b/test/test_helper.cc
index e5b78c927..71e12fd2c 100644
--- a/test/test_helper.cc
+++ b/test/test_helper.cc
@@ -60,8 +60,10 @@ void TestHelper::OnTraceData(std::vector<TracePacket> packets, bool has_more) {
for (auto& encoded_packet : packets) {
protos::TracePacket packet;
ASSERT_TRUE(encoded_packet.Decode(&packet));
- if (packet.has_clock_snapshot() || packet.has_trace_config())
+ if (packet.has_clock_snapshot() || packet.has_trace_config() ||
+ packet.has_trace_stats()) {
continue;
+ }
ASSERT_EQ(protos::TracePacket::kTrustedUid,
packet.optional_trusted_uid_case());
trace_.push_back(std::move(packet));