Add public API for Perfetto consumer

An API that services in the Android tree can depend on.
It relies on libperfetto.so .

Bug: 117604753
Bug: 115601774
Change-Id: Ia8a03a5c8f8cbf3fa3aa893f169073646790d6cb
diff --git a/Android.bp b/Android.bp
index 0da046f..83d55df 100644
--- a/Android.bp
+++ b/Android.bp
@@ -83,6 +83,7 @@
     "src/traced/probes/ps/process_stats_data_source.cc",
     "src/traced/probes/sys_stats/sys_stats_data_source.cc",
     "src/traced/service/service.cc",
+    "src/tracing/api_impl/consumer_api.cc",
     "src/tracing/core/chrome_config.cc",
     "src/tracing/core/commit_data_request.cc",
     "src/tracing/core/data_source_config.cc",
diff --git a/BUILD.gn b/BUILD.gn
index 011e158..15f2bd9 100644
--- a/BUILD.gn
+++ b/BUILD.gn
@@ -62,6 +62,7 @@
     if (!build_with_android) {
       deps += [
         ":trace_processor",
+        "src/tracing:consumer_api_test",
         "tools/trace_to_text",
       ]
     }
@@ -173,6 +174,7 @@
       "gn:default_deps",
       "src/traced/probes",
       "src/traced/service",
+      "src/tracing:api",
     ]
   }
 
