Trace Processor: add --httpd

This CL adds a built-in HTTP RPC interface to
TraceProcessor. It allows to use a native trace
proocessor with the UI over HTTP.
It also move the RPC-related classes in a rpc/ subdir.

Bug: 143074239
Change-Id: I05416f8f604bafa4a9d26dbe0ff0db25fdd4d555
diff --git a/BUILD b/BUILD
index 583f9c2..aa10c0d 100644
--- a/BUILD
+++ b/BUILD
@@ -587,6 +587,24 @@
     ],
 )
 
+# GN target: //src/trace_processor/rpc:httpd
+filegroup(
+    name = "src_trace_processor_rpc_httpd",
+    srcs = [
+        "src/trace_processor/rpc/httpd.cc",
+        "src/trace_processor/rpc/httpd.h",
+    ],
+)
+
+# GN target: //src/trace_processor/rpc:rpc
+filegroup(
+    name = "src_trace_processor_rpc_rpc",
+    srcs = [
+        "src/trace_processor/rpc/rpc.cc",
+        "src/trace_processor/rpc/rpc.h",
+    ],
+)
+
 # GN target: //src/trace_processor/sqlite:sqlite
 filegroup(
     name = "src_trace_processor_sqlite_sqlite",
@@ -1971,6 +1989,22 @@
     ],
 )
 
+# GN target: //protos/perfetto/trace_processor:zero
+perfetto_proto_library(
+    name = "protos_perfetto_trace_processor_protos",
+    srcs = [
+        "protos/perfetto/trace_processor/trace_processor.proto",
+    ],
+)
+
+# GN target: //protos/perfetto/trace_processor:zero
+perfetto_cc_protozero_library(
+    name = "protos_perfetto_trace_processor_zero",
+    deps = [
+        ":protos_perfetto_trace_processor_protos",
+    ],
+)
+
 # GN target: //protos/perfetto/trace/profiling:lite
 perfetto_cc_proto_library(
     name = "protos_perfetto_trace_profiling_lite",
@@ -2382,11 +2416,14 @@
         ":include_perfetto_protozero_protozero",
         ":include_perfetto_trace_processor_trace_processor",
         ":src_base_base",
+        ":src_base_unix_socket",
         ":src_protozero_protozero",
         ":src_trace_processor_common",
         ":src_trace_processor_db_lib",
         ":src_trace_processor_lib",
         ":src_trace_processor_metrics_lib",
+        ":src_trace_processor_rpc_httpd",
+        ":src_trace_processor_rpc_rpc",
         ":src_trace_processor_sqlite_sqlite",
         ":src_trace_processor_tables_tables",
     ],
@@ -2417,6 +2454,7 @@
                ":protos_perfetto_trace_perfetto_zero",
                ":protos_perfetto_trace_power_zero",
                ":protos_perfetto_trace_processor_metrics_impl_zero",
+               ":protos_perfetto_trace_processor_zero",
                ":protos_perfetto_trace_profiling_zero",
                ":protos_perfetto_trace_ps_zero",
                ":protos_perfetto_trace_sys_stats_zero",
diff --git a/gn/BUILD.gn b/gn/BUILD.gn
index 60c8654..3812690 100644
--- a/gn/BUILD.gn
+++ b/gn/BUILD.gn
@@ -57,6 +57,13 @@
   } else {
     perfetto_local_symbolizer = "0"
   }
+  if (enable_perfetto_trace_processor_httpd) {
+    perfetto_tp_httpd = "PERFETTO_BUILDFLAG_DEFINE_PERFETTO_OS_ANDROID() || " +
+                        "PERFETTO_BUILDFLAG_DEFINE_PERFETTO_OS_LINUX() || " +
+                        "PERFETTO_BUILDFLAG_DEFINE_PERFETTO_OS_MACOSX()"
+  } else {
+    perfetto_tp_httpd = "0"
+  }
 
   response_file_contents = [
     "--flags",  # Keep this marker first.
@@ -74,6 +81,7 @@
     "PERFETTO_TP_LINENOISE=$enable_perfetto_trace_processor_linenoise",
     "PERFETTO_TP_METRICS=$enable_perfetto_trace_processor_metrics",
     "PERFETTO_TP_FTRACE=$enable_perfetto_trace_processor_ftrace",
+    "PERFETTO_TP_HTTPD=$perfetto_tp_httpd",
     "PERFETTO_TP_SYSTEM_PROBES=$enable_perfetto_trace_processor_system_probes",
     "PERFETTO_TP_ANDROID_PROBES=$enable_perfetto_trace_processor_android_probes",
     "PERFETTO_TP_GRAPHICS=$enable_perfetto_trace_processor_graphics",
diff --git a/gn/perfetto.gni b/gn/perfetto.gni
index 1185ed4..a2ab7ef 100644
--- a/gn/perfetto.gni
+++ b/gn/perfetto.gni
@@ -233,6 +233,11 @@
   # Enables Fuchsia trace format support in trace processor.
   enable_perfetto_trace_processor_fuchsia =
       enable_perfetto_trace_processor && !build_with_chromium
+
+  # Enables httpd RPC support in the trace processor.
+  # Further per-OS conditionals are applied in gn/BUILD.gn.
+  enable_perfetto_trace_processor_httpd =
+      enable_perfetto_trace_processor && perfetto_build_standalone
 }
 
 # +---------------------------------------------------------------------------+
