Support shmem-less ABI for the producer port
This CL introduces an "emulation mode" for the shared memory ABI for
sending trace data from producers to the service without cross-process
shared memory:
1. When the producer IPC client sees a non-unix socket name like TCP or
vsock, it enables the "emulation" mode for the socket ABI to serialize
chunk content (header and payload) into the CommitDataRequest proto
message. Then the chunk is released as free on the producer side.
2. When the service sees serialized data from the CommitDataRequest, it
constructs the chunk by adopting the serialized chunk in the proto
message. The chunk is discarded instead of being released to the shared
memory buffer.
Bug: 284258446
Change-Id: I65f96b4f00553c75b5f8af8c955427c681d10476
diff --git a/Android.bp b/Android.bp
index af3bf28..1d880c5 100644
--- a/Android.bp
+++ b/Android.bp
@@ -11557,6 +11557,7 @@
name: "perfetto_src_tracing_core_core",
srcs: [
"src/tracing/core/id_allocator.cc",
+ "src/tracing/core/in_process_shared_memory.cc",
"src/tracing/core/null_trace_writer.cc",
"src/tracing/core/shared_memory_abi.cc",
"src/tracing/core/shared_memory_arbiter_impl.cc",
@@ -11725,7 +11726,6 @@
"src/tracing/test/fake_packet.cc",
"src/tracing/test/mock_consumer.cc",
"src/tracing/test/mock_producer.cc",
- "src/tracing/test/test_shared_memory.cc",
"src/tracing/test/traced_value_test_support.cc",
],
}
diff --git a/BUILD b/BUILD
index 21de111..3d9316d 100644
--- a/BUILD
+++ b/BUILD
@@ -2986,6 +2986,8 @@
"src/tracing/core/histogram.h",
"src/tracing/core/id_allocator.cc",
"src/tracing/core/id_allocator.h",
+ "src/tracing/core/in_process_shared_memory.cc",
+ "src/tracing/core/in_process_shared_memory.h",
"src/tracing/core/null_trace_writer.cc",
"src/tracing/core/null_trace_writer.h",
"src/tracing/core/patch_list.h",
diff --git a/include/perfetto/ext/base/unix_socket.h b/include/perfetto/ext/base/unix_socket.h
index df95793..74ce2d9 100644
--- a/include/perfetto/ext/base/unix_socket.h
+++ b/include/perfetto/ext/base/unix_socket.h
@@ -88,8 +88,17 @@
// - @abstract_name : for abstract AF_UNIX sockets.
// - 1.2.3.4:8080 : for Inet sockets.
// - [::1]:8080 : for Inet6 sockets.
+// - vsock://-1:3000 : for VM sockets.
SockFamily GetSockFamily(const char* addr);
+// Returns whether inter-process shared memory is supported for the socket.
+inline bool SockShmemSupported(SockFamily sock_family) {
+ return sock_family == SockFamily::kUnix;
+}
+inline bool SockShmemSupported(const char* addr) {
+ return SockShmemSupported(GetSockFamily(addr));
+}
+
// UnixSocketRaw is a basic wrapper around sockets. It exposes wrapper
// methods that take care of most common pitfalls (e.g., marking fd as
// O_CLOEXEC, avoiding SIGPIPE, properly handling partial writes). It is used as
diff --git a/include/perfetto/ext/ipc/service.h b/include/perfetto/ext/ipc/service.h
index ea49834..08de20c 100644
--- a/include/perfetto/ext/ipc/service.h
+++ b/include/perfetto/ext/ipc/service.h
@@ -52,6 +52,8 @@
return base::ScopedFile();
}
+ bool use_shmem_emulation() { return use_shmem_emulation_; }
+
private:
friend class HostImpl;
ClientInfo client_info_;
@@ -66,6 +68,10 @@
// invocation per client that supplies an FD and the client needs to
// wait for this one to return before calling another one.
base::ScopedFile* received_fd_;
+
+ // Whether the socket needs to emulate shared memory buffer. Set by HostImpl
+ // when the service is exposed.
+ bool use_shmem_emulation_ = false;
};
} // namespace ipc
diff --git a/include/perfetto/ext/tracing/core/shared_memory_abi.h b/include/perfetto/ext/tracing/core/shared_memory_abi.h
index 567a295..451cafb 100644
--- a/include/perfetto/ext/tracing/core/shared_memory_abi.h
+++ b/include/perfetto/ext/tracing/core/shared_memory_abi.h
@@ -176,6 +176,11 @@
// kChunkBeingRead |
// | (Service) |
// +------------------------+
+ //
+ // The ABI has an "emulation mode" for transports where shared memory isn't
+ // supported. In this mode, kChunkBeingRead is skipped. A chunk in the
+ // kChunkComplete state is released as free after the producer serializes
+ // chunk content to the protobuf message.
enum ChunkState : uint32_t {
// The Chunk is free. The Service shall never touch it, the Producer can
// acquire it and transition it into kChunkBeingWritten.
@@ -226,6 +231,16 @@
// Keep this consistent with the PageLayout enum above.
static constexpr uint32_t kNumChunksForLayout[] = {0, 1, 2, 4, 7, 14, 0, 0};
+ enum class ShmemMode {
+ // The default mode, where the shared buffer is visible to both the producer
+ // and the service.
+ kDefault,
+
+ // The emulation mode, used for producer ports without shared memory. The
+ // state transitions are all done in the producer process.
+ kShmemEmulation,
+ };
+
// Layout of a Page.
// +===================================================+
// | Page header [8 bytes] |
@@ -443,13 +458,22 @@
uint8_t* begin_ = nullptr;
uint16_t size_ = 0;
uint8_t chunk_idx_ = 0;
+
+ public:
+ static constexpr size_t kMaxSize = 1ULL << sizeof(size_) * 8;
};
// Construct an instance from an existing shared memory buffer.
- SharedMemoryABI(uint8_t* start, size_t size, size_t page_size);
+ SharedMemoryABI(uint8_t* start,
+ size_t size,
+ size_t page_size,
+ ShmemMode mode);
SharedMemoryABI();
- void Initialize(uint8_t* start, size_t size, size_t page_size);
+ void Initialize(uint8_t* start,
+ size_t size,
+ size_t page_size,
+ ShmemMode mode);
uint8_t* start() const { return start_; }
uint8_t* end() const { return start_ + size_; }
@@ -536,6 +560,15 @@
uint32_t page_layout,
size_t chunk_idx);
+ // Creates a Chunk by adopting the given buffer (|data| and |size|) and chunk
+ // index. This is used for chunk data passed over the wire (e.g. tcp or
+ // vsock). The chunk should *not* be freed to the shared memory.
+ static Chunk MakeChunkFromSerializedData(uint8_t* data,
+ uint16_t size,
+ uint8_t chunk_idx) {
+ return Chunk(data, size, chunk_idx);
+ }
+
// Puts a chunk into the kChunkComplete state. Returns the page index.
size_t ReleaseChunkAsComplete(Chunk chunk) {
return ReleaseChunk(std::move(chunk), kChunkComplete);
@@ -595,6 +628,7 @@
uint8_t* start_ = nullptr;
size_t size_ = 0;
size_t page_size_ = 0;
+ bool use_shmem_emulation_ = false;
size_t num_pages_ = 0;
std::array<uint16_t, kNumPageLayouts> chunk_sizes_;
};
diff --git a/include/perfetto/ext/tracing/core/shared_memory_arbiter.h b/include/perfetto/ext/tracing/core/shared_memory_arbiter.h
index a430e2a..850d7bb 100644
--- a/include/perfetto/ext/tracing/core/shared_memory_arbiter.h
+++ b/include/perfetto/ext/tracing/core/shared_memory_arbiter.h
@@ -25,6 +25,7 @@
#include "perfetto/base/export.h"
#include "perfetto/ext/tracing/core/basic_types.h"
+#include "perfetto/ext/tracing/core/shared_memory_abi.h"
#include "perfetto/ext/tracing/core/tracing_service.h"
#include "perfetto/tracing/buffer_exhausted_policy.h"
@@ -41,6 +42,8 @@
// from the SharedMemory it receives from the Service-side.
class PERFETTO_EXPORT_COMPONENT SharedMemoryArbiter {
public:
+ using ShmemMode = SharedMemoryABI::ShmemMode;
+
virtual ~SharedMemoryArbiter();
// Creates a new TraceWriter and assigns it a new WriterID. The WriterID is
@@ -182,6 +185,7 @@
static std::unique_ptr<SharedMemoryArbiter> CreateInstance(
SharedMemory*,
size_t page_size,
+ ShmemMode,
TracingService::ProducerEndpoint*,
base::TaskRunner*);
@@ -202,9 +206,8 @@
//
// Implemented in src/core/shared_memory_arbiter_impl.cc. See CreateInstance()
// for comments about the arguments.
- static std::unique_ptr<SharedMemoryArbiter> CreateUnboundInstance(
- SharedMemory*,
- size_t page_size);
+ static std::unique_ptr<SharedMemoryArbiter>
+ CreateUnboundInstance(SharedMemory*, size_t page_size, ShmemMode mode);
};
} // namespace perfetto
diff --git a/protos/perfetto/common/commit_data_request.proto b/protos/perfetto/common/commit_data_request.proto
index 7e6a1a9..7a1cb89 100644
--- a/protos/perfetto/common/commit_data_request.proto
+++ b/protos/perfetto/common/commit_data_request.proto
@@ -32,6 +32,13 @@
// The target buffer it should be moved onto. The service will check that
// the producer is allowed to write into that buffer before the move.
optional uint32 target_buffer = 3;
+
+ // Sending the chunk data over the wire. Used for transports that don't
+ // support shared memory (e.g. vsock or TCP sockets). In the default case
+ // (tracing protocol over a Unix socket), this field is not used and tracing
+ // data is stored in the shmem buffer and referenced by the fields above.
+ // See |use_shemem_emulation| in the codebase for reference.
+ optional bytes data = 4;
}
repeated ChunksToMove chunks_to_move = 1;
diff --git a/protos/perfetto/ipc/producer_port.proto b/protos/perfetto/ipc/producer_port.proto
index d63f5bb..5c204cb 100644
--- a/protos/perfetto/ipc/producer_port.proto
+++ b/protos/perfetto/ipc/producer_port.proto
@@ -185,6 +185,11 @@
// chunks that have not yet been committed to it.
// This field has been introduced in Android S.
optional bool direct_smb_patching_supported = 2;
+
+ // Indicates whether the service would like to use SMB emulation for the
+ // connection, and request the client to send chunk data over the socket e.g.
+ // for remote connection from a VM guest.
+ optional bool use_shmem_emulation = 3;
}
// Arguments for rpc RegisterDataSource().
diff --git a/src/base/unix_socket_unittest.cc b/src/base/unix_socket_unittest.cc
index 083e930..a0d0bb3 100644
--- a/src/base/unix_socket_unittest.cc
+++ b/src/base/unix_socket_unittest.cc
@@ -1165,6 +1165,21 @@
#endif
}
+TEST_F(UnixSocketTest, ShmemSupported) {
+ ASSERT_EQ(SockShmemSupported(""), false);
+ ASSERT_EQ(SockShmemSupported("/path/to/sock"), true);
+ ASSERT_EQ(SockShmemSupported("local_dir_sock"), true);
+ ASSERT_EQ(SockShmemSupported("@abstract"), true);
+ ASSERT_EQ(SockShmemSupported("0.0.0.0:80"), false);
+ ASSERT_EQ(SockShmemSupported("127.0.0.1:80"), false);
+ ASSERT_EQ(SockShmemSupported("[effe::acca]:1234"), false);
+ ASSERT_EQ(SockShmemSupported("[::1]:123456"), false);
+#if PERFETTO_BUILDFLAG(PERFETTO_OS_LINUX) || \
+ PERFETTO_BUILDFLAG(PERFETTO_OS_ANDROID)
+ ASSERT_EQ(SockShmemSupported("vsock://-1:10000"), false);
+#endif
+}
+
} // namespace
} // namespace base
} // namespace perfetto
diff --git a/src/ipc/host_impl.cc b/src/ipc/host_impl.cc
index dfbf793..c193dfb 100644
--- a/src/ipc/host_impl.cc
+++ b/src/ipc/host_impl.cc
@@ -125,6 +125,7 @@
PERFETTO_DLOG("Duplicate ExposeService(): %s", service_name.c_str());
return false;
}
+ service->use_shmem_emulation_ = !base::SockShmemSupported(sock()->family());
ServiceID sid = ++last_service_id_;
ExposedService exposed_service(sid, service_name, std::move(service));
services_.emplace(sid, std::move(exposed_service));
diff --git a/src/tracing/BUILD.gn b/src/tracing/BUILD.gn
index b4a28a0..19b3205 100644
--- a/src/tracing/BUILD.gn
+++ b/src/tracing/BUILD.gn
@@ -225,6 +225,7 @@
"../../gn:default_deps",
"../../include/perfetto/tracing/core",
"../base",
+ "core",
"core:service",
]
sources = [ "internal/in_process_tracing_backend.cc" ]
diff --git a/src/tracing/core/BUILD.gn b/src/tracing/core/BUILD.gn
index bdd4f93..1539b8f 100644
--- a/src/tracing/core/BUILD.gn
+++ b/src/tracing/core/BUILD.gn
@@ -33,6 +33,8 @@
"histogram.h",
"id_allocator.cc",
"id_allocator.h",
+ "in_process_shared_memory.cc",
+ "in_process_shared_memory.h",
"null_trace_writer.cc",
"null_trace_writer.h",
"patch_list.h",
diff --git a/src/tracing/core/in_process_shared_memory.cc b/src/tracing/core/in_process_shared_memory.cc
new file mode 100644
index 0000000..65ae53a
--- /dev/null
+++ b/src/tracing/core/in_process_shared_memory.cc
@@ -0,0 +1,31 @@
+/*
+ * Copyright (C) 2023 The Android Open Source Project
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "src/tracing/core/in_process_shared_memory.h"
+
+namespace perfetto {
+
+InProcessSharedMemory::~InProcessSharedMemory() = default;
+InProcessSharedMemory::Factory::~Factory() = default;
+
+void* InProcessSharedMemory::start() const {
+ return mem_.Get();
+}
+size_t InProcessSharedMemory::size() const {
+ return mem_.size();
+}
+
+} // namespace perfetto
diff --git a/src/tracing/core/in_process_shared_memory.h b/src/tracing/core/in_process_shared_memory.h
new file mode 100644
index 0000000..353d44b
--- /dev/null
+++ b/src/tracing/core/in_process_shared_memory.h
@@ -0,0 +1,62 @@
+/*
+ * Copyright (C) 2023 The Android Open Source Project
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef SRC_TRACING_CORE_IN_PROCESS_SHARED_MEMORY_H_
+#define SRC_TRACING_CORE_IN_PROCESS_SHARED_MEMORY_H_
+
+#include <memory>
+
+#include "perfetto/ext/base/paged_memory.h"
+#include "perfetto/ext/tracing/core/shared_memory.h"
+
+namespace perfetto {
+
+// An implementation of the ShareMemory interface that allocates memory that can
+// only be shared intra-process.
+class InProcessSharedMemory : public SharedMemory {
+ public:
+ static constexpr size_t kDefaultSize = 128 * 1024;
+
+ // Default ctor used for intra-process shmem between a producer and the
+ // service.
+ explicit InProcessSharedMemory(size_t size)
+ : mem_(base::PagedMemory::Allocate(size)) {}
+ ~InProcessSharedMemory() override;
+
+ static std::unique_ptr<InProcessSharedMemory> Create(
+ size_t size = kDefaultSize) {
+ return std::make_unique<InProcessSharedMemory>(size);
+ }
+
+ // SharedMemory implementation.
+ void* start() const override;
+ size_t size() const override;
+
+ class Factory : public SharedMemory::Factory {
+ public:
+ ~Factory() override;
+ std::unique_ptr<SharedMemory> CreateSharedMemory(size_t size) override {
+ return InProcessSharedMemory::Create(size);
+ }
+ };
+
+ private:
+ base::PagedMemory mem_;
+};
+
+} // namespace perfetto
+
+#endif // SRC_TRACING_CORE_IN_PROCESS_SHARED_MEMORY_H_
diff --git a/src/tracing/core/shared_memory_abi.cc b/src/tracing/core/shared_memory_abi.cc
index a9098dc..ee9fb75 100644
--- a/src/tracing/core/shared_memory_abi.cc
+++ b/src/tracing/core/shared_memory_abi.cc
@@ -73,16 +73,19 @@
SharedMemoryABI::SharedMemoryABI(uint8_t* start,
size_t size,
- size_t page_size) {
- Initialize(start, size, page_size);
+ size_t page_size,
+ ShmemMode mode) {
+ Initialize(start, size, page_size, mode);
}
void SharedMemoryABI::Initialize(uint8_t* start,
size_t size,
- size_t page_size) {
+ size_t page_size,
+ ShmemMode mode) {
start_ = start;
size_ = size;
page_size_ = page_size;
+ use_shmem_emulation_ = mode == ShmemMode::kShmemEmulation;
num_pages_ = size / page_size;
chunk_sizes_ = InitChunkSizes(page_size);
static_assert(sizeof(PageHeader) == 8, "PageHeader size");
@@ -243,18 +246,21 @@
// could crash us by putting the chunk in an invalid state. This should
// gracefully fail. Keep a CHECK until then.
PERFETTO_CHECK(chunk.size() == page_chunk_size);
- const uint32_t chunk_state =
- ((layout >> (chunk_idx * kChunkShift)) & kChunkMask);
+ const uint32_t chunk_state = GetChunkStateFromLayout(layout, chunk_idx);
// Verify that the chunk is still in a state that allows the transition to
// |desired_chunk_state|. The only allowed transitions are:
// 1. kChunkBeingWritten -> kChunkComplete (Producer).
// 2. kChunkBeingRead -> kChunkFree (Service).
+ // Or in the emulation mode, the allowed transitions are:
+ // 1. kChunkBeingWritten -> kChunkComplete (Producer).
+ // 2. kChunkComplete -> kChunkFree (Producer).
ChunkState expected_chunk_state;
if (desired_chunk_state == kChunkComplete) {
expected_chunk_state = kChunkBeingWritten;
} else {
- expected_chunk_state = kChunkBeingRead;
+ expected_chunk_state =
+ use_shmem_emulation_ ? kChunkComplete : kChunkBeingRead;
}
// TODO(primiano): should not be a CHECK (same rationale of comment above).
diff --git a/src/tracing/core/shared_memory_abi_unittest.cc b/src/tracing/core/shared_memory_abi_unittest.cc
index 7069cd7..d3e1754 100644
--- a/src/tracing/core/shared_memory_abi_unittest.cc
+++ b/src/tracing/core/shared_memory_abi_unittest.cc
@@ -34,7 +34,8 @@
INSTANTIATE_TEST_SUITE_P(PageSize, SharedMemoryABITest, ValuesIn(kPageSizes));
TEST_P(SharedMemoryABITest, NominalCases) {
- SharedMemoryABI abi(buf(), buf_size(), page_size());
+ SharedMemoryABI abi(buf(), buf_size(), page_size(),
+ SharedMemoryABI::ShmemMode::kDefault);
ASSERT_EQ(buf(), abi.start());
ASSERT_EQ(buf() + buf_size(), abi.end());
@@ -190,5 +191,63 @@
}
}
+// Tests chunk state transition in the emulation mode.
+TEST_P(SharedMemoryABITest, ShmemEmulation) {
+ SharedMemoryABI abi(buf(), buf_size(), page_size(),
+ SharedMemoryABI::ShmemMode::kShmemEmulation);
+
+ for (size_t i = 0; i < kNumPages; i++) {
+ ASSERT_TRUE(abi.is_page_free(i));
+ ASSERT_FALSE(abi.is_page_complete(i));
+ // GetFreeChunks() should return 0 for an unpartitioned page.
+ ASSERT_EQ(0u, abi.GetFreeChunks(i));
+ }
+
+ ASSERT_TRUE(abi.TryPartitionPage(0, SharedMemoryABI::kPageDiv14));
+ ASSERT_EQ(0x3fffu, abi.GetFreeChunks(0));
+
+ ASSERT_FALSE(abi.is_page_free(0));
+
+ const size_t num_chunks =
+ SharedMemoryABI::GetNumChunksForLayout(abi.GetPageLayout(0));
+ Chunk chunks[14];
+
+ for (size_t chunk_idx = 0; chunk_idx < num_chunks; chunk_idx++) {
+ Chunk& chunk = chunks[chunk_idx];
+ ChunkHeader header{};
+
+ ASSERT_EQ(SharedMemoryABI::kChunkFree, abi.GetChunkState(0, chunk_idx));
+
+ chunk = abi.TryAcquireChunkForWriting(0, chunk_idx, &header);
+ ASSERT_TRUE(chunk.is_valid());
+ ASSERT_EQ(SharedMemoryABI::kChunkBeingWritten,
+ abi.GetChunkState(0, chunk_idx));
+
+ // Reacquiring the same chunk should fail.
+ ASSERT_FALSE(
+ abi.TryAcquireChunkForWriting(0, chunk_idx, &header).is_valid());
+ }
+
+ // Now release chunks and check the Release() logic.
+ for (size_t chunk_idx = 0; chunk_idx < num_chunks; chunk_idx++) {
+ Chunk& chunk = chunks[chunk_idx];
+
+ size_t res = abi.ReleaseChunkAsComplete(std::move(chunk));
+ ASSERT_EQ(0u, res);
+ ASSERT_EQ(SharedMemoryABI::kChunkComplete, abi.GetChunkState(0, chunk_idx));
+ }
+
+ for (size_t chunk_idx = 0; chunk_idx < num_chunks; chunk_idx++) {
+ Chunk chunk = abi.GetChunkUnchecked(0, abi.GetPageLayout(0), chunk_idx);
+
+ // If this was the last chunk in the page, the full page should be marked
+ // as free.
+ size_t res = abi.ReleaseChunkAsFree(std::move(chunk));
+ ASSERT_EQ(0u, res);
+ ASSERT_EQ(chunk_idx == num_chunks - 1, abi.is_page_free(0));
+ ASSERT_EQ(SharedMemoryABI::kChunkFree, abi.GetChunkState(0u, chunk_idx));
+ }
+}
+
} // namespace
} // namespace perfetto
diff --git a/src/tracing/core/shared_memory_arbiter_impl.cc b/src/tracing/core/shared_memory_arbiter_impl.cc
index 61b3974..dfbb66b 100644
--- a/src/tracing/core/shared_memory_arbiter_impl.cc
+++ b/src/tracing/core/shared_memory_arbiter_impl.cc
@@ -25,6 +25,7 @@
#include "perfetto/base/time.h"
#include "perfetto/ext/tracing/core/commit_data_request.h"
#include "perfetto/ext/tracing/core/shared_memory.h"
+#include "perfetto/ext/tracing/core/shared_memory_abi.h"
#include "src/tracing/core/null_trace_writer.h"
#include "src/tracing/core/trace_writer_impl.h"
@@ -56,31 +57,35 @@
std::unique_ptr<SharedMemoryArbiter> SharedMemoryArbiter::CreateInstance(
SharedMemory* shared_memory,
size_t page_size,
+ ShmemMode mode,
TracingService::ProducerEndpoint* producer_endpoint,
base::TaskRunner* task_runner) {
- return std::unique_ptr<SharedMemoryArbiterImpl>(
- new SharedMemoryArbiterImpl(shared_memory->start(), shared_memory->size(),
- page_size, producer_endpoint, task_runner));
+ return std::unique_ptr<SharedMemoryArbiterImpl>(new SharedMemoryArbiterImpl(
+ shared_memory->start(), shared_memory->size(), mode, page_size,
+ producer_endpoint, task_runner));
}
// static
std::unique_ptr<SharedMemoryArbiter> SharedMemoryArbiter::CreateUnboundInstance(
SharedMemory* shared_memory,
- size_t page_size) {
+ size_t page_size,
+ ShmemMode mode) {
return std::unique_ptr<SharedMemoryArbiterImpl>(new SharedMemoryArbiterImpl(
- shared_memory->start(), shared_memory->size(), page_size,
+ shared_memory->start(), shared_memory->size(), mode, page_size,
/*producer_endpoint=*/nullptr, /*task_runner=*/nullptr));
}
SharedMemoryArbiterImpl::SharedMemoryArbiterImpl(
void* start,
size_t size,
+ ShmemMode mode,
size_t page_size,
TracingService::ProducerEndpoint* producer_endpoint,
base::TaskRunner* task_runner)
: producer_endpoint_(producer_endpoint),
+ use_shmem_emulation_(mode == ShmemMode::kShmemEmulation),
task_runner_(task_runner),
- shmem_abi_(reinterpret_cast<uint8_t*>(start), size, page_size),
+ shmem_abi_(reinterpret_cast<uint8_t*>(start), size, page_size, mode),
active_writer_ids_(kMaxWriterID),
fully_bound_(task_runner && producer_endpoint),
was_always_bound_(fully_bound_),
@@ -269,12 +274,15 @@
}
}
+ CommitDataRequest::ChunksToMove* ctm = nullptr; // Set if chunk is valid.
// If a valid chunk is specified, return it and attach it to the request.
if (chunk.is_valid()) {
PERFETTO_DCHECK(chunk.writer_id() == writer_id);
uint8_t chunk_idx = chunk.chunk_idx();
bytes_pending_commit_ += chunk.size();
size_t page_idx;
+
+ ctm = commit_data_req_->add_chunks_to_move();
// If the chunk needs patching, it should not be marked as complete yet,
// because this would indicate to the service that the producer will not
// be writing to it anymore, while the producer might still apply patches
@@ -299,8 +307,6 @@
// DO NOT access |chunk| after this point, it has been std::move()-d
// above.
- CommitDataRequest::ChunksToMove* ctm =
- commit_data_req_->add_chunks_to_move();
ctm->set_page(static_cast<uint32_t>(page_idx));
ctm->set_chunk(chunk_idx);
ctm->set_target_buffer(target_buffer);
@@ -529,7 +535,7 @@
// Since we are about to notify the service of all batched chunks, it will
// not be possible to apply any more patches to them and we need to move
// them to kChunkComplete - otherwise the service won't look at them.
- for (auto& ctm : commit_data_req_->chunks_to_move()) {
+ for (auto& ctm : *commit_data_req_->mutable_chunks_to_move()) {
uint32_t layout = shmem_abi_.GetPageLayout(ctm.page());
auto chunk_state =
shmem_abi_.GetChunkStateFromLayout(layout, ctm.chunk());
@@ -537,12 +543,23 @@
// patching is also the subset of chunks that are still being written
// to. The rest of the chunks in |commit_data_req_| do not need patching
// and have already been marked as complete.
- if (chunk_state != SharedMemoryABI::kChunkBeingWritten)
- continue;
+ if (chunk_state == SharedMemoryABI::kChunkBeingWritten) {
+ auto chunk =
+ shmem_abi_.GetChunkUnchecked(ctm.page(), layout, ctm.chunk());
+ shmem_abi_.ReleaseChunkAsComplete(std::move(chunk));
+ }
- SharedMemoryABI::Chunk chunk =
- shmem_abi_.GetChunkUnchecked(ctm.page(), layout, ctm.chunk());
- shmem_abi_.ReleaseChunkAsComplete(std::move(chunk));
+ if (use_shmem_emulation_) {
+ // When running in the emulation mode:
+ // 1. serialize the chunk data to |ctm| as we won't modify the chunk
+ // anymore.
+ // 2. free the chunk as the service won't be able to do this.
+ auto chunk =
+ shmem_abi_.GetChunkUnchecked(ctm.page(), layout, ctm.chunk());
+ PERFETTO_CHECK(chunk.is_valid());
+ ctm.set_data(chunk.begin(), chunk.size());
+ shmem_abi_.ReleaseChunkAsFree(std::move(chunk));
+ }
}
req = std::move(commit_data_req_);
diff --git a/src/tracing/core/shared_memory_arbiter_impl.h b/src/tracing/core/shared_memory_arbiter_impl.h
index 29637d4..12316dd 100644
--- a/src/tracing/core/shared_memory_arbiter_impl.h
+++ b/src/tracing/core/shared_memory_arbiter_impl.h
@@ -109,8 +109,19 @@
// boundaries of the shared memory buffer. ProducerEndpoint and TaskRunner may
// be |nullptr| if created unbound, see
// SharedMemoryArbiter::CreateUnboundInstance().
+
+ // SharedMemoryArbiterImpl(void* start,
+ // size_t size,
+ // size_t page_size,
+ // TracingService::ProducerEndpoint*
+ // producer_endpoint, base::TaskRunner* task_runner) :
+ // SharedMemoryArbiterImpl(start, size, page_size, false, producer_endpoint,
+ // task_runner) {
+ // }
+
SharedMemoryArbiterImpl(void* start,
size_t size,
+ ShmemMode mode,
size_t page_size,
TracingService::ProducerEndpoint*,
base::TaskRunner*);
@@ -248,6 +259,10 @@
// Only accessed on |task_runner_| after the producer endpoint was bound.
TracingService::ProducerEndpoint* producer_endpoint_ = nullptr;
+ // Set to true when this instance runs in a emulation mode for a producer
+ // endpoint that doesn't support shared memory (e.g. vsock).
+ const bool use_shmem_emulation_ = false;
+
// --- Begin lock-protected members ---
std::mutex lock_;
diff --git a/src/tracing/core/shared_memory_arbiter_impl_unittest.cc b/src/tracing/core/shared_memory_arbiter_impl_unittest.cc
index 6b9443b..8f7184c 100644
--- a/src/tracing/core/shared_memory_arbiter_impl_unittest.cc
+++ b/src/tracing/core/shared_memory_arbiter_impl_unittest.cc
@@ -23,8 +23,10 @@
#include "perfetto/ext/tracing/core/shared_memory_abi.h"
#include "perfetto/ext/tracing/core/trace_packet.h"
#include "perfetto/ext/tracing/core/trace_writer.h"
+#include "perfetto/ext/tracing/core/tracing_service.h"
#include "src/base/test/gtest_test_suite.h"
#include "src/base/test/test_task_runner.h"
+#include "src/tracing/core/in_process_shared_memory.h"
#include "src/tracing/core/patch_list.h"
#include "src/tracing/test/aligned_buffer_test.h"
#include "src/tracing/test/mock_producer_endpoint.h"
@@ -42,6 +44,8 @@
using testing::NiceMock;
using testing::UnorderedElementsAreArray;
+using ShmemMode = SharedMemoryABI::ShmemMode;
+
class SharedMemoryArbiterImplTest : public AlignedBufferTest {
public:
void SetUp() override {
@@ -49,9 +53,9 @@
SharedMemoryArbiterImpl::default_page_layout_for_testing();
AlignedBufferTest::SetUp();
task_runner_.reset(new base::TestTaskRunner());
- arbiter_.reset(new SharedMemoryArbiterImpl(buf(), buf_size(), page_size(),
- &mock_producer_endpoint_,
- task_runner_.get()));
+ arbiter_.reset(new SharedMemoryArbiterImpl(
+ buf(), buf_size(), ShmemMode::kDefault, page_size(),
+ &mock_producer_endpoint_, task_runner_.get()));
}
bool IsArbiterFullyBound() { return arbiter_->fully_bound_; }
@@ -207,6 +211,63 @@
arbiter_->FlushPendingCommitDataRequests();
}
+TEST_P(SharedMemoryArbiterImplTest, UseShmemEmulation) {
+ arbiter_.reset(new SharedMemoryArbiterImpl(
+ buf(), buf_size(), ShmemMode::kShmemEmulation, page_size(),
+ &mock_producer_endpoint_, task_runner_.get()));
+
+ SharedMemoryArbiterImpl::set_default_layout_for_testing(
+ SharedMemoryABI::PageLayout::kPageDiv1);
+
+ size_t page_idx;
+ size_t chunk_idx;
+ auto* abi = arbiter_->shmem_abi_for_testing();
+
+ // Test returning a completed chunk.
+ SharedMemoryABI::Chunk chunk =
+ arbiter_->GetNewChunk({}, BufferExhaustedPolicy::kDefault);
+ std::tie(page_idx, chunk_idx) = abi->GetPageAndChunkIndex(chunk);
+ ASSERT_TRUE(chunk.is_valid());
+ EXPECT_CALL(mock_producer_endpoint_, CommitData(_, _)).Times(1);
+ PatchList ignored;
+ arbiter_->ReturnCompletedChunk(std::move(chunk), 0, &ignored);
+ task_runner_->RunUntilIdle();
+ ASSERT_TRUE(Mock::VerifyAndClearExpectations(&mock_producer_endpoint_));
+ // When running in the emulation mode, the chunk is freed when the
+ // CommitDataRequest is flushed.
+ ASSERT_EQ(
+ SharedMemoryABI::kChunkFree,
+ arbiter_->shmem_abi_for_testing()->GetChunkState(page_idx, chunk_idx));
+
+ // Direct patching is supported in the emulation mode.
+ arbiter_->SetDirectSMBPatchingSupportedByService();
+ ASSERT_TRUE(arbiter_->EnableDirectSMBPatching());
+
+ chunk = arbiter_->GetNewChunk({}, BufferExhaustedPolicy::kDefault);
+ std::tie(page_idx, chunk_idx) = abi->GetPageAndChunkIndex(chunk);
+ ASSERT_TRUE(chunk.is_valid());
+ EXPECT_CALL(mock_producer_endpoint_, CommitData(_, _))
+ .WillOnce(Invoke([&](const CommitDataRequest& req,
+ MockProducerEndpoint::CommitDataCallback) {
+ ASSERT_EQ(1, req.chunks_to_move_size());
+
+ ASSERT_EQ(page_idx, req.chunks_to_move()[0].page());
+ ASSERT_EQ(chunk_idx, req.chunks_to_move()[0].chunk());
+ ASSERT_EQ(1u, req.chunks_to_move()[0].target_buffer());
+
+ // The request should contain chunk data.
+ ASSERT_TRUE(req.chunks_to_move()[0].has_data());
+ }));
+ chunk.SetFlag(SharedMemoryABI::ChunkHeader::kChunkNeedsPatching);
+ arbiter_->ReturnCompletedChunk(std::move(chunk), 1, &ignored);
+ task_runner_->RunUntilIdle();
+ ASSERT_TRUE(Mock::VerifyAndClearExpectations(&mock_producer_endpoint_));
+ // A chunk is freed after being flushed.
+ ASSERT_EQ(
+ SharedMemoryABI::kChunkFree,
+ arbiter_->shmem_abi_for_testing()->GetChunkState(page_idx, chunk_idx));
+}
+
// Check that we can actually create up to kMaxWriterID TraceWriter(s).
TEST_P(SharedMemoryArbiterImplTest, WriterIDsAllocation) {
auto checkpoint = task_runner_->CreateCheckpoint("last_unregistered");
@@ -306,8 +367,8 @@
auto checkpoint_flush = task_runner_->CreateCheckpoint("flush_completed");
// Create an unbound arbiter and bind immediately.
- arbiter_.reset(new SharedMemoryArbiterImpl(buf(), buf_size(), page_size(),
- nullptr, nullptr));
+ arbiter_.reset(new SharedMemoryArbiterImpl(
+ buf(), buf_size(), ShmemMode::kDefault, page_size(), nullptr, nullptr));
arbiter_->BindToProducerEndpoint(&mock_producer_endpoint_,
task_runner_.get());
EXPECT_TRUE(IsArbiterFullyBound());
@@ -338,8 +399,9 @@
public:
void SetupArbiter(InitialBindingState initial_state) {
if (initial_state == InitialBindingState::kUnbound) {
- arbiter_.reset(new SharedMemoryArbiterImpl(buf(), buf_size(), page_size(),
- nullptr, nullptr));
+ arbiter_.reset(
+ new SharedMemoryArbiterImpl(buf(), buf_size(), ShmemMode::kDefault,
+ page_size(), nullptr, nullptr));
EXPECT_FALSE(IsArbiterFullyBound());
} else {
// A bound arbiter is already set up by the base class.
diff --git a/src/tracing/core/trace_writer_impl_unittest.cc b/src/tracing/core/trace_writer_impl_unittest.cc
index cd0032c..a5b8606 100644
--- a/src/tracing/core/trace_writer_impl_unittest.cc
+++ b/src/tracing/core/trace_writer_impl_unittest.cc
@@ -42,6 +42,7 @@
namespace {
using ChunkHeader = SharedMemoryABI::ChunkHeader;
+using ShmemMode = SharedMemoryABI::ShmemMode;
using ::protozero::ScatteredStreamWriter;
using ::testing::AllOf;
using ::testing::ElementsAre;
@@ -70,9 +71,9 @@
SharedMemoryABI::PageLayout::kPageDiv4);
AlignedBufferTest::SetUp();
task_runner_.reset(new base::TestTaskRunner());
- arbiter_.reset(new SharedMemoryArbiterImpl(buf(), buf_size(), page_size(),
- &mock_producer_endpoint_,
- task_runner_.get()));
+ arbiter_.reset(new SharedMemoryArbiterImpl(
+ buf(), buf_size(), ShmemMode::kDefault, page_size(),
+ &mock_producer_endpoint_, task_runner_.get()));
ON_CALL(mock_producer_endpoint_, CommitData)
.WillByDefault([&](const CommitDataRequest& req,
MockProducerEndpoint::CommitDataCallback cb) {
diff --git a/src/tracing/core/tracing_service_impl.cc b/src/tracing/core/tracing_service_impl.cc
index 5530bc6..08d0d3d 100644
--- a/src/tracing/core/tracing_service_impl.cc
+++ b/src/tracing/core/tracing_service_impl.cc
@@ -21,6 +21,7 @@
#include <string.h>
#include <cinttypes>
+#include <limits>
#include <optional>
#include <regex>
#include <unordered_set>
@@ -4156,8 +4157,24 @@
if (page_idx >= shmem_abi_.num_pages())
continue; // A buggy or malicious producer.
- SharedMemoryABI::Chunk chunk =
- shmem_abi_.TryAcquireChunkForReading(page_idx, entry.chunk());
+ SharedMemoryABI::Chunk chunk;
+ bool commit_data_over_ipc = entry.has_data();
+ if (PERFETTO_UNLIKELY(commit_data_over_ipc)) {
+ // Chunk data is passed over the wire. Create a chunk using the serialized
+ // protobuf message.
+ const std::string& data = entry.data();
+ if (data.size() > SharedMemoryABI::Chunk::kMaxSize) {
+ PERFETTO_DFATAL("IPC data commit too large: %zu", data.size());
+ continue; // A malicious or buggy producer
+ }
+ // |data| is not altered, but we need to const_cast becasue Chunk data
+ // members are non-const.
+ chunk = SharedMemoryABI::MakeChunkFromSerializedData(
+ reinterpret_cast<uint8_t*>(const_cast<char*>(data.data())),
+ static_cast<uint16_t>(entry.data().size()),
+ static_cast<uint8_t>(entry.chunk()));
+ } else
+ chunk = shmem_abi_.TryAcquireChunkForReading(page_idx, entry.chunk());
if (!chunk.is_valid()) {
PERFETTO_DLOG("Asked to move chunk %d:%d, but it's not complete",
entry.page(), entry.chunk());
@@ -4182,8 +4199,10 @@
chunk_flags,
/*chunk_complete=*/true, chunk.payload_begin(), chunk.payload_size());
- // This one has release-store semantics.
- shmem_abi_.ReleaseChunkAsFree(std::move(chunk));
+ if (!commit_data_over_ipc) {
+ // This one has release-store semantics.
+ shmem_abi_.ReleaseChunkAsFree(std::move(chunk));
+ }
} // for(chunks_to_move)
service_->ApplyChunkPatches(id_, req_untrusted.chunks_to_patch());
@@ -4212,10 +4231,12 @@
shmem_abi_.Initialize(reinterpret_cast<uint8_t*>(shared_memory_->start()),
shared_memory_->size(),
- shared_buffer_page_size_kb() * 1024);
+ shared_buffer_page_size_kb() * 1024,
+ SharedMemoryABI::ShmemMode::kDefault);
if (in_process_) {
inproc_shmem_arbiter_.reset(new SharedMemoryArbiterImpl(
shared_memory_->start(), shared_memory_->size(),
+ SharedMemoryABI::ShmemMode::kDefault,
shared_buffer_page_size_kb_ * 1024, this, task_runner_));
inproc_shmem_arbiter_->SetDirectSMBPatchingSupportedByService();
}
diff --git a/src/tracing/internal/in_process_tracing_backend.cc b/src/tracing/internal/in_process_tracing_backend.cc
index ee1283d..57f5f1e 100644
--- a/src/tracing/internal/in_process_tracing_backend.cc
+++ b/src/tracing/internal/in_process_tracing_backend.cc
@@ -22,6 +22,8 @@
#include "perfetto/ext/tracing/core/shared_memory.h"
#include "perfetto/ext/tracing/core/tracing_service.h"
+#include "src/tracing/core/in_process_shared_memory.h"
+
// TODO(primiano): When the in-process backend is used, we should never end up
// in a situation where the thread where the TracingService and Producer live
// writes a packet and hence can get into the GetNewChunk() stall.
@@ -34,46 +36,6 @@
namespace perfetto {
namespace internal {
-namespace {
-
-class InProcessShm : public SharedMemory {
- public:
- explicit InProcessShm(size_t size);
- ~InProcessShm() override;
- void* start() const override;
- size_t size() const override;
-
- private:
- base::PagedMemory mem_;
-};
-
-class InProcessShmFactory : public SharedMemory::Factory {
- public:
- ~InProcessShmFactory() override;
- std::unique_ptr<SharedMemory> CreateSharedMemory(size_t) override;
-};
-
-InProcessShm::~InProcessShm() = default;
-
-InProcessShm::InProcessShm(size_t size)
- : mem_(base::PagedMemory::Allocate(size)) {}
-
-void* InProcessShm::start() const {
- return mem_.Get();
-}
-
-size_t InProcessShm::size() const {
- return mem_.size();
-}
-
-InProcessShmFactory::~InProcessShmFactory() = default;
-std::unique_ptr<SharedMemory> InProcessShmFactory::CreateSharedMemory(
- size_t size) {
- return std::unique_ptr<SharedMemory>(new InProcessShm(size));
-}
-
-} // namespace
-
// static
TracingBackend* InProcessTracingBackend::GetInstance() {
static auto* instance = new InProcessTracingBackend();
@@ -102,7 +64,8 @@
TracingService* InProcessTracingBackend::GetOrCreateService(
base::TaskRunner* task_runner) {
if (!service_) {
- std::unique_ptr<InProcessShmFactory> shm(new InProcessShmFactory());
+ std::unique_ptr<InProcessSharedMemory::Factory> shm(
+ new InProcessSharedMemory::Factory());
service_ = TracingService::CreateInstance(std::move(shm), task_runner);
service_->SetSMBScrapingEnabled(true);
}
diff --git a/src/tracing/internal/system_tracing_backend.cc b/src/tracing/internal/system_tracing_backend.cc
index 18c3e08..9488495 100644
--- a/src/tracing/internal/system_tracing_backend.cc
+++ b/src/tracing/internal/system_tracing_backend.cc
@@ -61,8 +61,8 @@
#else
shm = PosixSharedMemory::Create(shmem_size_hint);
#endif
- arbiter = SharedMemoryArbiter::CreateUnboundInstance(shm.get(),
- shmem_page_size_hint);
+ arbiter = SharedMemoryArbiter::CreateUnboundInstance(
+ shm.get(), shmem_page_size_hint, SharedMemoryABI::ShmemMode::kDefault);
}
auto endpoint = ProducerIPCClient::Connect(
diff --git a/src/tracing/ipc/producer/BUILD.gn b/src/tracing/ipc/producer/BUILD.gn
index a6b28d7..1a81ab6 100644
--- a/src/tracing/ipc/producer/BUILD.gn
+++ b/src/tracing/ipc/producer/BUILD.gn
@@ -34,6 +34,7 @@
"../../../../gn:default_deps",
"../../../base",
"../../../base:version",
+ "../../core",
]
if (perfetto_component_type == "static_library") {
deps += [ "../../../ipc:perfetto_ipc" ]
diff --git a/src/tracing/ipc/producer/producer_ipc_client_impl.cc b/src/tracing/ipc/producer/producer_ipc_client_impl.cc
index 95a304a..6caf224 100644
--- a/src/tracing/ipc/producer/producer_ipc_client_impl.cc
+++ b/src/tracing/ipc/producer/producer_ipc_client_impl.cc
@@ -22,15 +22,18 @@
#include "perfetto/base/logging.h"
#include "perfetto/base/task_runner.h"
+#include "perfetto/ext/base/unix_socket.h"
#include "perfetto/ext/base/version.h"
#include "perfetto/ext/ipc/client.h"
#include "perfetto/ext/tracing/core/commit_data_request.h"
#include "perfetto/ext/tracing/core/producer.h"
+#include "perfetto/ext/tracing/core/shared_memory_abi.h"
#include "perfetto/ext/tracing/core/shared_memory_arbiter.h"
#include "perfetto/ext/tracing/core/trace_writer.h"
#include "perfetto/tracing/core/data_source_config.h"
#include "perfetto/tracing/core/data_source_descriptor.h"
#include "perfetto/tracing/core/trace_config.h"
+#include "src/tracing/core/in_process_shared_memory.h"
#if PERFETTO_BUILDFLAG(PERFETTO_OS_WIN)
#include "src/tracing/ipc/shared_memory_windows.h"
@@ -156,7 +159,8 @@
OnConnectionInitialized(
resp.success(),
resp.success() ? resp->using_shmem_provided_by_producer() : false,
- resp.success() ? resp->direct_smb_patching_supported() : false);
+ resp.success() ? resp->direct_smb_patching_supported() : false,
+ resp.success() ? resp->use_shmem_emulation() : false);
});
protos::gen::InitializeConnectionRequest req;
req.set_producer_name(name_);
@@ -239,7 +243,8 @@
void ProducerIPCClientImpl::OnConnectionInitialized(
bool connection_succeeded,
bool using_shmem_provided_by_producer,
- bool direct_smb_patching_supported) {
+ bool direct_smb_patching_supported,
+ bool use_shmem_emulation) {
PERFETTO_DCHECK_THREAD(thread_checker_);
// If connection_succeeded == false, the OnDisconnect() call will follow next
// and there we'll notify the |producer_|. TODO: add a test for this.
@@ -247,6 +252,11 @@
return;
is_shmem_provided_by_producer_ = using_shmem_provided_by_producer;
direct_smb_patching_supported_ = direct_smb_patching_supported;
+ // The tracing service may reject using shared memory and tell the client to
+ // commit data over the socket. This can happen when the client connects to
+ // the service via a relay service:
+ // client <-Unix socket-> relay service <- vsock -> tracing service.
+ use_shmem_emulation_ = use_shmem_emulation;
producer_->OnConnect();
// Bail out if the service failed to adopt our producer-allocated SMB.
@@ -324,7 +334,19 @@
/*require_seals_if_supported=*/false);
}
#endif
+ if (use_shmem_emulation_) {
+ PERFETTO_CHECK(!ipc_shared_memory);
+ // Need to create an emulated shmem buffer when the transport deosn't
+ // support it.
+ // TODO(chinglinyu): Let the tracing service decide on the shmem size and
+ // propagate the size in InitializeConnectionResponse.
+ ipc_shared_memory = InProcessSharedMemory::Create(
+ /*size=*/InProcessSharedMemory::kDefaultSize);
+ }
if (ipc_shared_memory) {
+ auto shmem_mode = use_shmem_emulation_
+ ? SharedMemoryABI::ShmemMode::kShmemEmulation
+ : SharedMemoryABI::ShmemMode::kDefault;
// This is the nominal case used in most configurations, where the service
// provides the SMB.
PERFETTO_CHECK(!is_shmem_provided_by_producer_ && !shared_memory_);
@@ -332,8 +354,8 @@
shared_buffer_page_size_kb_ =
cmd.setup_tracing().shared_buffer_page_size_kb();
shared_memory_arbiter_ = SharedMemoryArbiter::CreateInstance(
- shared_memory_.get(), shared_buffer_page_size_kb_ * 1024, this,
- task_runner_);
+ shared_memory_.get(), shared_buffer_page_size_kb_ * 1024, shmem_mode,
+ this, task_runner_);
if (direct_smb_patching_supported_)
shared_memory_arbiter_->SetDirectSMBPatchingSupportedByService();
} else {
diff --git a/src/tracing/ipc/producer/producer_ipc_client_impl.h b/src/tracing/ipc/producer/producer_ipc_client_impl.h
index 06e73ae..664b698 100644
--- a/src/tracing/ipc/producer/producer_ipc_client_impl.h
+++ b/src/tracing/ipc/producer/producer_ipc_client_impl.h
@@ -100,7 +100,8 @@
// Invoked soon after having established the connection with the service.
void OnConnectionInitialized(bool connection_succeeded,
bool using_shmem_provided_by_producer,
- bool direct_smb_patching_supported);
+ bool direct_smb_patching_supported,
+ bool use_shmem_emulation);
// Invoked when the remote Service sends an IPC to tell us to do something
// (e.g. start/stop a data source).
@@ -131,6 +132,7 @@
TracingService::ProducerSMBScrapingMode const smb_scraping_mode_;
bool is_shmem_provided_by_producer_ = false;
bool direct_smb_patching_supported_ = false;
+ bool use_shmem_emulation_ = false;
std::vector<std::function<void()>> pending_sync_reqs_;
base::WeakPtrFactory<ProducerIPCClientImpl> weak_factory_{this};
PERFETTO_THREAD_CHECKER(thread_checker_)
diff --git a/src/tracing/ipc/service/producer_ipc_service.cc b/src/tracing/ipc/service/producer_ipc_service.cc
index 5f9a387..d13130a 100644
--- a/src/tracing/ipc/service/producer_ipc_service.cc
+++ b/src/tracing/ipc/service/producer_ipc_service.cc
@@ -129,7 +129,9 @@
return;
}
+ bool use_shmem_emulation = ipc::Service::use_shmem_emulation();
bool using_producer_shmem =
+ !use_shmem_emulation &&
producer->service_endpoint->IsShmemProvidedByProducer();
producers_.emplace(ipc_client_id, std::move(producer));
@@ -139,6 +141,7 @@
ipc::AsyncResult<protos::gen::InitializeConnectionResponse>::Create();
async_res->set_using_shmem_provided_by_producer(using_producer_shmem);
async_res->set_direct_smb_patching_supported(true);
+ async_res->set_use_shmem_emulation(use_shmem_emulation);
response.Resolve(std::move(async_res));
}
diff --git a/src/tracing/test/BUILD.gn b/src/tracing/test/BUILD.gn
index 51f2348..84a0e9f 100644
--- a/src/tracing/test/BUILD.gn
+++ b/src/tracing/test/BUILD.gn
@@ -41,7 +41,6 @@
"aligned_buffer_test.h",
"fake_packet.cc",
"fake_packet.h",
- "test_shared_memory.cc",
"test_shared_memory.h",
"traced_value_test_support.cc",
]
diff --git a/src/tracing/test/test_shared_memory.cc b/src/tracing/test/test_shared_memory.cc
deleted file mode 100644
index b0a845c..0000000
--- a/src/tracing/test/test_shared_memory.cc
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Copyright (C) 2017 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/tracing/test/test_shared_memory.h"
-
-#include <stdlib.h>
-#include <string.h>
-
-#include "perfetto/base/logging.h"
-
-namespace perfetto {
-
-TestSharedMemory::TestSharedMemory(size_t size) {
- mem_ = base::PagedMemory::Allocate(size);
- size_ = size;
-}
-
-TestSharedMemory::~TestSharedMemory() {}
-
-TestSharedMemory::Factory::~Factory() {}
-
-std::unique_ptr<SharedMemory> TestSharedMemory::Factory::CreateSharedMemory(
- size_t size) {
- return std::unique_ptr<SharedMemory>(new TestSharedMemory(size));
-}
-
-} // namespace perfetto
diff --git a/src/tracing/test/test_shared_memory.h b/src/tracing/test/test_shared_memory.h
index 9db6bcb..e932ad6 100644
--- a/src/tracing/test/test_shared_memory.h
+++ b/src/tracing/test/test_shared_memory.h
@@ -23,28 +23,13 @@
#include "perfetto/ext/base/paged_memory.h"
#include "perfetto/ext/tracing/core/shared_memory.h"
+#include "src/tracing/core/in_process_shared_memory.h"
namespace perfetto {
// A dummy implementation of shared memory for single process unittests
// (just a wrapper around malloc() that fits the SharedMemory API).
-class TestSharedMemory : public SharedMemory {
- public:
- class Factory : public SharedMemory::Factory {
- public:
- ~Factory() override;
- std::unique_ptr<SharedMemory> CreateSharedMemory(size_t size) override;
- };
-
- explicit TestSharedMemory(size_t size);
- ~TestSharedMemory() override;
-
- void* start() const override { return mem_.Get(); }
- size_t size() const override { return size_; }
-
- base::PagedMemory mem_;
- size_t size_;
-};
+using TestSharedMemory = InProcessSharedMemory;
} // namespace perfetto
diff --git a/test/test_helper.h b/test/test_helper.h
index 055182d..ba00422 100644
--- a/test/test_helper.h
+++ b/test/test_helper.h
@@ -246,7 +246,8 @@
PosixSharedMemory::Factory factory;
#endif
shm_ = factory.CreateSharedMemory(1024 * 1024);
- shm_arbiter_ = SharedMemoryArbiter::CreateUnboundInstance(shm_.get(), 4096);
+ shm_arbiter_ = SharedMemoryArbiter::CreateUnboundInstance(
+ shm_.get(), 4096, SharedMemoryABI::ShmemMode::kDefault);
}
void ProduceStartupEventBatch(const protos::gen::TestConfig& config,
diff --git a/test/traced_integrationtest.cc b/test/traced_integrationtest.cc
index 0181ba9..589e7cd 100644
--- a/test/traced_integrationtest.cc
+++ b/test/traced_integrationtest.cc
@@ -14,6 +14,8 @@
* limitations under the License.
*/
+#include <string>
+
#include "perfetto/base/build_config.h"
#include "perfetto/base/logging.h"
#include "perfetto/ext/base/file_utils.h"
@@ -21,6 +23,7 @@
#include "perfetto/ext/base/scoped_file.h"
#include "perfetto/ext/base/string_utils.h"
#include "perfetto/ext/base/temp_file.h"
+#include "perfetto/ext/base/unix_socket.h"
#include "perfetto/ext/base/utils.h"
#include "perfetto/ext/tracing/core/commit_data_request.h"
#include "perfetto/ext/tracing/core/trace_packet.h"
@@ -512,6 +515,63 @@
for (const auto& sock_name : producer_socket_names)
remove(sock_name.c_str());
}
+
+TEST(PerfettoTracedIntegrationTest, TestShmemEmulation) {
+ base::TestTaskRunner task_runner;
+ auto temp_dir = base::TempDir::Create();
+
+ std::string sock_name;
+ {
+ // Set up a server UnixSocket to find an unused TCP port.
+ base::UnixSocket::EventListener event_listener;
+ auto srv = base::UnixSocket::Listen("127.0.0.1:0", &event_listener,
+ &task_runner, base::SockFamily::kInet,
+ base::SockType::kStream);
+ ASSERT_TRUE(srv->is_listening());
+ sock_name = srv->GetSockAddr();
+ // Shut down |srv| here to free the port. It's unlikely that the port will
+ // be taken by another process so quickly before we reach the code below.
+ }
+
+ TestHelper helper(&task_runner, TestHelper::Mode::kStartDaemons,
+ sock_name.c_str());
+ ASSERT_EQ(helper.num_producers(), 1u);
+ helper.StartServiceIfRequired();
+ // Setup the 1st producer (default).
+ helper.ConnectFakeProducer();
+ helper.ConnectConsumer();
+ helper.WaitForConsumerConnect();
+
+ TraceConfig trace_config;
+ trace_config.add_buffers()->set_size_kb(1024);
+ trace_config.set_duration_ms(200);
+
+ static constexpr uint32_t kMsgSize = 1024;
+ static constexpr uint32_t kRandomSeed = 42;
+ // Enable the producer.
+ auto* ds_config = trace_config.add_data_sources()->mutable_config();
+ ds_config->set_name("android.perfetto.FakeProducer");
+ ds_config->set_target_buffer(0);
+ ds_config->mutable_for_testing()->set_seed(kRandomSeed);
+ ds_config->mutable_for_testing()->set_message_count(12);
+ ds_config->mutable_for_testing()->set_message_size(kMsgSize);
+ ds_config->mutable_for_testing()->set_send_batch_on_register(true);
+
+ helper.StartTracing(trace_config);
+ helper.WaitForTracingDisabled();
+
+ helper.ReadData();
+ helper.WaitForReadData();
+
+ const auto& packets = helper.trace();
+ ASSERT_EQ(packets.size(), 12u);
+
+ std::minstd_rand0 rnd_engine(kRandomSeed);
+ for (const auto& packet : packets) {
+ ASSERT_TRUE(packet.has_for_testing());
+ ASSERT_EQ(packet.for_testing().seq_value(), rnd_engine());
+ }
+}
#endif
} // namespace perfetto