diff --git a/include/perfetto/public/consumer_api.h b/include/perfetto/public/consumer_api.h
new file mode 100644
index 0000000..0dd86c9
--- /dev/null
+++ b/include/perfetto/public/consumer_api.h
@@ -0,0 +1,162 @@
+/*
+ * 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.
+ */
+
+#ifndef INCLUDE_PERFETTO_PUBLIC_CONSUMER_API_H_
+#define INCLUDE_PERFETTO_PUBLIC_CONSUMER_API_H_
+
+#include <stddef.h>
+#include <stdint.h>
+
+// Public API for perfetto consumer, exposed to the rest of the Android tree.
+
+namespace perfetto {
+namespace consumer {
+
+// State diagram (without error states):
+// +-------------+
+// | kConnecting |----------------------------+
+// +-------------+                            | Create(deferred_start:true)
+//        |  Create(deferred_start:false)     V
+//        |                            +-------------+
+//        |                            | kConfigured |
+//        |                            +-------------+
+//        |                                   |
+//        V                                   | StartTracing()
+//   +----------+                             |
+//   | kTracing |<----------------------------+
+//   +----------+
+//        |
+//        | [after TraceConfig.duration_ms]
+//        V
+// +-------------+
+// | kTraceEnded |
+// +-------------+
+enum class State {
+  // The trace session failed (e.g. the trace config is malformed or mmap
+  // failed). Look at logcat -s perfetto to find out more.
+  // This state is final and unrecoverable. The sessions needs to be destroyed
+  // and recreated if this error happens.
+  kTraceFailed = -3,
+
+  // Failed to connect to the traced daemon.
+  // This state is final and unrecoverable. The sessions needs to be destroyed
+  // and recreated if this error happens.
+  kConnectionError = -2,
+
+  // Not really a state. This is only returned when an invalid handle is passed
+  // to PollState().
+  kSessionNotFound = -1,
+
+  // Idle state.
+  // This state is transitional an internal-only. Client will never see it.
+  kIdle = 0,
+
+  // Establishing the connection to the traced daemon.
+  // This state is transitional. It is set soon after the Create() call and
+  // transitions automatically to:
+  // - kConfigured, if |deferred_start| == true in the trace config.
+  // - kTracing, if |deferred_start| == false.
+  // - An error state, e.g. if cannot reach the traced daemon.
+  kConnecting = 1,
+
+  // Tracing configured (buffers allocated) but not started.
+  // This state is reached only when setting |deferred_start| == true and is
+  // held until the client calls StartTracing().
+  kConfigured = 2,
+
+  // Tracing is active.
+  // This state lasts for the whole duration of the trace session (i.e.
+  // |duration_ms| in the trace config), after which the session transitions
+  // either to the kTraceEnded state (if successful) or an error state.
+  kTracing = 3,
+
+  // Tracing ended succesfully. The trace buffer can now be retrieved through
+  // the ReadTrace() call.
+  // This state is final.
+  kTraceEnded = 4,
+};
+
+using Handle = int64_t;
+constexpr Handle kInvalidHandle = -1;
+
+// Signature for callback function provided by the embedder to get notified
+// about state changes.
+using OnStateChangedCb = void (*)(Handle, State, void* /*callback_arg*/);
+
+// None of the calls below are blocking, unless otherwise specified.
+
+// Enables tracing with the given TraceConfig. If the trace config has the
+// |deferred_start| flag set (see trace_config.proto) tracing is initialized
+// but not started. An explicit call to StartTracing() must be issued in order
+// to start the capture.
+// Args:
+//   [trace_config_proto, trace_config_len] point to a binary-encoded proto
+//     containing the trace config. See //external/perfetto/docs/trace-config.md
+//     for more details.
+//   callback: a user-defined callback that will be invoked upon state changes.
+//     The callback will be invoked on an internal thread and must not block.
+//   callback_arg: an optional user-define argument that will be passed back to
+//     all callback invocations.
+// Return value:
+//    Returns a handle that can be used to retrieve the trace or kInvalidHandle
+//    in case of failure (e.g., the trace config is malformed).
+Handle Create(const void* config_proto,
+              size_t config_len,
+              OnStateChangedCb callback,
+              void* callback_arg);
+
+// Starts recording the trace. Can be used only when setting the
+// |deferred_start| flag in the trace config passed to Create().
+// If the session is in the kConfigured state it transitions it to the kTracing
+// state, starting the trace. In any other state, instead, it does nothing other
+// than logging an error message. Hence, this method can be called only once.
+// The estimated end-to-end (this call to ftrace enabling) latency is 2-3 ms
+// on a Pixel 2.
+// TODO(primiano): relax this and allow to recycle handles without
+// re-configuring the trace session.
+void StartTracing(Handle);
+
+struct TraceBuffer {
+  char* begin;
+  size_t size;
+};
+
+// Retrieves the whole trace buffer. It avoids extra copies by directly mmaping
+// the tmp fd passed to the traced daemon.
+// Return value:
+//   If the trace is ended (state == kTraceEnded) returns a buffer containing
+//   the whole trace. This buffer can be parsed directly with libprotobuf.
+//   The buffer lifetime is tied to the tracing session and is valid until the
+//   Destroy() call.
+//   If called before the session reaches the kTraceEnded state, a null buffer
+//   is returned.
+TraceBuffer ReadTrace(Handle);
+
+// Destroys all the resources associated to the tracing session (connection to
+// traced and trace buffer). The handle must not be used after this point.
+// It's safe to call this regardless of the handle's current state and validity.
+void Destroy(Handle);
+
+// Returns the state of the tracing session (for debugging).
+// Return value:
+//   Returns the state of the session, if the handle is valid, otherwise returns
+//   kSessionNotFound.
+State PollState(Handle);
+
+}  // namespace consumer
+}  // namespace perfetto
+
+#endif  // INCLUDE_PERFETTO_PUBLIC_CONSUMER_API_H_
diff --git a/src/tracing/BUILD.gn b/src/tracing/BUILD.gn
index 4455bfb..ba44411 100644
--- a/src/tracing/BUILD.gn
+++ b/src/tracing/BUILD.gn
@@ -65,6 +65,32 @@
   ]
 }
 