diff --git a/include/perfetto/base/build_configs/android_tree/perfetto_build_flags.h b/include/perfetto/base/build_configs/android_tree/perfetto_build_flags.h
index be6d5fc..3de1849 100644
--- a/include/perfetto/base/build_configs/android_tree/perfetto_build_flags.h
+++ b/include/perfetto/base/build_configs/android_tree/perfetto_build_flags.h
@@ -19,6 +19,7 @@
 #define PERFETTO_BUILDFLAG_DEFINE_PERFETTO_TP_LINENOISE() (0)
 #define PERFETTO_BUILDFLAG_DEFINE_PERFETTO_TP_METRICS() (1)
 #define PERFETTO_BUILDFLAG_DEFINE_PERFETTO_TP_FTRACE() (1)
+#define PERFETTO_BUILDFLAG_DEFINE_PERFETTO_TP_HTTPD() (0)
 #define PERFETTO_BUILDFLAG_DEFINE_PERFETTO_TP_SYSTEM_PROBES() (1)
 #define PERFETTO_BUILDFLAG_DEFINE_PERFETTO_TP_ANDROID_PROBES() (1)
 #define PERFETTO_BUILDFLAG_DEFINE_PERFETTO_TP_GRAPHICS() (1)
diff --git a/include/perfetto/base/build_configs/bazel/perfetto_build_flags.h b/include/perfetto/base/build_configs/bazel/perfetto_build_flags.h
index db367e5..4dcec86 100644
--- a/include/perfetto/base/build_configs/bazel/perfetto_build_flags.h
+++ b/include/perfetto/base/build_configs/bazel/perfetto_build_flags.h
@@ -19,6 +19,7 @@
 #define PERFETTO_BUILDFLAG_DEFINE_PERFETTO_TP_LINENOISE() (1)
 #define PERFETTO_BUILDFLAG_DEFINE_PERFETTO_TP_METRICS() (1)
 #define PERFETTO_BUILDFLAG_DEFINE_PERFETTO_TP_FTRACE() (1)
+#define PERFETTO_BUILDFLAG_DEFINE_PERFETTO_TP_HTTPD() (PERFETTO_BUILDFLAG_DEFINE_PERFETTO_OS_ANDROID() || PERFETTO_BUILDFLAG_DEFINE_PERFETTO_OS_LINUX() || PERFETTO_BUILDFLAG_DEFINE_PERFETTO_OS_MACOSX())
 #define PERFETTO_BUILDFLAG_DEFINE_PERFETTO_TP_SYSTEM_PROBES() (1)
 #define PERFETTO_BUILDFLAG_DEFINE_PERFETTO_TP_ANDROID_PROBES() (1)
 #define PERFETTO_BUILDFLAG_DEFINE_PERFETTO_TP_GRAPHICS() (1)
diff --git a/protos/perfetto/trace_processor/proto_files.gni b/protos/perfetto/trace_processor/proto_files.gni
index 2b7d53e..246b803 100644
--- a/protos/perfetto/trace_processor/proto_files.gni
+++ b/protos/perfetto/trace_processor/proto_files.gni
@@ -14,7 +14,4 @@
 
 # This variable is used both by ./BUILD.gn (for the C++ proto codegen) and by
 # //ui/BUIlD.gn (for the TypeScript/JS proto codegen).
-trace_processor_protos = [
-  "raw_query",
-  "trace_processor",
-]
+trace_processor_protos = [ "trace_processor" ]
diff --git a/protos/perfetto/trace_processor/raw_query.proto b/protos/perfetto/trace_processor/raw_query.proto
deleted file mode 100644
index 904b902..0000000
--- a/protos/perfetto/trace_processor/raw_query.proto
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * 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;
-
-message RawQueryArgs {
-  optional string sql_query = 1;
-
-  // Wall time when the query was queued. Used only for query stats.
-  optional uint64 time_queued_ns = 2;
-}
-
-message RawQueryResult {
-  message ColumnDesc {
-    optional string name = 1;
-    enum Type {
-      UNKNOWN = 0;
-      LONG = 1;
-      DOUBLE = 2;
-      STRING = 3;
-    }
-    optional Type type = 2;
-  }
-  message ColumnValues {
-    // Only one of this field will be filled for each column (according to the
-    // corresponding descriptor) and that one will have precisely |num_records|
-    // elements.
-    repeated int64 long_values = 1;
-    repeated double double_values = 2;
-    repeated string string_values = 3;
-
-    // This will be set to true or false depending on whether the data at the
-    // given index is NULL.
-    repeated bool is_nulls = 4;
-  }
-  repeated ColumnDesc column_descriptors = 1;
-  optional uint64 num_records = 2;
-  repeated ColumnValues columns = 3;
-  optional string error = 4;
-  optional uint64 execution_time_ns = 5;
-}
diff --git a/protos/perfetto/trace_processor/trace_processor.proto b/protos/perfetto/trace_processor/trace_processor.proto
index 79cbcc0..64db589 100644
--- a/protos/perfetto/trace_processor/trace_processor.proto
+++ b/protos/perfetto/trace_processor/trace_processor.proto
@@ -17,10 +17,45 @@
 syntax = "proto2";
 option optimize_for = LITE_RUNTIME;
 
-import "protos/perfetto/trace_processor/raw_query.proto";
-
 package perfetto.protos;
 
 service TraceProcessor {
   rpc RawQuery(RawQueryArgs) returns (RawQueryResult) {}
 }
+
+message RawQueryArgs {
+  optional string sql_query = 1;
+
+  // Wall time when the query was queued. Used only for query stats.
+  optional uint64 time_queued_ns = 2;
+}
+
+message RawQueryResult {
+  message ColumnDesc {
+    optional string name = 1;
+    enum Type {
+      UNKNOWN = 0;
+      LONG = 1;
+      DOUBLE = 2;
+      STRING = 3;
+    }
+    optional Type type = 2;
+  }
+  message ColumnValues {
+    // Only one of this field will be filled for each column (according to the
+    // corresponding descriptor) and that one will have precisely |num_records|
+    // elements.
+    repeated int64 long_values = 1;
+    repeated double double_values = 2;
+    repeated string string_values = 3;
+
+    // This will be set to true or false depending on whether the data at the
+    // given index is NULL.
+    repeated bool is_nulls = 4;
+  }
+  repeated ColumnDesc column_descriptors = 1;
+  optional uint64 num_records = 2;
+  repeated ColumnValues columns = 3;
+  optional string error = 4;
+  optional uint64 execution_time_ns = 5;
+}
diff --git a/src/trace_processor/BUILD.gn b/src/trace_processor/BUILD.gn
index 23a10de..13a8f42 100644
--- a/src/trace_processor/BUILD.gn
+++ b/src/trace_processor/BUILD.gn
@@ -21,24 +21,6 @@
 # Prevent that this file is accidentally included in embedder builds.
 assert(enable_perfetto_trace_processor)
 
-if (enable_perfetto_ui) {
-  wasm_lib("trace_processor_wasm") {
-    name = "trace_processor"
-    sources = [
-      "wasm_bridge.cc",
-    ]
-
-    # All the deps below are implicitly xxx($wasm_toolchain) because of the
-    # outer "is_wasm" block.
-    deps = [
-      ":lib",
-      ":rpc",
-      "../../gn:default_deps",
-      "../../include/perfetto/ext/base",
-    ]
-  }
-}
-
 # The library which eases processing of Perfetto traces by exposing reading
 # friendly APIs.
 static_library("trace_processor") {
@@ -48,6 +30,18 @@
   ]
 }
 
+if (enable_perfetto_ui) {
+  wasm_lib("trace_processor_wasm") {
+    name = "trace_processor"
+    deps = [
+      ":lib",
+      "../../gn:default_deps",
+      "../base",
+      "rpc:wasm_bridge",
+    ]
+  }
+}
+
 source_set("lib") {
   sources = [
     "android_logs_table.cc",
@@ -276,22 +270,6 @@
   ]
 }
 