+source_set("api") {
+  deps = [
+    ":ipc",
+    ":tracing",
+    "../../gn:default_deps",
+    "../../protos/perfetto/config:lite",
+    "../base",
+  ]
+  sources = [
+    "api_impl/consumer_api.cc",
+  ]
+}
+
+executable("consumer_api_test") {
+  deps = [
+    ":api",
+    "../../gn:default_deps",
+    "../../protos/perfetto/config:lite",
+    "../../protos/perfetto/trace:lite",
+    "../base",
+  ]
+  sources = [
+    "api_impl/consumer_api_test.cc",
+  ]
+}
+
 source_set("unittests") {
   testonly = true
   deps = [
diff --git a/src/tracing/api_impl/consumer_api.cc b/src/tracing/api_impl/consumer_api.cc
new file mode 100644
index 0000000..89c0608
--- /dev/null
+++ b/src/tracing/api_impl/consumer_api.cc
@@ -0,0 +1,403 @@
+/*
+ * 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.
+ */
+
+#include "perfetto/public/consumer_api.h"
+
+#include <fcntl.h>
+#include <inttypes.h>
+#include <stdlib.h>
+#include <sys/mman.h>
+#include <sys/select.h>
+#include <sys/stat.h>
+#include <sys/time.h>
+#include <sys/uio.h>
+#include <unistd.h>
+
+#include <atomic>
+#include <memory>
+#include <mutex>
+#include <thread>
+
+#include "perfetto/base/build_config.h"
+#include "perfetto/base/event.h"
+#include "perfetto/base/scoped_file.h"
+#include "perfetto/base/temp_file.h"
+#include "perfetto/base/thread_checker.h"
+#include "perfetto/base/unix_task_runner.h"
+#include "perfetto/base/utils.h"
+#include "perfetto/tracing/core/consumer.h"
+#include "perfetto/tracing/core/trace_config.h"
+#include "perfetto/tracing/core/trace_packet.h"
+#include "perfetto/tracing/ipc/consumer_ipc_client.h"
+#include "src/tracing/ipc/default_socket.h"
+
+#if PERFETTO_BUILDFLAG(PERFETTO_OS_ANDROID)
+#include <linux/memfd.h>
+#include <sys/syscall.h>
+#endif
+
+#include "perfetto/config/trace_config.pb.h"
+
+#define PERFETTO_EXPORTED_API __attribute__((visibility("default")))
+
+namespace perfetto {
+namespace consumer {
+
+namespace {
+
+class TracingSession : public Consumer {
+ public:
+  TracingSession(base::TaskRunner*,
+                 Handle,
+                 OnStateChangedCb,
+                 void* callback_arg,
+                 const perfetto::protos::TraceConfig&);
+  ~TracingSession() override;
+
+  // Note: if making this class moveable, the move-ctor/dtor must be updated
+  // to clear up mapped_buf_ on dtor.
+
+  // These methods are called on a thread != |task_runner_|.
+  State state() const { return state_; }
+  std::pair<char*, size_t> mapped_buf() const {
+    // The comparison operator will do an acquire-load on the atomic |state_|.
+    if (state_ == State::kTraceEnded)
+      return std::make_pair(mapped_buf_, mapped_buf_size_);
+    return std::make_pair(nullptr, 0);
+  }
+
+  // All the methods below are called only on the |task_runner_| thread.
+
+  bool Initialize();
+  void StartTracing();
+
+  // perfetto::Consumer implementation.
+  void OnConnect() override;
+  void OnDisconnect() override;
+  void OnTracingDisabled() override;
+  void OnTraceData(std::vector<TracePacket>, bool has_more) override;
+
+ private:
+  TracingSession(const TracingSession&) = delete;
+  TracingSession& operator=(const TracingSession&) = delete;
+
+  void DestroyConnection();
+  void NotifyCallback();
+
+  base::TaskRunner* const task_runner_;
+  Handle const handle_;
+  OnStateChangedCb const callback_ = nullptr;
+  void* const callback_arg_ = nullptr;
+  TraceConfig trace_config_;
+  base::ScopedFile buf_fd_;
+  std::unique_ptr<TracingService::ConsumerEndpoint> consumer_endpoint_;
+
+  // |mapped_buf_| and |mapped_buf_size_| are seq-consistent with |state_|.
+  std::atomic<State> state_{State::kIdle};
+  char* mapped_buf_ = nullptr;
+  size_t mapped_buf_size_ = 0;
+
+  PERFETTO_THREAD_CHECKER(thread_checker_)
+};
+
+TracingSession::TracingSession(
+    base::TaskRunner* task_runner,
+    Handle handle,
+    OnStateChangedCb callback,
+    void* callback_arg,
+    const perfetto::protos::TraceConfig& trace_config_proto)
+    : task_runner_(task_runner),
+      handle_(handle),
+      callback_(callback),
+      callback_arg_(callback_arg) {
+  PERFETTO_DETACH_FROM_THREAD(thread_checker_);
+  trace_config_.FromProto(trace_config_proto);
+  trace_config_.set_write_into_file(true);
+
+  // TODO(primiano): this really doesn't matter because the trace will be
+  // flushed into the file when stopping. We need a way to be able to say
+  // "disable periodic flushing and flush only when stopping".
+  trace_config_.set_file_write_period_ms(60000);
+}
+
+TracingSession::~TracingSession() {
+  PERFETTO_DCHECK_THREAD(thread_checker_);
+  if (mapped_buf_)
+    PERFETTO_CHECK(munmap(mapped_buf_, mapped_buf_size_) == 0);
+}
+
+bool TracingSession::Initialize() {
+  PERFETTO_DCHECK_THREAD(thread_checker_);
+
+  if (state_ != State::kIdle)
+    return false;
+
+#if PERFETTO_BUILDFLAG(PERFETTO_OS_ANDROID)
+  char memfd_name[64];
+  snprintf(memfd_name, sizeof(memfd_name), "perfetto_trace_%" PRId64, handle_);
+  buf_fd_.reset(
+      static_cast<int>(syscall(__NR_memfd_create, memfd_name, MFD_CLOEXEC)));
+#else
+  // Fallback for testing on Linux/mac.
+  buf_fd_ = base::TempFile::CreateUnlinked().ReleaseFD();
+#endif
+
+  if (!buf_fd_) {
+    PERFETTO_PLOG("Failed to allocate temporary tracing buffer");
+    return false;
+  }
+
+  state_ = State::kConnecting;
+  consumer_endpoint_ =
+      ConsumerIPCClient::Connect(GetConsumerSocket(), this, task_runner_);
+
+  return true;
+}
+
+// Called after EnabledTracing, soon after the IPC connection is established.
+void TracingSession::OnConnect() {
+  PERFETTO_DCHECK_THREAD(thread_checker_);
+
+  PERFETTO_DLOG("OnConnect");
+  PERFETTO_DCHECK(state_ == State::kConnecting);
+  consumer_endpoint_->EnableTracing(trace_config_,
+                                    base::ScopedFile(dup(*buf_fd_)));
+  if (trace_config_.deferred_start())
+    state_ = State::kConfigured;
+  else
+    state_ = State::kTracing;
+  NotifyCallback();
+}
+
+void TracingSession::StartTracing() {
+  PERFETTO_DCHECK_THREAD(thread_checker_);
+
+  auto state = state_.load();
+  if (state != State::kConfigured) {
+    PERFETTO_ELOG("StartTracing(): invalid state (%d)", state);
+    return;
+  }
+  state_ = State::kTracing;
+  consumer_endpoint_->StartTracing();
+}
+
+void TracingSession::OnTracingDisabled() {
+  PERFETTO_DCHECK_THREAD(thread_checker_);
+  PERFETTO_DLOG("OnTracingDisabled");
+
+  struct stat stat_buf {};
+  int res = fstat(buf_fd_.get(), &stat_buf);
+  mapped_buf_size_ = res == 0 ? static_cast<size_t>(stat_buf.st_size) : 0;
+  mapped_buf_ =
+      static_cast<char*>(mmap(nullptr, mapped_buf_size_, PROT_READ | PROT_WRITE,
+                              MAP_SHARED, buf_fd_.get(), 0));
+  DestroyConnection();
+  if (mapped_buf_size_ == 0 || mapped_buf_ == MAP_FAILED) {
+    mapped_buf_ = nullptr;
+    mapped_buf_size_ = 0;
+    state_ = State::kTraceFailed;
+    PERFETTO_ELOG("Tracing session failed");
+  } else {
+    state_ = State::kTraceEnded;
+  }
+  NotifyCallback();
+}
+
+void TracingSession::OnDisconnect() {
+  PERFETTO_DCHECK_THREAD(thread_checker_);
+  PERFETTO_DLOG("OnDisconnect");
+  DestroyConnection();
+  state_ = State::kConnectionError;
+  NotifyCallback();
+}
+
+void TracingSession::DestroyConnection() {
+  // Destroys the connection in a separate task. This is to avoid destroying
+  // the IPC connection directly from within the IPC callback.
+  TracingService::ConsumerEndpoint* endpoint = consumer_endpoint_.release();
+  task_runner_->PostTask([endpoint] { delete endpoint; });
+}
+
+void TracingSession::OnTraceData(std::vector<TracePacket>, bool) {
+  // This should be never called because we are using |write_into_file| and
+  // asking the traced service to directly write into the |buf_fd_|.
+  PERFETTO_DCHECK(false);
+}
+
+void TracingSession::NotifyCallback() {
+  if (!callback_)
+    return;
+  auto state = state_.load();
+  auto callback = callback_;
+  auto handle = handle_;
+  auto callback_arg = callback_arg_;
+  task_runner_->PostTask([callback, callback_arg, handle, state] {
+    callback(handle, state, callback_arg);
+  });
+}
+
+class TracingController {
+ public:
+  static TracingController* GetInstance();
+  TracingController();
+
+  // These methods are called from a thread != |task_runner_|.
+  Handle Create(const void*, size_t, OnStateChangedCb, void* callback_arg);
+  void StartTracing(Handle);
+  State PollState(Handle);
+  TraceBuffer ReadTrace(Handle);
+  void Destroy(Handle);
+
+ private:
+  void ThreadMain();  // Called on |task_runner_| thread.
+
+  std::mutex mutex_;
+  std::thread thread_;
+  std::unique_ptr<base::UnixTaskRunner> task_runner_;
+  std::condition_variable task_runner_initialized_;
+  Handle last_handle_ = 0;
+  std::map<Handle, std::unique_ptr<TracingSession>> sessions_;
+};
+
+TracingController* TracingController::GetInstance() {
+  static TracingController* instance = new TracingController();
+  return instance;
+}
+
+TracingController::TracingController()
+    : thread_(&TracingController::ThreadMain, this) {
+  std::unique_lock<std::mutex> lock(mutex_);
+  task_runner_initialized_.wait(lock, [this] { return !!task_runner_; });
+}
+
+void TracingController::ThreadMain() {
+  {
+    std::unique_lock<std::mutex> lock(mutex_);
+    task_runner_.reset(new base::UnixTaskRunner());
+  }
+  task_runner_initialized_.notify_one();
+  task_runner_->Run();
+}
+
+Handle TracingController::Create(const void* config_proto_buf,
+                                 size_t config_len,
+                                 OnStateChangedCb callback,
+                                 void* callback_arg) {
+  perfetto::protos::TraceConfig config_proto;
+  bool parsed = config_proto.ParseFromArray(config_proto_buf,
+                                            static_cast<int>(config_len));
+  if (!parsed) {
+    PERFETTO_ELOG("Failed to decode TraceConfig proto");
+    return kInvalidHandle;
+  }
+
+  if (!config_proto.duration_ms()) {
+    PERFETTO_ELOG("The trace config must specify a duration");
+    return kInvalidHandle;
+  }
+
+  std::unique_lock<std::mutex> lock(mutex_);
+  Handle handle = ++last_handle_;
+  auto* session = new TracingSession(task_runner_.get(), handle, callback,
+                                     callback_arg, config_proto);
+  sessions_.emplace(handle, std::unique_ptr<TracingSession>(session));
+
+  // Enable the TracingSession on its own thread.
+  task_runner_->PostTask([session] { session->Initialize(); });
+
+  return handle;
+}
+
+void TracingController::StartTracing(Handle handle) {
+  std::unique_lock<std::mutex> lock(mutex_);
+  auto it = sessions_.find(handle);
+  if (it == sessions_.end()) {
+    PERFETTO_ELOG("StartTracing(): Invalid tracing session handle");
+    return;
+  };
+  TracingSession* session = it->second.get();
+  task_runner_->PostTask([session] { session->StartTracing(); });
+}
+
+State TracingController::PollState(Handle handle) {
+  std::unique_lock<std::mutex> lock(mutex_);
+  auto it = sessions_.find(handle);
+  if (it == sessions_.end())
+    return State::kSessionNotFound;
+  return it->second->state();
+}
+
+TraceBuffer TracingController::ReadTrace(Handle handle) {
+  TraceBuffer buf{};
+
+  std::unique_lock<std::mutex> lock(mutex_);
+  auto it = sessions_.find(handle);
+  if (it == sessions_.end()) {
+    PERFETTO_DLOG("Handle invalid");
+    return buf;
+  }
+
+  TracingSession* session = it->second.get();
+  auto state = session->state();
+  if (state == State::kTraceEnded) {
+    std::tie(buf.begin, buf.size) = session->mapped_buf();
+    return buf;
+  }
+
+  PERFETTO_DLOG("ReadTrace(): called in an unexpected state (%d)", state);
+  return buf;
+}
+
+void TracingController::Destroy(Handle handle) {
+  // Post an empty task on the task runner to delete the session on its own
+  // thread.
+  std::unique_lock<std::mutex> lock(mutex_);
+  auto it = sessions_.find(handle);
+  if (it == sessions_.end())
+    return;
+  TracingSession* session = it->second.release();
+  sessions_.erase(it);
+  task_runner_->PostTask([session] { delete session; });
+}
+
+}  // namespace
+
+PERFETTO_EXPORTED_API Handle Create(const void* config_proto,
+                                    size_t config_len,
+                                    OnStateChangedCb callback,
+                                    void* callback_arg) {
+  return TracingController::GetInstance()->Create(config_proto, config_len,
+                                                  callback, callback_arg);
+}
+
+PERFETTO_EXPORTED_API
+void StartTracing(Handle handle) {
+  return TracingController::GetInstance()->StartTracing(handle);
+}
+
+PERFETTO_EXPORTED_API State PollState(Handle handle) {
+  return TracingController::GetInstance()->PollState(handle);
+}
+
+PERFETTO_EXPORTED_API TraceBuffer ReadTrace(Handle handle) {
+  return TracingController::GetInstance()->ReadTrace(handle);
+}
+
+PERFETTO_EXPORTED_API void Destroy(Handle handle) {
+  TracingController::GetInstance()->Destroy(handle);
+}
+}  // namespace consumer
+}  // namespace perfetto
diff --git a/src/tracing/api_impl/consumer_api_test.cc b/src/tracing/api_impl/consumer_api_test.cc
new file mode 100644
index 0000000..12183e1
--- /dev/null
+++ b/src/tracing/api_impl/consumer_api_test.cc
@@ -0,0 +1,189 @@
+/*
+ * 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.
+ */
+
+#include <inttypes.h>
+
+#include <array>
+#include <atomic>
+
+#include "perfetto/base/logging.h"
+#include "perfetto/public/consumer_api.h"
+
+#include "perfetto/config/trace_config.pb.h"
+#include "perfetto/trace/trace.pb.h"
+
+using namespace perfetto::consumer;
+
+namespace {
+
+int g_pointer = 0;
+
+std::string GetConfig(uint32_t duration_ms) {
+  perfetto::protos::TraceConfig trace_config;
+  trace_config.set_duration_ms(duration_ms);
+  trace_config.add_buffers()->set_size_kb(4096);
+  trace_config.set_deferred_start(true);
+  auto* ds_config = trace_config.add_data_sources()->mutable_config();
+  ds_config->set_name("linux.ftrace");
+  ds_config->mutable_ftrace_config()->add_ftrace_events("sched_switch");
+  ds_config->mutable_ftrace_config()->add_ftrace_events(
+      "mm_filemap_add_to_page_cache");
+  ds_config->mutable_ftrace_config()->add_ftrace_events(
+      "mm_filemap_delete_from_page_cache");
+  ds_config->set_target_buffer(0);
+  return trace_config.SerializeAsString();
+}
+
+void DumpTrace(TraceBuffer buf) {
+  perfetto::protos::Trace trace;
+  bool parsed = trace.ParseFromArray(buf.begin, static_cast<int>(buf.size));
+  if (!parsed) {
+    PERFETTO_ELOG("Failed to parse the trace");
+    return;
+  }
+
+  PERFETTO_LOG("Parsing %d trace packets", trace.packet_size());
+  int num_filemap_events = 0;
+  for (const auto& packet : trace.packet()) {
+    if (packet.has_ftrace_events()) {
+      const auto& bundle = packet.ftrace_events();
+      for (const auto& ftrace : bundle.event()) {
+        if (ftrace.has_mm_filemap_add_to_page_cache()) {
+          num_filemap_events++;
+          // const auto& evt = ftrace.mm_filemap_add_to_page_cache();
+          // PERFETTO_LOG(
+          //     "mm_filemap_add_to_page_cache pfn=%llu, dev=%llu, ino=%llu",
+          //     evt.pfn(), evt.s_dev(), evt.i_ino());
+        }
+        if (ftrace.has_mm_filemap_delete_from_page_cache()) {
+          num_filemap_events++;
+          // const auto& evt = ftrace.mm_filemap_delete_from_page_cache();
+          // PERFETTO_LOG(
+          //     "mm_filemap_delete_from_page_cache pfn=%llu, dev=%llu,
+          //     ino=%llu", evt.pfn(), evt.s_dev(), evt.i_ino());
+        }
+      }
+    }
+  }
+  PERFETTO_LOG("Got %d mm_filemap events", num_filemap_events);
+}
+
+void OnStateChanged(Handle handle, State state, void* ptr) {
+  PERFETTO_LOG("Callback: handle=%" PRId64 " state=%d", handle, state);
+  PERFETTO_CHECK(ptr == &g_pointer);
+}
+
+void TestSingle() {
+  std::string cfg = GetConfig(1000);
+  auto handle = Create(cfg.data(), cfg.size(), &OnStateChanged, &g_pointer);
+  PERFETTO_ILOG("Starting, handle=%" PRId64 " state=%d", handle,
+                PollState(handle));
+  usleep(100000);
+  StartTracing(handle);
+  // Wait for either completion or error.
+  while (static_cast<int>(PollState(handle)) > 0 &&
+         PollState(handle) != State::kTraceEnded) {
+    usleep(10000);
+  }
+
+  if (PollState(handle) == State::kTraceEnded) {
+    auto buf = ReadTrace(handle);
+    DumpTrace(buf);
+  } else {
+    PERFETTO_ELOG("Trace failed");
+  }
+
+  PERFETTO_ILOG("Destroying");
+  Destroy(handle);
+}
+
+void TestMany() {
+  std::string cfg = GetConfig(8000);
+
+  std::array<Handle, 5> handles{};
+  for (size_t i = 0; i < handles.size(); i++) {
+    auto handle = Create(cfg.data(), cfg.size(), &OnStateChanged, &g_pointer);
+    handles[i] = handle;
+    PERFETTO_ILOG("Creating handle=%" PRId64 " state=%d", handle,
+                  PollState(handle));
+  }
+
+  // Wait that all sessions are connected.
+  for (bool all_connected = false; !all_connected;) {
+    all_connected = true;
+    for (size_t i = 0; i < handles.size(); i++) {
+      if (PollState(handles[i]) != State::kConfigured) {
+        all_connected = false;
+      }
+    }
+    usleep(10000);
+  }
+
+  // Start only 3 out of 5 sessions, scattering them with 1 second delay.
+  for (size_t i = 0; i < handles.size(); i++) {
+    if (i % 2 == 0) {
+      StartTracing(handles[i]);
+      sleep(1);
+    }
+  }
+
+  // Wait until all sessions are complete.
+  for (int num_complete = 0; num_complete != 3;) {
+    num_complete = 0;
+    for (size_t i = 0; i < handles.size(); i++) {
+      if (PollState(handles[i]) == State::kTraceEnded) {
+        num_complete++;
+      }
+    }
+    usleep(10000);
+  }
+
+  // Read the trace buffers.
+  for (size_t i = 0; i < handles.size(); i++) {
+    auto buf = ReadTrace(handles[i]);
+    PERFETTO_ILOG("ReadTrace[%zu] buf=%p %zu", i, static_cast<void*>(buf.begin),
+                  buf.size);
+    if (i % 2 == 0) {
+      if (!buf.begin) {
+        PERFETTO_ELOG("FAIL: the buffer was supposed to be not empty");
+      } else {
+        DumpTrace(buf);
+      }
+    }
+  }
+
+  PERFETTO_ILOG("Destroying");
+  for (size_t i = 0; i < handles.size(); i++)
+    Destroy(handles[i]);
+}
+}  // namespace
+
+int main() {
+  PERFETTO_LOG("Testing single trace");
+  PERFETTO_LOG("=============================================================");
+  TestSingle();
+  PERFETTO_LOG("=============================================================");
+
+  PERFETTO_LOG("\n");
+
+  PERFETTO_LOG("\n");
+  PERFETTO_LOG("Testing concurrent traces");
+  PERFETTO_LOG("=============================================================");
+  TestMany();
+  PERFETTO_LOG("=============================================================");
+
+  return 0;
+}