-# This source_set is used both by WASM (for the function-call-based query
-# interface) and by the upcoming :httpd module for the HTTP interface.
-source_set("rpc") {
-  sources = [
-    "rpc.cc",
-    "rpc.h",
-  ]
-  deps = [
-    ":lib",
-    "../../gn:default_deps",
-    "../../include/perfetto/ext/base",
-    "../../include/perfetto/protozero",
-    "../../protos/perfetto/trace_processor:zero",
-  ]
-}
-
 if (enable_perfetto_trace_processor_metrics) {  # shell requires metrics.
   perfetto_host_executable("trace_processor_shell") {
     deps = [
@@ -307,7 +285,9 @@
     if (enable_perfetto_trace_processor_linenoise) {
       deps += [ "../../gn:linenoise" ]
     }
-
+    if (enable_perfetto_trace_processor_httpd) {
+      deps += [ "rpc:httpd" ]
+    }
     sources = [
       "proto_to_json.cc",
       "proto_to_json.h",
diff --git a/src/trace_processor/rpc/BUILD.gn b/src/trace_processor/rpc/BUILD.gn
new file mode 100644
index 0000000..f6c188d
--- /dev/null
+++ b/src/trace_processor/rpc/BUILD.gn
@@ -0,0 +1,66 @@
+# Copyright (C) 2019 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.
+
+import("../../../gn/perfetto.gni")
+import("../../../gn/wasm.gni")
+
+# Prevent that this file is accidentally included in embedder builds.
+assert(enable_perfetto_trace_processor)
+
+# This source_set is used both by WASM (for the function-call-based query
+# interface) and by the :httpd module for the HTTP interface.
+source_set("rpc") {
+  sources = [
+    "rpc.cc",
+    "rpc.h",
+  ]
+  deps = [
+    "../../../gn:default_deps",
+    "../../../include/perfetto/trace_processor",
+    "../../../protos/perfetto/trace_processor:zero",
+    "../../base",
+    "../../protozero",
+  ]
+}
+
+if (enable_perfetto_trace_processor_httpd) {
+  source_set("httpd") {
+    sources = [
+      "httpd.cc",
+      "httpd.h",
+    ]
+    deps = [
+      ":rpc",
+      "../../../gn:default_deps",
+      "../../../include/perfetto/trace_processor",
+      "../../../protos/perfetto/trace_processor:zero",
+      "../../base",
+      "../../base:unix_socket",
+    ]
+  }
+}
+
+if (enable_perfetto_ui && is_wasm) {
+  source_set("wasm_bridge") {
+    sources = [
+      "wasm_bridge.cc",
+    ]
+    deps = [
+      ":rpc",
+      "../../../gn:default_deps",
+      "../../../include/perfetto/trace_processor",
+      "../../base",
+    ]
+  }
+}
diff --git a/src/trace_processor/rpc/README.md b/src/trace_processor/rpc/README.md
new file mode 100644
index 0000000..802be0e
--- /dev/null
+++ b/src/trace_processor/rpc/README.md
@@ -0,0 +1,15 @@
+# TraceProcessor RPC
+
+This directory contains the RPC interfaces to use the Perfetto Trace Processor
+remotely (i.e. not just in-process). It consists of two targets:
+
+## `wasm_bridge`
+
+The WASM (Web Asssembly) interop bridge. It's used to call the Trace Processor
+from HTML/JS using WASM's `ccall`.
+
+## `httpd`
+
+The HTTP RPC module. It exposes a protobuf-over-HTTP RPC interface that allows
+interacting with a remote trace processor instance. It's used for special UI
+use cases (very large traces > 2GB) and for python interoperability.
diff --git a/src/trace_processor/rpc/httpd.cc b/src/trace_processor/rpc/httpd.cc
new file mode 100644
index 0000000..c7c35bf
--- /dev/null
+++ b/src/trace_processor/rpc/httpd.cc
@@ -0,0 +1,301 @@
+/*
+ * Copyright (C) 2019 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/base/build_config.h"
+
+#if PERFETTO_BUILDFLAG(PERFETTO_TP_HTTPD)
+
+#include "src/trace_processor/rpc/httpd.h"
+
+#include <map>
+#include <string>
+
+#include "perfetto/ext/base/paged_memory.h"
+#include "perfetto/ext/base/string_utils.h"
+#include "perfetto/ext/base/string_view.h"
+#include "perfetto/ext/base/unix_socket.h"
+#include "perfetto/ext/base/unix_task_runner.h"
+#include "perfetto/trace_processor/trace_processor.h"
+#include "src/trace_processor/rpc/rpc.h"
+
+#include "protos/perfetto/trace_processor/trace_processor.pbzero.h"
+
+namespace perfetto {
+namespace trace_processor {
+
+namespace {
+
+constexpr char kBindAddr[] = "127.0.0.1:9001";
+constexpr auto kBlocking = base::UnixSocket::BlockingMode::kBlocking;
+
+// 32 MiB payload + 128K for HTTP headers.
+constexpr size_t kMaxRequestSize = (32 * 1024 + 128) * 1024;
+
+// Owns the socket and data for one HTTP client connection.
+struct Client {
+  Client(std::unique_ptr<base::UnixSocket> s)
+      : sock(std::move(s)),
+        rxbuf(base::PagedMemory::Allocate(kMaxRequestSize)) {}
+  size_t rxbuf_avail() { return rxbuf.size() - rxbuf_used; }
+
+  std::unique_ptr<base::UnixSocket> sock;
+  base::PagedMemory rxbuf;
+  size_t rxbuf_used = 0;
+};
+
+struct HttpRequest {
+  base::StringView method;
+  base::StringView uri;
+  base::StringView origin;
+  base::StringView body;
+};
+
+class HttpServer : public base::UnixSocket::EventListener {
+ public:
+  ~HttpServer() override;
+  void Run();
+
+ private:
+  size_t ParseOneHttpRequest(Client* client);
+  void HandleRequest(Client*, const HttpRequest&);
+
+  void OnNewIncomingConnection(base::UnixSocket*,
+                               std::unique_ptr<base::UnixSocket>) override;
+  void OnConnect(base::UnixSocket* self, bool connected) override;
+  void OnDisconnect(base::UnixSocket* self) override;
+  void OnDataAvailable(base::UnixSocket* self) override;
+
+  Rpc trace_processor_rpc_;
+  base::UnixTaskRunner task_runner_;
+  std::unique_ptr<base::UnixSocket> sock_;
+  std::vector<Client> clients_;
+};
+
+void Append(std::vector<char>& buf, const char* str) {
+  buf.insert(buf.end(), str, str + strlen(str));
+}
+
+void Append(std::vector<char>& buf, const std::string& str) {
+  buf.insert(buf.end(), str.begin(), str.end());
+}
+
+void HttpReply(base::UnixSocket* sock,
+               const char* http_code,
+               std::initializer_list<const char*> headers = {},
+               const uint8_t* body = nullptr,
+               size_t body_len = 0) {
+  std::vector<char> response;
+  response.reserve(4096);
+  Append(response, "HTTP/1.1 ");
+  Append(response, http_code);
+  Append(response, "\r\n");
+  for (const char* hdr : headers) {
+    Append(response, hdr);
+    Append(response, "\r\n");
+  }
+  Append(response, "Content-Length: ");
+  Append(response, std::to_string(body_len));
+  Append(response, "\r\n\r\n");  // End-of-headers marker.
+  sock->Send(response.data(), response.size(), /*fd=*/-1, kBlocking);
+  if (body_len)
+    sock->Send(body, body_len, /*fd=*/-1, kBlocking);
+}
+
+void ShutdownBadRequest(base::UnixSocket* sock, const char* reason) {
+  HttpReply(sock, "500 Bad Request", {},
+            reinterpret_cast<const uint8_t*>(reason), strlen(reason));
+  sock->Shutdown(/*notify=*/true);
+}
+
+HttpServer::~HttpServer() = default;
+
+void HttpServer::Run() {
+  PERFETTO_ILOG("[HTTP] Starting RPC server on %s", kBindAddr);
+  sock_ = base::UnixSocket::Listen(kBindAddr, this, &task_runner_,
+                                   base::SockFamily::kInet,
+                                   base::SockType::kStream);
+  task_runner_.Run();
+}
+
+void HttpServer::OnNewIncomingConnection(
+    base::UnixSocket*,
+    std::unique_ptr<base::UnixSocket> sock) {
+  PERFETTO_DLOG("[HTTP] New connection");
+  clients_.emplace_back(std::move(sock));
+}
+
+void HttpServer::OnConnect(base::UnixSocket*, bool) {}
+
+void HttpServer::OnDisconnect(base::UnixSocket* sock) {
+  PERFETTO_DLOG("[HTTP] Client disconnected");
+  for (auto it = clients_.begin(); it != clients_.end(); ++it) {
+    if (it->sock.get() == sock) {
+      clients_.erase(it);
+      return;
+    }
+  }
+  PERFETTO_DFATAL("[HTTP] untracked client in OnDisconnect()");
+}
+
+void HttpServer::OnDataAvailable(base::UnixSocket* sock) {
+  Client* client = nullptr;
+  for (auto it = clients_.begin(); it != clients_.end() && !client; ++it)
+    client = (it->sock.get() == sock) ? &*it : nullptr;
+  PERFETTO_CHECK(client);
+
+  char* rxbuf = reinterpret_cast<char*>(client->rxbuf.Get());
+  for (;;) {
+    size_t avail = client->rxbuf_avail();
+    PERFETTO_CHECK(avail <= kMaxRequestSize);
+    if (avail == 0)
+      return ShutdownBadRequest(sock, "Request body too big");
+    size_t rsize = sock->Receive(&rxbuf[client->rxbuf_used], avail);
+    client->rxbuf_used += rsize;
+    if (rsize == 0 || client->rxbuf_avail() == 0)
+      break;
+  }
+
+  // At this point |rxbuf| can contain a partial HTTP request, a full one or
+  // more (in case of HTTP Keepalive pipelining).
+  for (;;) {
+    size_t bytes_consumed = ParseOneHttpRequest(client);
+    if (bytes_consumed == 0)
+      break;
+    memmove(rxbuf, &rxbuf[bytes_consumed], client->rxbuf_used - bytes_consumed);
+    client->rxbuf_used -= bytes_consumed;
+  }
+}
+
+// Parses the HTTP request and invokes HandleRequest(). It returns the size of
+// the HTTP header + body that has been processed or 0 if there isn't enough
+// data for a full HTTP request in the buffer.
+size_t HttpServer::ParseOneHttpRequest(Client* client) {
+  auto* rxbuf = reinterpret_cast<char*>(client->rxbuf.Get());
+  base::StringView buf_view(rxbuf, client->rxbuf_used);
+  size_t pos = 0;
+  size_t body_offset = 0;
+  size_t body_size = 0;
+  bool has_parsed_first_line = false;
+  HttpRequest http_req;
+
+  // This loop parses the HTTP request headers and sets the |body_offset|.
+  for (;;) {
+    size_t next = buf_view.find("\r\n", pos);
+    size_t col;
+    if (next == std::string::npos)
+      break;
+
+    if (!has_parsed_first_line) {
+      // Parse the "GET /xxx HTTP/1.1" line.
+      has_parsed_first_line = true;
+      size_t space = buf_view.find(' ');
+      if (space == std::string::npos || space + 2 >= client->rxbuf_used) {
+        ShutdownBadRequest(client->sock.get(), "Malformed HTTP request");
+        return 0;
+      }
+      http_req.method = buf_view.substr(0, space);
+      size_t uri_size = buf_view.find(' ', space + 1) - space - 1;
+      http_req.uri = buf_view.substr(space + 1, uri_size);
+    } else if (next == pos) {
+      // The CR-LF marker that separates headers from body.
+      body_offset = next + 2;
+      break;
+    } else if ((col = buf_view.find(':', pos)) < next) {
+      // Parse HTTP headers. They look like: "Content-Length: 1234".
+      auto hdr_name = buf_view.substr(pos, col - pos);
+      auto hdr_value = buf_view.substr(col + 2, next - col - 2);
+      if (hdr_name.CaseInsensitiveEq("content-length")) {
+        body_size = static_cast<size_t>(atoi(hdr_value.ToStdString().c_str()));
+      } else if (hdr_name.CaseInsensitiveEq("origin")) {
+        http_req.origin = hdr_value;
+      }
+    }
+    pos = next + 2;
+  }
+
+  // If we have a full header but not yet the full body, return and try again
+  // next time we receive some more data.
+  size_t http_req_size = body_offset + body_size;
+  if (!body_offset || client->rxbuf_used < http_req_size)
+    return 0;
+
+  http_req.body = base::StringView(&rxbuf[body_offset], body_size);
+  HandleRequest(client, http_req);
+  return http_req_size;
+}
+
+void HttpServer::HandleRequest(Client* client, const HttpRequest& req) {
+  PERFETTO_LOG("[HTTP] %s %s (body: %zu bytes)",
+               req.method.ToStdString().c_str(), req.uri.ToStdString().c_str(),
+               req.body.size());
+  std::string allow_origin_hdr =
+      "Access-Control-Allow-Origin: " + req.origin.ToStdString();
+  std::initializer_list<const char*> headers = {
+      "Connection: Keep-Alive",                //
+      "Access-Control-Expose-Headers: *",      //
+      "Keep-Alive: timeout=5, max=1000",       //
+      "Content-Type: application/x-protobuf",  //
+      allow_origin_hdr.c_str()};
+
+  if (req.method == "OPTIONS") {
+    // CORS headers.
+    return HttpReply(client->sock.get(), "204 No Content",
+                     {
+                         "Access-Control-Allow-Methods: POST, GET, OPTIONS",
+                         "Access-Control-Allow-Headers: *",
+                         "Access-Control-Max-Age: 600",
+                         allow_origin_hdr.c_str(),
+                     });
+  }
+
+  if (req.uri == "/parse") {
+    trace_processor_rpc_.LoadTrace(
+        reinterpret_cast<const uint8_t*>(req.body.data()), req.body.size(),
+        /*eof=*/false);
+    return HttpReply(client->sock.get(), "200 OK", headers);
+  }
+
+  if (req.uri == "/notifyeof") {
+    trace_processor_rpc_.LoadTrace(nullptr, 0, /*eof=*/true);
+    return HttpReply(client->sock.get(), "200 OK", headers);
+  }
+
+  if (req.uri == "/rawquery") {
+    PERFETTO_CHECK(req.body.size() > 0u);
+    std::vector<uint8_t> response = trace_processor_rpc_.RawQuery(
+        reinterpret_cast<const uint8_t*>(req.body.data()), req.body.size());
+    return HttpReply(client->sock.get(), "200 OK", headers, response.data(),
+                     response.size());
+  }
+
+  if (req.uri == "/status") {
+    return HttpReply(client->sock.get(), "200 OK", headers, nullptr, 0);
+  }
+
+  return HttpReply(client->sock.get(), "404 Not Found", headers);
+}
+
+}  // namespace
+
+void RunHttpRPCServer() {
+  HttpServer srv;
+  srv.Run();
+}
+
+}  // namespace trace_processor
+}  // namespace perfetto
+
+#endif  // PERFETTO_TP_HTTPD
diff --git a/src/trace_processor/rpc/httpd.h b/src/trace_processor/rpc/httpd.h
new file mode 100644
index 0000000..235e083
--- /dev/null
+++ b/src/trace_processor/rpc/httpd.h
@@ -0,0 +1,30 @@
+/*
+ * Copyright (C) 2019 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_TRACE_PROCESSOR_RPC_HTTPD_H_
+#define SRC_TRACE_PROCESSOR_RPC_HTTPD_H_
+
+namespace perfetto {
+namespace trace_processor {
+
+// Starts a RPC server that handles requests using protobuf-over-HTTP.
+// It takes control of the calling thread and does not return.
+void RunHttpRPCServer();
+
+}  // namespace trace_processor
+}  // namespace perfetto
+
+#endif  // SRC_TRACE_PROCESSOR_RPC_HTTPD_H_
diff --git a/src/trace_processor/rpc.cc b/src/trace_processor/rpc/rpc.cc
similarity index 98%
rename from src/trace_processor/rpc.cc
rename to src/trace_processor/rpc/rpc.cc
index e84df1c..a006b19 100644
--- a/src/trace_processor/rpc.cc
+++ b/src/trace_processor/rpc/rpc.cc
@@ -14,14 +14,14 @@
  * limitations under the License.
  */
 
-#include "src/trace_processor/rpc.h"
+#include "src/trace_processor/rpc/rpc.h"
 
 #include <vector>
 
 #include "perfetto/base/time.h"
 #include "perfetto/protozero/scattered_heap_buffer.h"
 #include "perfetto/trace_processor/trace_processor.h"
-#include "protos/perfetto/trace_processor/raw_query.pbzero.h"
+#include "protos/perfetto/trace_processor/trace_processor.pbzero.h"
 
 namespace perfetto {
 namespace trace_processor {
diff --git a/src/trace_processor/rpc.h b/src/trace_processor/rpc/rpc.h
similarity index 94%
rename from src/trace_processor/rpc.h
rename to src/trace_processor/rpc/rpc.h
index 4ce3e32..6cf91c2 100644
--- a/src/trace_processor/rpc.h
+++ b/src/trace_processor/rpc/rpc.h
@@ -14,8 +14,8 @@
  * limitations under the License.
  */
 
-#ifndef SRC_TRACE_PROCESSOR_RPC_H_
-#define SRC_TRACE_PROCESSOR_RPC_H_
+#ifndef SRC_TRACE_PROCESSOR_RPC_RPC_H_
+#define SRC_TRACE_PROCESSOR_RPC_RPC_H_
 
 #include <memory>
 #include <vector>
@@ -65,7 +65,7 @@
   // Executes a SQL query and returns the results.
   // Args: RawQueryArgs proto-encoded bytes.
   // Returns: RawQueryResult proto-encoded bytes.
-  // See protos/perfetto/trace_processor/raw_query.proto for the proto schema.
+  // See trace_processor.proto for the proto schema.
   // If the query fails the |error| RawQueryResult.field is set accordingly
   std::vector<uint8_t> RawQuery(const uint8_t* args, size_t len);
 
@@ -80,4 +80,4 @@
 }  // namespace trace_processor
 }  // namespace perfetto
 
-#endif  // SRC_TRACE_PROCESSOR_RPC_H_
+#endif  // SRC_TRACE_PROCESSOR_RPC_RPC_H_
diff --git a/src/trace_processor/wasm_bridge.cc b/src/trace_processor/rpc/wasm_bridge.cc
similarity index 98%
rename from src/trace_processor/wasm_bridge.cc
rename to src/trace_processor/rpc/wasm_bridge.cc
index a09385c..f78eeb9 100644
--- a/src/trace_processor/wasm_bridge.cc
+++ b/src/trace_processor/rpc/wasm_bridge.cc
@@ -20,7 +20,7 @@
 
 #include "perfetto/base/logging.h"
 #include "perfetto/trace_processor/trace_processor.h"
-#include "src/trace_processor/rpc.h"
+#include "src/trace_processor/rpc/rpc.h"
 
 namespace perfetto {
 namespace trace_processor {
diff --git a/src/trace_processor/trace_processor_shell.cc b/src/trace_processor/trace_processor_shell.cc
index c113e95..25ffb6f 100644
--- a/src/trace_processor/trace_processor_shell.cc
+++ b/src/trace_processor/trace_processor_shell.cc
@@ -39,6 +39,10 @@
 #include "src/trace_processor/metrics/metrics.descriptor.h"
 #include "src/trace_processor/proto_to_json.h"
 
+#if PERFETTO_BUILDFLAG(PERFETTO_TP_HTTPD)
+#include "src/trace_processor/rpc/httpd.h"
+#endif
+
 #if PERFETTO_BUILDFLAG(PERFETTO_OS_LINUX) ||   \
     PERFETTO_BUILDFLAG(PERFETTO_OS_ANDROID) || \
     PERFETTO_BUILDFLAG(PERFETTO_OS_MACOSX)
@@ -635,6 +639,7 @@
   std::string metric_extra;
   std::string trace_file_path;
   bool launch_shell = false;
+  bool enable_httpd = false;
   bool wide = false;
   bool force_full_sort = false;
 };
@@ -694,6 +699,7 @@
                                       If used with --run-metrics, the query is
                                       executed after the selected metrics and
                                       the metrics output is suppressed.
+ -D, --httpd                          Enables the HTTP RPC server.
  -i, --interactive                    Starts interactive mode even after a query
                                       file is specified with -q or
                                       --run-metrics.
@@ -730,6 +736,7 @@
       {"help", no_argument, nullptr, 'h'},
       {"version", no_argument, nullptr, 'v'},
       {"wide", no_argument, nullptr, 'W'},
+      {"httpd", no_argument, nullptr, 'D'},
       {"interactive", no_argument, nullptr, 'i'},
       {"debug", no_argument, nullptr, 'd'},
       {"perf-file", required_argument, nullptr, 'p'},
@@ -745,7 +752,7 @@
   int option_index = 0;
   for (;;) {
     int option =
-        getopt_long(argc, argv, "hvWidp:q:e:", long_options, &option_index);
+        getopt_long(argc, argv, "hvWiDdp:q:e:", long_options, &option_index);
 
     if (option == -1)
       break;  // EOF.
@@ -760,6 +767,15 @@
       continue;
     }
 
+    if (option == 'D') {
+#if PERFETTO_BUILDFLAG(PERFETTO_TP_HTTPD)
+      command_line_options.enable_httpd = true;
+#else
+      PERFETTO_FATAL("HTTP RPC module not supported in this build");
+#endif
+      continue;
+    }
+
     if (option == 'W') {
       command_line_options.wide = true;
       continue;
@@ -873,6 +889,13 @@
 int TraceProcessorMain(int argc, char** argv) {
   CommandLineOptions options = ParseCommandLineOptions(argc, argv);
 
+#if PERFETTO_BUILDFLAG(PERFETTO_TP_HTTPD)
+  if (options.enable_httpd) {
+    RunHttpRPCServer();
+    return 0;
+  }
+#endif
+
   // Load the trace file into the trace processor.
   Config config;
   config.force_full_sort = options.force_full_sort;