Proto filtering: add core MessageFilter and FilterUtil

This CL adds:
- The core MessageFilter class, which will take care
  of filtering on-device.
- The FilterUtil class and related tools/proto_filter
  to generate / test filters offline on the host.

Design doc: go/trace-filtering
Bug: 181306741
Test: see new test and fuzzers introduced

Change-Id: I516669d3b6eee746ab306b1f0edd23e89bfbafb3
diff --git a/Android.bp b/Android.bp
index 7f7656a..f81a6a7 100644
--- a/Android.bp
+++ b/Android.bp
@@ -7290,12 +7290,30 @@
   ],
 }
 
+// GN: //src/protozero/filtering:filter_util
+filegroup {
+  name: "perfetto_src_protozero_filtering_filter_util",
+  srcs: [
+    "src/protozero/filtering/filter_util.cc",
+  ],
+}
+
+// GN: //src/protozero/filtering:message_filter
+filegroup {
+  name: "perfetto_src_protozero_filtering_message_filter",
+  srcs: [
+    "src/protozero/filtering/message_filter.cc",
+  ],
+}
+
 // GN: //src/protozero/filtering:unittests
 filegroup {
   name: "perfetto_src_protozero_filtering_unittests",
   srcs: [
     "src/protozero/filtering/filter_bytecode_generator_unittest.cc",
     "src/protozero/filtering/filter_bytecode_parser_unittest.cc",
+    "src/protozero/filtering/filter_util_unittest.cc",
+    "src/protozero/filtering/message_filter_unittest.cc",
     "src/protozero/filtering/message_tokenizer_unittest.cc",
   ],
 }
@@ -9033,6 +9051,8 @@
     ":perfetto_src_protozero_filtering_bytecode_common",
     ":perfetto_src_protozero_filtering_bytecode_generator",
     ":perfetto_src_protozero_filtering_bytecode_parser",
+    ":perfetto_src_protozero_filtering_filter_util",
+    ":perfetto_src_protozero_filtering_message_filter",
     ":perfetto_src_protozero_filtering_unittests",
     ":perfetto_src_protozero_protozero",
     ":perfetto_src_protozero_testing_messages_cpp_gen",
@@ -9126,6 +9146,7 @@
     "libbase",
     "liblog",
     "libprocinfo",
+    "libprotobuf-cpp-full",
     "libprotobuf-cpp-lite",
     "libsqlite",
     "libunwindstack",
diff --git a/gn/BUILD.gn b/gn/BUILD.gn
index a297a84..1f799ad 100644
--- a/gn/BUILD.gn
+++ b/gn/BUILD.gn
@@ -195,7 +195,9 @@
 protobuf_full_deps_allowlist = [
   "../src/ipc/protoc_plugin:*",
   "../src/protozero/protoc_plugin:*",
+  "../src/protozero/filtering:filter_util",
   "../src/trace_processor:trace_processor_shell",
+  "../src/protozero/filtering:filter_util",
   "../tools/*",
 ]
 
diff --git a/gn/perfetto_benchmarks.gni b/gn/perfetto_benchmarks.gni
index 2bde376..e154064 100644
--- a/gn/perfetto_benchmarks.gni
+++ b/gn/perfetto_benchmarks.gni
@@ -18,6 +18,7 @@
   "gn:default_deps",
   "src/base:benchmarks",
   "src/protozero:benchmarks",
+  "src/protozero/filtering:benchmarks",
   "src/trace_processor/sqlite:benchmarks",
   "src/trace_processor/containers:benchmarks",
   "src/trace_processor/tables:benchmarks",
diff --git a/gn/perfetto_fuzzers.gni b/gn/perfetto_fuzzers.gni
index 049d525..5874f8e 100644
--- a/gn/perfetto_fuzzers.gni
+++ b/gn/perfetto_fuzzers.gni
@@ -19,6 +19,7 @@
   "src/ipc:buffered_frame_deserializer_fuzzer",
   "src/protozero:protozero_decoder_fuzzer",
   "src/protozero/filtering:protozero_bytecode_parser_fuzzer",
+  "src/protozero/filtering:protozero_message_filter_fuzzer",
   "src/tracing/core:packet_stream_validator_fuzzer",
   "src/trace_processor:trace_processor_fuzzer",
   "src/traced/probes/ftrace:cpu_reader_fuzzer",
diff --git a/include/perfetto/protozero/proto_utils.h b/include/perfetto/protozero/proto_utils.h
index 841042b..6b195cb 100644
--- a/include/perfetto/protozero/proto_utils.h
+++ b/include/perfetto/protozero/proto_utils.h
@@ -201,12 +201,20 @@
 // used to backfill fixed-size reservations for the length field using a
 // non-canonical varint encoding (e.g. \x81\x80\x80\x00 instead of \x01).
 // See https://github.com/google/protobuf/issues/1530.
-// In particular, this is used for nested messages. The size of a nested message
-// is not known until all its field have been written. |kMessageLengthFieldSize|
-// bytes are reserved to encode the size field and backfilled at the end.
-inline void WriteRedundantVarInt(uint32_t value, uint8_t* buf) {
-  for (size_t i = 0; i < kMessageLengthFieldSize; ++i) {
-    const uint8_t msb = (i < kMessageLengthFieldSize - 1) ? 0x80 : 0;
+// This is used mainly in two cases:
+// 1) At trace writing time, when starting a nested messages. The size of a
+//    nested message is not known until all its field have been written.
+//    |kMessageLengthFieldSize| bytes are reserved to encode the size field and
+//    backfilled at the end.
+// 2) When rewriting a message at trace filtering time, in protozero/filtering.
+//    At that point we know only the upper bound of the length (a filtered
+//    message is <= the original one) and we backfill after the message has been
+//    filtered.
+inline void WriteRedundantVarInt(uint32_t value,
+                                 uint8_t* buf,
+                                 size_t size = kMessageLengthFieldSize) {
+  for (size_t i = 0; i < size; ++i) {
+    const uint8_t msb = (i < size - 1) ? 0x80 : 0;
     buf[i] = static_cast<uint8_t>(value) | msb;
     value >>= 7;
   }
diff --git a/src/protozero/filtering/BUILD.gn b/src/protozero/filtering/BUILD.gn
index 3afdaab..6403195 100644
--- a/src/protozero/filtering/BUILD.gn
+++ b/src/protozero/filtering/BUILD.gn
@@ -17,6 +17,19 @@
 import("../../../gn/proto_library.gni")
 import("../../../gn/test.gni")
 
+source_set("message_filter") {
+  sources = [
+    "message_filter.cc",
+    "message_filter.h",
+  ]
+  deps = [
+    ":bytecode_parser",
+    "..:protozero",
+    "../../../gn:default_deps",
+    "../../base",
+  ]
+}
+
 source_set("bytecode_common") {
   sources = [ "filter_bytecode_common.h" ]
   deps = [ "../../../gn:default_deps" ]
@@ -48,25 +61,59 @@
   ]
 }
 
+source_set("filter_util") {
+  testonly = true
+  sources = [
+    "filter_util.cc",
+    "filter_util.h",
+  ]
+  deps = [
+    ":bytecode_generator",
+    "..:protozero",
+    "../../../gn:default_deps",
+    "../../../gn:protobuf_full",
+    "../../base",
+  ]
+}
+
 perfetto_unittest_source_set("unittests") {
   testonly = true
   deps = [
     ":bytecode_common",
     ":bytecode_generator",
     ":bytecode_parser",
+    ":filter_util",
+    ":message_filter",
     "..:protozero",
     "../../../gn:default_deps",
     "../../../gn:gtest_and_gmock",
+    "../../../protos/perfetto/trace:lite",
     "../../base",
     "../../base:test_support",
   ]
   sources = [
     "filter_bytecode_generator_unittest.cc",
     "filter_bytecode_parser_unittest.cc",
+    "filter_util_unittest.cc",
+    "message_filter_unittest.cc",
     "message_tokenizer_unittest.cc",
   ]
 }
 
+if (enable_perfetto_benchmarks) {
+  source_set("benchmarks") {
+    testonly = true
+    deps = [
+      ":message_filter",
+      "../../../gn:benchmark",
+      "../../../gn:default_deps",
+      "../../base",
+      "../../base:test_support",
+    ]
+    sources = [ "message_filter_benchmark.cc" ]
+  }
+}
+
 perfetto_fuzzer_test("protozero_bytecode_parser_fuzzer") {
   sources = [ "filter_bytecode_parser_fuzzer.cc" ]
   deps = [
@@ -76,3 +123,13 @@
     "../../base",
   ]
 }
+
+perfetto_fuzzer_test("protozero_message_filter_fuzzer") {
+  sources = [ "message_filter_fuzzer.cc" ]
+  deps = [
+    ":message_filter",
+    "..:protozero",
+    "../../../gn:default_deps",
+    "../../base",
+  ]
+}
diff --git a/src/protozero/filtering/filter_util.cc b/src/protozero/filtering/filter_util.cc
new file mode 100644
index 0000000..c17b620
--- /dev/null
+++ b/src/protozero/filtering/filter_util.cc
@@ -0,0 +1,301 @@
+/*
+ * Copyright (C) 2021 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/protozero/filtering/filter_util.h"
+
+#include <algorithm>
+#include <map>
+#include <memory>
+#include <set>
+
+#include <google/protobuf/compiler/importer.h>
+
+#include "perfetto/ext/base/file_utils.h"
+#include "perfetto/ext/base/getopt.h"
+#include "perfetto/ext/base/string_utils.h"
+#include "perfetto/ext/base/version.h"
+#include "perfetto/protozero/proto_utils.h"
+#include "src/protozero/filtering/filter_bytecode_generator.h"
+
+namespace protozero {
+
+namespace {
+
+class MultiFileErrorCollectorImpl
+    : public google::protobuf::compiler::MultiFileErrorCollector {
+ public:
+  ~MultiFileErrorCollectorImpl() override;
+  void AddError(const std::string&, int, int, const std::string&) override;
+  void AddWarning(const std::string&, int, int, const std::string&) override;
+};
+
+MultiFileErrorCollectorImpl::~MultiFileErrorCollectorImpl() = default;
+
+void MultiFileErrorCollectorImpl::AddError(const std::string& filename,
+                                           int line,
+                                           int column,
+                                           const std::string& message) {
+  PERFETTO_ELOG("Error %s %d:%d: %s", filename.c_str(), line, column,
+                message.c_str());
+}
+
+void MultiFileErrorCollectorImpl::AddWarning(const std::string& filename,
+                                             int line,
+                                             int column,
+                                             const std::string& message) {
+  PERFETTO_ELOG("Warning %s %d:%d: %s", filename.c_str(), line, column,
+                message.c_str());
+}
+
+}  // namespace
+
+FilterUtil::FilterUtil() = default;
+FilterUtil::~FilterUtil() = default;
+
+bool FilterUtil::LoadMessageDefinition(const std::string& proto_file,
+                                       const std::string& root_message,
+                                       const std::string& proto_dir_path) {
+  google::protobuf::compiler::DiskSourceTree dst;
+  dst.MapPath("/", "/");
+  dst.MapPath("", proto_dir_path);
+  MultiFileErrorCollectorImpl mfe;
+  google::protobuf::compiler::Importer importer(&dst, &mfe);
+  const google::protobuf::FileDescriptor* root_file =
+      importer.Import(proto_file);
+  const google::protobuf::Descriptor* root_msg = nullptr;
+  if (!root_message.empty()) {
+    root_msg = importer.pool()->FindMessageTypeByName(root_message);
+  } else if (root_file->message_type_count() > 0) {
+    // The user didn't specfy the root type. Pick the first type in the file,
+    // most times it's the right guess.
+    root_msg = root_file->message_type(0);
+    if (root_msg)
+      PERFETTO_LOG(
+          "The guessed root message name is \"%s\". Pass -r com.MyName to "
+          "override",
+          root_msg->full_name().c_str());
+  }
+
+  if (!root_msg) {
+    PERFETTO_ELOG("Could not find the root message \"%s\" in %s",
+                  root_message.c_str(), proto_file.c_str());
+    return false;
+  }
+
+  // |descriptors_by_full_name| is passed by argument rather than being a member
+  // field so that we don't risk leaving it out of sync (and depending on it in
+  // future without realizing) when performing the Dedupe() pass.
+  DescriptorsByNameMap descriptors_by_full_name;
+  ParseProtoDescriptor(root_msg, &descriptors_by_full_name);
+  return true;
+}
+
+// Generates a Message object for the given libprotobuf message descriptor.
+// Recurses as needed into nested fields.
+FilterUtil::Message* FilterUtil::ParseProtoDescriptor(
+    const google::protobuf::Descriptor* proto,
+    DescriptorsByNameMap* descriptors_by_full_name) {
+  auto descr_it = descriptors_by_full_name->find(proto->full_name());
+  if (descr_it != descriptors_by_full_name->end())
+    return descr_it->second;
+
+  descriptors_.emplace_back();
+  Message* msg = &descriptors_.back();
+  msg->full_name = proto->full_name();
+  (*descriptors_by_full_name)[msg->full_name] = msg;
+  for (int i = 0; i < proto->field_count(); ++i) {
+    const auto* proto_field = proto->field(i);
+    const uint32_t field_id = static_cast<uint32_t>(proto_field->number());
+    PERFETTO_CHECK(msg->fields.count(field_id) == 0);
+    auto& field = msg->fields[field_id];
+    field.name = proto_field->name();
+    field.type = proto_field->type_name();
+    if (proto_field->message_type()) {
+      msg->has_nested_fields = true;
+      // Recurse.
+      field.nested_type = ParseProtoDescriptor(proto_field->message_type(),
+                                               descriptors_by_full_name);
+    }
+  }
+  return msg;
+}
+
+void FilterUtil::Dedupe() {
+  std::map<std::string /*identity*/, Message*> index;
+
+  std::map<Message*, Message*> dupe_graph;  // K,V: K shall be duped against V.
+
+  // As a first pass, generate an |identity| string for each leaf message. The
+  // identity is simply the comma-separated stringification of its field ids.
+  // If another message with the same identity exists, add an edge to the graph.
+  const size_t initial_count = descriptors_.size();
+  size_t field_count = 0;
+  for (auto& descr : descriptors_) {
+    if (descr.has_nested_fields)
+      continue;  // Dedupe only leaf messages without nested fields.
+    std::string identity;
+    for (const auto& id_and_field : descr.fields)
+      identity.append(std::to_string(id_and_field.first) + ",");
+    auto it_and_inserted = index.emplace(identity, &descr);
+    if (!it_and_inserted.second) {
+      // insertion failed, a dupe exists already.
+      Message* dupe_against = it_and_inserted.first->second;
+      dupe_graph.emplace(&descr, dupe_against);
+    }
+  }
+
+  // Now apply de-duplications by re-directing the nested_type pointer to the
+  // equivalent descriptors that have the same set of allowed field ids.
+  std::set<Message*> referenced_descriptors;
+  referenced_descriptors.emplace(&descriptors_.front());  // The root.
+  for (auto& descr : descriptors_) {
+    for (auto& id_and_field : descr.fields) {
+      Message* target = id_and_field.second.nested_type;
+      if (!target)
+        continue;  // Only try to dedupe nested types.
+      auto it = dupe_graph.find(target);
+      if (it == dupe_graph.end()) {
+        referenced_descriptors.emplace(target);
+        continue;
+      }
+      ++field_count;
+      // Replace with the dupe.
+      id_and_field.second.nested_type = it->second;
+    }  // for (nested_fields).
+  }    // for (descriptors_).
+
+  // Remove unreferenced descriptors. We should much rather crash in the case of
+  // a logic bug rathern than trying to use them but don't emit them.
+  size_t removed_count = 0;
+  for (auto it = descriptors_.begin(); it != descriptors_.end();) {
+    if (referenced_descriptors.count(&*it)) {
+      ++it;
+    } else {
+      ++removed_count;
+      it = descriptors_.erase(it);
+    }
+  }
+  PERFETTO_LOG(
+      "Deduplication removed %zu duped descriptors out of %zu descriptors from "
+      "%zu fields",
+      removed_count, initial_count, field_count);
+}
+
+// Prints the list of messages and fields in a diff-friendly text format.
+void FilterUtil::PrintAsText() {
+  using perfetto::base::StripPrefix;
+  const std::string& root_name = descriptors_.front().full_name;
+  std::string root_prefix = root_name.substr(0, root_name.rfind('.'));
+  if (!root_prefix.empty())
+    root_prefix.append(".");
+
+  for (const auto& descr : descriptors_) {
+    for (const auto& id_and_field : descr.fields) {
+      const uint32_t field_id = id_and_field.first;
+      const auto& field = id_and_field.second;
+      const Message* nested_type = id_and_field.second.nested_type;
+      auto stripped_name = StripPrefix(descr.full_name, root_prefix);
+      std::string stripped_nested =
+          nested_type ? StripPrefix(nested_type->full_name, root_prefix) : "";
+      printf("%-60s %3u %-8s %-32s %s\n", stripped_name.c_str(), field_id,
+             field.type.c_str(), field.name.c_str(), stripped_nested.c_str());
+    }
+  }
+}
+
+std::string FilterUtil::GenerateFilterBytecode() {
+  protozero::FilterBytecodeGenerator bytecode_gen;
+
+  // Assign indexes to descriptors, simply by counting them in order;
+  std::map<Message*, uint32_t> descr_to_idx;
+  for (auto& descr : descriptors_)
+    descr_to_idx[&descr] = static_cast<uint32_t>(descr_to_idx.size());
+
+  for (auto& descr : descriptors_) {
+    for (auto it = descr.fields.begin(); it != descr.fields.end();) {
+      uint32_t field_id = it->first;
+      const Message::Field& field = it->second;
+      if (field.nested_type) {
+        // Append the index of the target submessage.
+        PERFETTO_CHECK(descr_to_idx.count(field.nested_type));
+        uint32_t nested_msg_index = descr_to_idx[field.nested_type];
+        bytecode_gen.AddNestedField(field_id, nested_msg_index);
+        ++it;
+        continue;
+      }
+      // Simple field. Lookahead to see if we have a range of contiguous simple
+      // fields.
+      for (uint32_t range_len = 1;; ++range_len) {
+        ++it;
+        if (it != descr.fields.end() && it->first == field_id + range_len &&
+            it->second.is_simple()) {
+          continue;
+        }
+        // At this point it points to either the end() of the vector or a
+        // non-contiguous or non-simple field (which will be picked up by the
+        // next iteration).
+        if (range_len == 1) {
+          bytecode_gen.AddSimpleField(field_id);
+        } else {
+          bytecode_gen.AddSimpleFieldRange(field_id, range_len);
+        }
+        break;
+      }  // for (range_len)
+    }    // for (descr.fields)
+    bytecode_gen.EndMessage();
+  }  // for (descriptors)
+  return bytecode_gen.Serialize();
+}
+
+std::string FilterUtil::LookupField(const std::string& varint_encoded_path) {
+  const uint8_t* ptr =
+      reinterpret_cast<const uint8_t*>(varint_encoded_path.data());
+  const uint8_t* const end = ptr + varint_encoded_path.size();
+
+  std::vector<uint32_t> fields;
+  while (ptr < end) {
+    uint64_t varint;
+    const uint8_t* next = proto_utils::ParseVarInt(ptr, end, &varint);
+    PERFETTO_CHECK(next != ptr);
+    fields.emplace_back(static_cast<uint32_t>(varint));
+    ptr = next;
+  }
+  return LookupField(fields.data(), fields.size());
+}
+
+std::string FilterUtil::LookupField(const uint32_t* field_ids,
+                                    size_t num_fields) {
+  const Message* msg = descriptors_.empty() ? nullptr : &descriptors_.front();
+  std::string res;
+  for (size_t i = 0; i < num_fields; ++i) {
+    const uint32_t field_id = field_ids[i];
+    const Message::Field* field = nullptr;
+    if (msg) {
+      auto it = msg->fields.find(field_id);
+      field = it == msg->fields.end() ? nullptr : &it->second;
+    }
+    res.append(".");
+    if (field) {
+      res.append(field->name);
+      msg = field->nested_type;
+    } else {
+      res.append(std::to_string(field_id));
+    }
+  }
+  return res;
+}
+
+}  // namespace protozero
diff --git a/src/protozero/filtering/filter_util.h b/src/protozero/filtering/filter_util.h
new file mode 100644
index 0000000..3cc7655
--- /dev/null
+++ b/src/protozero/filtering/filter_util.h
@@ -0,0 +1,107 @@
+/*
+ * Copyright (C) 2021 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_PROTOZERO_FILTERING_FILTER_UTIL_H_
+#define SRC_PROTOZERO_FILTERING_FILTER_UTIL_H_
+
+#include <stdint.h>
+
+#include <list>
+#include <map>
+#include <string>
+
+namespace google {
+namespace protobuf {
+class Descriptor;
+}
+}  // namespace google
+
+namespace protozero {
+
+// Parses a .proto message definition, recursing into its sub-messages, and
+// builds up a set of Messages and Field definitions.
+// Depends on libprotobuf-full and should be used only in host tools.
+// See the //tools/proto_filter for an executable that wraps this class with
+// a cmdline interface.
+class FilterUtil {
+ public:
+  FilterUtil();
+  ~FilterUtil();
+
+  // Loads a message schema from a .proto file, recursing into nested types.
+  // Args:
+  // proto_file: path to the .proto file.
+  // root_message: fully qualified message name (e.g., perfetto.protos.Trace).
+  //     If empty, the first message in the file will be used.
+  // proto_dir_path: the root for .proto includes. If empty uses CWD.
+  bool LoadMessageDefinition(const std::string& proto_file,
+                             const std::string& root_message,
+                             const std::string& proto_dir_path);
+
+  // Deduplicates leaf messages having the same sets of field ids.
+  // It changes the internal state and affects the behavior of next calls to
+  // GenerateFilterBytecode() and PrintAsText().
+  void Dedupe();
+
+  // Generates the filter bytecode for the root message previously loaded by
+  // LoadMessageDefinition() using FilterBytecodeGenerator.
+  // The returned string is a binary-encoded proto message of type
+  // perfetto.protos.ProtoFilter (see proto_filter.proto).
+  std::string GenerateFilterBytecode();
+
+  // Prints the list of messages and fields onto stdout in a diff-friendly text
+  // format. Example:
+  // PowerRails                 2 message  energy_data     PowerRails.EnergyData
+  // PowerRails.RailDescriptor  1 uint32   index
+  void PrintAsText();
+
+  // Resolves an array of field ids into a dot-concatenated field names.
+  // E.g., [2,5,1] -> ".trace.packet.timestamp".
+  std::string LookupField(const uint32_t* field_ids, size_t num_fields);
+
+  // Like the above but the array of field is passed as a buffer containing
+  // varints, e.g. "\x02\x05\0x01".
+  std::string LookupField(const std::string& varint_encoded_path);
+
+ private:
+  struct Message {
+    struct Field {
+      std::string name;
+      std::string type;  // "uint32", "string", "message"
+      // Only when type == "message". Note that when using Dedupe() this can
+      // be aliased against a different submessage which happens to have the
+      // same set of field ids.
+      Message* nested_type = nullptr;
+      bool is_simple() const { return nested_type == nullptr; }
+    };
+    std::string full_name;  // e.g., "perfetto.protos.Foo.Bar";
+    std::map<uint32_t /*field_id*/, Field> fields;
+
+    // True if at least one field has a non-null |nestd_type|.
+    bool has_nested_fields = false;
+  };
+
+  using DescriptorsByNameMap = std::map<std::string, Message*>;
+  Message* ParseProtoDescriptor(const google::protobuf::Descriptor*,
+                                DescriptorsByNameMap*);
+
+  // list<> because pointers need to be stable.
+  std::list<Message> descriptors_;
+};
+
+}  // namespace protozero
+
+#endif  // SRC_PROTOZERO_FILTERING_FILTER_UTIL_H_
diff --git a/src/protozero/filtering/filter_util_unittest.cc b/src/protozero/filtering/filter_util_unittest.cc
new file mode 100644
index 0000000..9c7cf0d
--- /dev/null
+++ b/src/protozero/filtering/filter_util_unittest.cc
@@ -0,0 +1,182 @@
+/*
+ * Copyright (C) 2021 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 "test/gtest_and_gmock.h"
+
+#include "perfetto/ext/base/file_utils.h"
+#include "perfetto/ext/base/temp_file.h"
+#include "src/protozero/filtering/filter_bytecode_parser.h"
+#include "src/protozero/filtering/filter_util.h"
+
+namespace protozero {
+
+namespace {
+
+perfetto::base::TempFile MkTemp(const char* str) {
+  auto tmp = perfetto::base::TempFile::Create();
+  perfetto::base::WriteAll(*tmp, str, strlen(str));
+  perfetto::base::FlushFile(*tmp);
+  return tmp;
+}
+
+TEST(SchemaParserTest, SchemaToBytecode_Simple) {
+  auto schema = MkTemp(R"(
+  syntax = "proto2";
+  message Root {
+    optional int32 i32 = 13;
+    optional fixed64 f64 = 5;
+    optional string str = 71;
+  }
+  )");
+  FilterUtil filter;
+  ASSERT_TRUE(filter.LoadMessageDefinition(schema.path(), "Root", ""));
+  std::string bytecode = filter.GenerateFilterBytecode();
+  FilterBytecodeParser fbp;
+  ASSERT_TRUE(fbp.Load(bytecode.data(), bytecode.size()));
+  EXPECT_TRUE(fbp.Query(0, 13).allowed);
+  EXPECT_TRUE(fbp.Query(0, 13).simple_field());
+  EXPECT_TRUE(fbp.Query(0, 5).allowed);
+  EXPECT_TRUE(fbp.Query(0, 5).simple_field());
+  EXPECT_TRUE(fbp.Query(0, 71).allowed);
+  EXPECT_TRUE(fbp.Query(0, 71).simple_field());
+  EXPECT_FALSE(fbp.Query(0, 1).allowed);
+  EXPECT_FALSE(fbp.Query(0, 12).allowed);
+  EXPECT_FALSE(fbp.Query(0, 70).allowed);
+}
+
+TEST(SchemaParserTest, SchemaToBytecode_Nested) {
+  auto schema = MkTemp(R"(
+  syntax = "proto2";
+  message Root {
+    message Child {
+      repeated fixed64 f64 = 3;
+      optional Child recurse = 4;
+    }
+    oneof xxx { int32 i32 = 1; }
+    optional Child chld = 2;
+  }
+  )");
+  FilterUtil filter;
+  ASSERT_TRUE(filter.LoadMessageDefinition(schema.path(), "", ""));
+  std::string bytecode = filter.GenerateFilterBytecode();
+  FilterBytecodeParser fbp;
+  ASSERT_TRUE(fbp.Load(bytecode.data(), bytecode.size()));
+  EXPECT_TRUE(fbp.Query(0, 1).allowed);
+  EXPECT_TRUE(fbp.Query(0, 1).simple_field());
+  EXPECT_TRUE(fbp.Query(0, 2).allowed);
+  EXPECT_FALSE(fbp.Query(0, 2).simple_field());
+  // False as those fields exist only in Child, not in the root (0).
+  EXPECT_FALSE(fbp.Query(0, 3).allowed);
+  EXPECT_FALSE(fbp.Query(0, 4).allowed);
+
+  EXPECT_TRUE(fbp.Query(1, 3).allowed);
+  EXPECT_TRUE(fbp.Query(1, 3).simple_field());
+  EXPECT_TRUE(fbp.Query(1, 4).allowed);
+  EXPECT_FALSE(fbp.Query(1, 4).simple_field());
+  EXPECT_EQ(fbp.Query(1, 4).nested_msg_index, 1u);  // Self
+}
+
+TEST(SchemaParserTest, SchemaToBytecode_Dedupe) {
+  auto schema = MkTemp(R"(
+  syntax = "proto2";
+  message Root {
+    message Nested {
+      message Child1 {
+        optional int32 f1 = 3;
+        optional int64 f2 = 4;
+      }
+      message Child2 {
+        optional string f1 = 3;
+        optional bytes f2 = 4;
+      }
+      message ChildNonDedupe {
+        optional string f1 = 3;
+        optional bytes f2 = 4;
+        optional int32 extra = 1;
+      }
+      optional Child1 chld1 = 1;
+      optional Child2 chld2 = 2;
+      optional ChildNonDedupe chld3 = 3;
+    }
+    repeated Nested nested = 1;
+  }
+  )");
+  FilterUtil filter;
+  ASSERT_TRUE(filter.LoadMessageDefinition(schema.path(), "Root", ""));
+  filter.Dedupe();
+  std::string bytecode = filter.GenerateFilterBytecode();
+  FilterBytecodeParser fbp;
+  ASSERT_TRUE(fbp.Load(bytecode.data(), bytecode.size()));
+
+  // 0: Root
+  EXPECT_TRUE(fbp.Query(0, 1).allowed);
+  EXPECT_FALSE(fbp.Query(0, 1).simple_field());
+
+  // 1: Nested
+  EXPECT_TRUE(fbp.Query(1, 1).allowed);
+  EXPECT_FALSE(fbp.Query(1, 1).simple_field());
+  EXPECT_TRUE(fbp.Query(1, 2).allowed);
+  EXPECT_FALSE(fbp.Query(1, 2).simple_field());
+  EXPECT_TRUE(fbp.Query(1, 3).allowed);
+  EXPECT_FALSE(fbp.Query(1, 3).simple_field());
+
+  // Check deduping.
+  // Fields chld1 and chld2 should point to the same sub-filter because they
+  // have the same field ids.
+  EXPECT_EQ(fbp.Query(1, 1).nested_msg_index, fbp.Query(1, 2).nested_msg_index);
+
+  // Field chld3 should point to a different one because it has an extra field.
+  EXPECT_NE(fbp.Query(1, 1).nested_msg_index, fbp.Query(1, 3).nested_msg_index);
+}
+
+TEST(SchemaParserTest, FieldLookup) {
+  auto schema = MkTemp(R"(
+  syntax = "proto2";
+  message Root {
+    message Nested {
+      message Child1 {
+        optional int32 f1 = 3;
+        optional int64 f2 = 4;
+        repeated Child2 c2 = 5;
+      }
+      message Child2 {
+        optional string f3 = 6;
+        optional bytes f4 = 7;
+        repeated Child1 c1 = 8;
+      }
+      optional Child1 x1 = 1;
+      optional Child2 x2 = 2;
+    }
+    repeated Nested n = 1;
+  }
+  )");
+
+  FilterUtil filter;
+  ASSERT_TRUE(filter.LoadMessageDefinition(schema.path(), "Root", ""));
+  std::vector<uint32_t> fld;
+
+  fld = {1, 1, 3};
+  ASSERT_EQ(filter.LookupField(fld.data(), fld.size()), ".n.x1.f1");
+
+  fld = {1, 2, 7};
+  ASSERT_EQ(filter.LookupField(fld.data(), fld.size()), ".n.x2.f4");
+
+  fld = {1, 2, 8, 5, 8, 5, 7};
+  ASSERT_EQ(filter.LookupField(fld.data(), fld.size()), ".n.x2.c1.c2.c1.c2.f4");
+}
+
+}  // namespace
+}  // namespace protozero
diff --git a/src/protozero/filtering/message_filter.cc b/src/protozero/filtering/message_filter.cc
new file mode 100644
index 0000000..a971dfa
--- /dev/null
+++ b/src/protozero/filtering/message_filter.cc
@@ -0,0 +1,310 @@
+/*
+ * Copyright (C) 2021 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/protozero/filtering/message_filter.h"
+
+#include "perfetto/base/logging.h"
+#include "perfetto/protozero/proto_utils.h"
+
+namespace protozero {
+
+namespace {
+
+// Inline helpers to append proto fields in output. They are the equivalent of
+// the protozero::Message::AppendXXX() fields but don't require building and
+// maintaining a full protozero::Message object or dealing with scattered
+// output slices.
+// All these functions assume there is enough space in the output buffer, which
+// should be always the case assuming that we don't end up generating more
+// output than input.
+
+inline void AppendVarInt(uint32_t field_id, uint64_t value, uint8_t** out) {
+  *out = proto_utils::WriteVarInt(proto_utils::MakeTagVarInt(field_id), *out);
+  *out = proto_utils::WriteVarInt(value, *out);
+}
+
+// For fixed32 / fixed64.
+template <typename INT_T /* uint32_t | uint64_t*/>
+inline void AppendFixed(uint32_t field_id, INT_T value, uint8_t** out) {
+  *out = proto_utils::WriteVarInt(proto_utils::MakeTagFixed<INT_T>(field_id),
+                                  *out);
+  memcpy(*out, &value, sizeof(value));
+  *out += sizeof(value);
+}
+
+// For length-delimited (string, bytes) fields. Note: this function appends only
+// the proto preamble and the varint field that states the length of the payload
+// not the payload itself.
+// In the case of submessages, the caller needs to re-write the length at the
+// end in the in the returned memory area.
+// The problem here is that, because of filtering, the length of a submessage
+// might be < original length (the original length is still an upper-bound).
+// Returns a pair with: (1) the pointer where the final length should be written
+// into, (2) the length of the size field.
+// The caller must write a redundant varint to match the original size (i.e.
+// needs to use WriteRedundantVarInt()).
+inline std::pair<uint8_t*, uint32_t> AppendLenDelim(uint32_t field_id,
+                                                    uint32_t len,
+                                                    uint8_t** out) {
+  *out = proto_utils::WriteVarInt(proto_utils::MakeTagLengthDelimited(field_id),
+                                  *out);
+  uint8_t* size_field_start = *out;
+  *out = proto_utils::WriteVarInt(len, *out);
+  const size_t size_field_len = static_cast<size_t>(*out - size_field_start);
+  return std::make_pair(size_field_start, size_field_len);
+}
+}  // namespace
+
+MessageFilter::MessageFilter() {
+  // Push a state on the stack for the implicit root message.
+  stack_.emplace_back();
+}
+
+MessageFilter::~MessageFilter() = default;
+
+bool MessageFilter::LoadFilterBytecode(const void* filter_data, size_t len) {
+  return filter_.Load(filter_data, len);
+}
+
+bool MessageFilter::SetFilterRoot(const uint32_t* field_ids,
+                                  size_t num_fields) {
+  uint32_t root_msg_idx = 0;
+  for (const uint32_t* it = field_ids; it < field_ids + num_fields; ++it) {
+    uint32_t field_id = *it;
+    auto res = filter_.Query(root_msg_idx, field_id);
+    if (!res.allowed || res.simple_field())
+      return false;
+    root_msg_idx = res.nested_msg_index;
+  }
+  root_msg_index_ = root_msg_idx;
+  return true;
+}
+
+MessageFilter::FilteredMessage MessageFilter::FilterMessageFragments(
+    const InputSlice* slices,
+    size_t num_slices) {
+  // First compute the upper bound for the output. The filtered message cannot
+  // be > the original message.
+  uint32_t total_len = 0;
+  for (size_t i = 0; i < num_slices; ++i)
+    total_len += slices[i].len;
+  out_buf_.reset(new uint8_t[total_len]);
+  out_ = out_buf_.get();
+  out_end_ = out_ + total_len;
+
+  // Reset the parser state.
+  tokenizer_ = MessageTokenizer();
+  error_ = false;
+  stack_.clear();
+  stack_.resize(2);
+  // stack_[0] is a sentinel and should never be hit in nominal cases. If we
+  // end up there we will just keep consuming the input stream and detecting
+  // at the end, without hurting the fastpath.
+  stack_[0].in_bytes_limit = UINT32_MAX;
+  stack_[0].eat_next_bytes = UINT32_MAX;
+  // stack_[1] is the actual root message.
+  stack_[1].in_bytes_limit = total_len;
+  stack_[1].msg_index = root_msg_index_;
+
+  // Process the input data and write the output.
+  for (size_t slice_idx = 0; slice_idx < num_slices; ++slice_idx) {
+    const InputSlice& slice = slices[slice_idx];
+    const uint8_t* data = static_cast<const uint8_t*>(slice.data);
+    for (size_t i = 0; i < slice.len; ++i)
+      FilterOneByte(data[i]);
+  }
+
+  // Construct the output object.
+  PERFETTO_CHECK(out_ >= out_buf_.get() && out_ <= out_end_);
+  auto used_size = static_cast<size_t>(out_ - out_buf_.get());
+  FilteredMessage res{std::move(out_buf_), used_size};
+  res.error = error_;
+  if (stack_.size() != 1 || !tokenizer_.idle() ||
+      stack_[0].in_bytes != total_len) {
+    res.error = true;
+  }
+  return res;
+}
+
+void MessageFilter::FilterOneByte(uint8_t octet) {
+  PERFETTO_DCHECK(!stack_.empty());
+
+  auto* state = &stack_.back();
+  StackState next_state{};
+  bool push_next_state = false;
+
+  if (state->eat_next_bytes > 0) {
+    // This is the case where the previous tokenizer_.Push() call returned a
+    // length delimited message which is NOT a submessage (a string or a bytes
+    // field). We just want to consume it, and pass it through in output
+    // if the field was allowed.
+    --state->eat_next_bytes;
+    if (state->passthrough_eaten_bytes)
+      *(out_++) = octet;
+  } else {
+    MessageTokenizer::Token token = tokenizer_.Push(octet);
+    // |token| will not be valid() in most cases and this is WAI. When pushing
+    // a varint field, only the last byte yields a token, all the other bytes
+    // return an invalid token, they just update the internal tokenizer state.
+    if (token.valid()) {
+      auto filter = filter_.Query(state->msg_index, token.field_id);
+      switch (token.type) {
+        case proto_utils::ProtoWireType::kVarInt:
+          if (filter.allowed && filter.simple_field())
+            AppendVarInt(token.field_id, token.value, &out_);
+          break;
+        case proto_utils::ProtoWireType::kFixed32:
+          if (filter.allowed && filter.simple_field())
+            AppendFixed(token.field_id, static_cast<uint32_t>(token.value),
+                        &out_);
+          break;
+        case proto_utils::ProtoWireType::kFixed64:
+          if (filter.allowed && filter.simple_field())
+            AppendFixed(token.field_id, static_cast<uint64_t>(token.value),
+                        &out_);
+          break;
+        case proto_utils::ProtoWireType::kLengthDelimited:
+          // Here we have two cases:
+          // A. A simple string/bytes field: we just want to consume the next
+          //    bytes (the string payload), optionally passing them through in
+          //    output if the field is allowed.
+          // B. This is a nested submessage. In this case we want to recurse and
+          //    push a new state on the stack.
+          // Note that we can't tell the difference between a
+          // "non-allowed string" and a "non-allowed submessage". But it doesn't
+          // matter because in both cases we just want to skip the next N bytes.
+          const auto submessage_len = static_cast<uint32_t>(token.value);
+          auto in_bytes_left = state->in_bytes_limit - state->in_bytes - 1;
+          if (PERFETTO_UNLIKELY(submessage_len > in_bytes_left)) {
+            // This is a malicious / malformed string/bytes/submessage that
+            // claims to be larger than the outer message that contains it.
+            return SetUnrecoverableErrorState();
+          }
+
+          if (filter.allowed && !filter.simple_field() && submessage_len > 0) {
+            // submessage_len == 0 is the edge case of a message with a 0-len
+            // (but present) submessage. In this case, if allowed, we don't want
+            // to push any further state (doing so would desync the FSM) but we
+            // still want to emit it.
+            // At this point |submessage_len| is only an upper bound. The
+            // final message written in output can be <= the one in input,
+            // only some of its fields might be allowed (also remember that
+            // this class implicitly removes redundancy varint encoding of
+            // len-delimited field lengths). The final length varint (the
+            // return value of AppendLenDelim()) will be filled when popping
+            // from |stack_|.
+            auto size_field =
+                AppendLenDelim(token.field_id, submessage_len, &out_);
+            push_next_state = true;
+            next_state.field_id = token.field_id;
+            next_state.msg_index = filter.nested_msg_index;
+            next_state.in_bytes_limit = submessage_len;
+            next_state.size_field = size_field.first;
+            next_state.size_field_len = size_field.second;
+            next_state.out_bytes_written_at_start = out_written();
+          } else {
+            // A string or bytes field, or a 0 length submessage.
+            state->eat_next_bytes = submessage_len;
+            state->passthrough_eaten_bytes = filter.allowed;
+            if (filter.allowed)
+              AppendLenDelim(token.field_id, submessage_len, &out_);
+          }
+          break;
+      }  // switch(type)
+
+      if (PERFETTO_UNLIKELY(track_field_usage_)) {
+        IncrementCurrentFieldUsage(token.field_id, filter.allowed);
+      }
+    }  // if (token.valid)
+  }    // if (eat_next_bytes == 0)
+
+  ++state->in_bytes;
+  while (state->in_bytes >= state->in_bytes_limit) {
+    PERFETTO_DCHECK(state->in_bytes == state->in_bytes_limit);
+    push_next_state = false;
+
+    // We can't possibly write more than we read.
+    const uint32_t msg_bytes_written = static_cast<uint32_t>(
+        out_written() - state->out_bytes_written_at_start);
+    PERFETTO_DCHECK(msg_bytes_written <= state->in_bytes_limit);
+
+    // Backfill the length field of the
+    proto_utils::WriteRedundantVarInt(msg_bytes_written, state->size_field,
+                                      state->size_field_len);
+
+    const uint32_t in_bytes_processes_for_last_msg = state->in_bytes;
+    stack_.pop_back();
+    PERFETTO_CHECK(!stack_.empty());
+    state = &stack_.back();
+    state->in_bytes += in_bytes_processes_for_last_msg;
+    if (PERFETTO_UNLIKELY(!tokenizer_.idle())) {
+      // If we hit this case, it means that we got to the end of a submessage
+      // while decoding a field. We can't recover from this and we don't want to
+      // propagate a broken sub-message.
+      return SetUnrecoverableErrorState();
+    }
+  }
+
+  if (push_next_state) {
+    PERFETTO_DCHECK(tokenizer_.idle());
+    stack_.emplace_back(std::move(next_state));
+    state = &stack_.back();
+  }
+}
+
+void MessageFilter::SetUnrecoverableErrorState() {
+  error_ = true;
+  stack_.clear();
+  stack_.resize(1);
+  auto& state = stack_[0];
+  state.eat_next_bytes = UINT32_MAX;
+  state.in_bytes_limit = UINT32_MAX;
+  state.passthrough_eaten_bytes = false;
+  out_ = out_buf_.get();  // Reset the write pointer.
+}
+
+void MessageFilter::IncrementCurrentFieldUsage(uint32_t field_id,
+                                               bool allowed) {
+  // Slowpath. Used mainly in offline tools and tests to workout used fields in
+  // a proto.
+  PERFETTO_DCHECK(track_field_usage_);
+
+  // Field path contains a concatenation of varints, one for each nesting level.
+  // e.g. y in message Root { Sub x = 2; }; message Sub { SubSub y = 7; }
+  // is encoded as [varint(2) + varint(7)].
+  // We use varint to take the most out of SSO (small string opt). In most cases
+  // the path will fit in the on-stack 22 bytes, requiring no heap.
+  std::string field_path;
+
+  auto append_field_id = [&field_path](uint32_t id) {
+    uint8_t buf[10];
+    uint8_t* end = proto_utils::WriteVarInt(id, buf);
+    field_path.append(reinterpret_cast<char*>(buf),
+                      static_cast<size_t>(end - buf));
+  };
+
+  // Append all the ancestors IDs from the state stack.
+  // The first entry of the stack has always ID 0 and we skip it (we don't know
+  // the ID of the root message itself).
+  PERFETTO_DCHECK(stack_.size() >= 2 && stack_[1].field_id == 0);
+  for (size_t i = 2; i < stack_.size(); ++i)
+    append_field_id(stack_[i].field_id);
+  // Append the id of the field in the current message.
+  append_field_id(field_id);
+  field_usage_[field_path] += allowed ? 1 : -1;
+}
+
+}  // namespace protozero
diff --git a/src/protozero/filtering/message_filter.h b/src/protozero/filtering/message_filter.h
new file mode 100644
index 0000000..37dc9b5
--- /dev/null
+++ b/src/protozero/filtering/message_filter.h
@@ -0,0 +1,208 @@
+/*
+ * Copyright (C) 2021 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_PROTOZERO_FILTERING_MESSAGE_FILTER_H_
+#define SRC_PROTOZERO_FILTERING_MESSAGE_FILTER_H_
+
+#include <stdint.h>
+
+#include <memory>
+#include <string>
+#include <unordered_map>
+
+#include "src/protozero/filtering/filter_bytecode_parser.h"
+#include "src/protozero/filtering/message_tokenizer.h"
+
+namespace protozero {
+
+// A class to filter binary-encoded proto messages using an allow-list of field
+// ids, also known as "filter bytecode". The filter determines which fields are
+// allowed to be passed through in output and strips all the other fields.
+// See go/trace-filtering for full design.
+// This class takes in input:
+// 1) The filter bytecode, loaded once via the LoadFilterBytecode() method.
+// 2) A proto-encoded binary message. The message doesn't have to be contiguous,
+//    it can be passed as an array of arbitrarily chunked fragments.
+// The FilterMessage*() method returns in output a proto message, stripping out
+// all unknown fields. If the input is malformed (e.g., unknown proto field wire
+// types, lengths out of bound) the whole filtering failed and the |error| flag
+// of the FilteredMessage object is set to true.
+// The filtering operation is based on rewriting a copy of the message into a
+// self-allocated buffer, which is then returned in the output. The input buffer
+// is NOT altered.
+// Note also that the process of rewriting the protos gets rid of most redundant
+// varint encoding (if present). So even if all fields are allow-listed, the
+// output might NOT be bitwise identical to the input (but it will be
+// semantically equivalent).
+// Furthermore the enable_field_usage_tracking() method allows to keep track of
+// a histogram of allowed / denied fields. It slows down filtering and is
+// intended only on host tools.
+class MessageFilter {
+ public:
+  MessageFilter();
+  ~MessageFilter();
+
+  struct InputSlice {
+    const void* data;
+    size_t len;
+  };
+
+  struct FilteredMessage {
+    FilteredMessage(std::unique_ptr<uint8_t[]> d, size_t s)
+        : data(std::move(d)), size(s) {}
+    std::unique_ptr<uint8_t[]> data;
+    size_t size;  // The used bytes in |data|. This is <= sizeof(data).
+    bool error = false;
+  };
+
+  // Loads the filter bytecode that will be used to filter any subsequent
+  // message. Must be called before the first call to FilterMessage*().
+  // |filter_data| must point to a byte buffer for a proto-encoded ProtoFilter
+  // message (see proto_filter.proto).
+  bool LoadFilterBytecode(const void* filter_data, size_t len);
+
+  // This affects the filter starting point of the subsequent FilterMessage*()
+  // calls. By default the filtering process starts from the message @ index 0,
+  // the root message passed to proto_filter when generating the bytecode
+  // (in typical tracing use-cases, this is perfetto.protos.Trace). However, the
+  // caller (TracingServiceImpl) might want to filter packets from the 2nd level
+  // (perfetto.protos.TracePacket) because the root level is pre-pended after
+  // the fact. This call allows to change the root message for the filter.
+  // The argument |field_ids| is an array of proto field ids and determines the
+  // path to the new root. For instance, in the case of [1,2,3] SetFilterRoot
+  // will identify the sub-message for the field "root.1.2.3" and use that.
+  // In order for this to succeed all the fields in the path must be allowed
+  // in the filter and must be a nested message type.
+  bool SetFilterRoot(const uint32_t* field_ids, size_t num_fields);
+
+  // Takes an input message, fragmented in arbitrary slices, and returns a
+  // filtered message in output.
+  FilteredMessage FilterMessageFragments(const InputSlice*, size_t num_slices);
+
+  // Helper for tests, where the input is a contiguous buffer.
+  FilteredMessage FilterMessage(const void* data, size_t len) {
+    InputSlice slice{data, len};
+    return FilterMessageFragments(&slice, 1);
+  }
+
+  // When enabled returns a map of "field path" to "usage counter".
+  // The key (std::string) is a binary buffer (i.e. NOT an ASCII/UTF-8 string)
+  // which contains a varint for each field. Consider the following:
+  // message Root { Sub1 f1 = 1; };
+  // message Sub1 { Sub2 f2 = 7;}
+  // message Sub2 { string f3 = 5; }
+  // The field .f1.f2.f3 will be encoded as \x01\0x07\x05.
+  // The value is the number of times that field has been encountered. If the
+  // field is not allow-listed in the bytecode (the field is stripped in output)
+  // the count will be negative.
+  void enable_field_usage_tracking(bool x) { track_field_usage_ = x; }
+  const std::unordered_map<std::string, int32_t>& field_usage() const {
+    return field_usage_;
+  }
+
+  // Exposed only for DCHECKS in TracingServiceImpl.
+  uint32_t root_msg_index() { return root_msg_index_; }
+
+ private:
+  // This is called by FilterMessageFragments().
+  // Inlining allows the compiler turn the per-byte call/return into a for loop,
+  // while, at the same time, keeping the code easy to read and reason about.
+  // It gives a 20-25% speedup (265ms vs 215ms for a 25MB trace).
+  void FilterOneByte(uint8_t octet) PERFETTO_ALWAYS_INLINE;
+
+  // No-inline because this is a slowpath (only when usage tracking is enabled).
+  void IncrementCurrentFieldUsage(uint32_t field_id,
+                                  bool allowed) PERFETTO_NO_INLINE;
+
+  // Gets into an error state which swallows all the input and emits no output.
+  void SetUnrecoverableErrorState();
+
+  // We keep track of the the nest of messages in a stack. Each StackState
+  // object corresponds to a level of nesting in the proto message structure.
+  // Every time a new field of type len-delimited that has a corresponding
+  // sub-message in the bytecode is encountered, a new StackState is pushed in
+  // |stack_|. stack_[0] is a sentinel to prevent over-popping without adding
+  // extra branches in the fastpath.
+  // |stack_|. stack_[1] is the state of the root message.
+  struct StackState {
+    uint32_t in_bytes = 0;  // Number of input bytes processed.
+
+    // When |in_bytes| reaches this value, the current state should be popped.
+    // This is set when recursing into nested submessages. This is 0 only for
+    // stack_[0] (we don't know the size of the root message upfront).
+    uint32_t in_bytes_limit = 0;
+
+    // This is set when a len-delimited message is encountered, either a string
+    // or a nested submessage that is NOT allow-listed in the bytecode.
+    // This causes input bytes to be consumed without being parsed from the
+    // input stream. If |passthrough_eaten_bytes| == true, they will be copied
+    // as-is in output (e.g. in the case of an allowed string/bytes field).
+    uint32_t eat_next_bytes = 0;
+
+    // Keeps tracks of the stream_writer output counter (out_.written()) then
+    // the StackState is pushed. This is used to work out, when popping, how
+    // many bytes have been written for the current submessage.
+    uint32_t out_bytes_written_at_start = 0;
+
+    uint32_t field_id = 0;   // The proto field id for the current message.
+    uint32_t msg_index = 0;  // The index of the message filter in the bytecode.
+
+    // This is a pointer to the proto preamble for the current submessage
+    // (it's nullptr for stack_[0] and non-null elsewhere). This will be filled
+    // with the actual size of the message (out_.written() -
+    // |out_bytes_written_at_start|) when finishing (popping) the message.
+    // This must be filled using WriteRedundantVarint(). Note that the
+    // |size_field_len| is variable and depends on the actual length of the
+    // input message. If the output message has roughly the same size of the
+    // input message, the length will not be redundant.
+    // In other words: the length of the field is reserved when the submessage
+    // starts. At that point we know the upper-bound for the output message
+    // (a filtered submessage can be <= the original one, but not >). So we
+    // reserve as many bytes it takes to write the input length in varint.
+    // Then, when the message is finalized and we know the actual output size
+    // we backfill the field.
+    // Consider the example of a submessage where the input size = 130 (>127,
+    // 2 varint bytes) and the output is 120 bytes. The length will be 2 bytes
+    // wide even though could have been encoded with just one byte.
+    uint8_t* size_field = nullptr;
+    uint32_t size_field_len = 0;
+
+    // When true the next |eat_next_bytes| are copied as-is in output.
+    // It seems that keeping this field at the end rather than next to
+    // |eat_next_bytes| makes the filter a little (but measurably) faster.
+    // (likely something related with struct layout vs cache sizes).
+    bool passthrough_eaten_bytes = false;
+  };
+
+  uint32_t out_written() { return static_cast<uint32_t>(out_ - &out_buf_[0]); }
+
+  std::unique_ptr<uint8_t[]> out_buf_;
+  uint8_t* out_ = nullptr;
+  uint8_t* out_end_ = nullptr;
+  uint32_t root_msg_index_ = 0;
+
+  FilterBytecodeParser filter_;
+  MessageTokenizer tokenizer_;
+  std::vector<StackState> stack_;
+
+  bool error_ = false;
+  bool track_field_usage_ = false;
+  std::unordered_map<std::string, int32_t> field_usage_;
+};
+
+}  // namespace protozero
+
+#endif  // SRC_PROTOZERO_FILTERING_MESSAGE_FILTER_H_
diff --git a/src/protozero/filtering/message_filter_benchmark.cc b/src/protozero/filtering/message_filter_benchmark.cc
new file mode 100644
index 0000000..c800c82
--- /dev/null
+++ b/src/protozero/filtering/message_filter_benchmark.cc
@@ -0,0 +1,48 @@
+// Copyright (C) 2021 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 <benchmark/benchmark.h>
+
+#include <algorithm>
+#include <string>
+
+#include "perfetto/ext/base/file_utils.h"
+#include "src/base/test/utils.h"
+#include "src/protozero/filtering/message_filter.h"
+
+static void BM_ProtozeroMessageFilter(benchmark::State& state) {
+  std::string trace_data;
+  static const char kTestTrace[] = "test/data/example_android_trace_30s.pb";
+  perfetto::base::ReadFile(perfetto::base::GetTestDataPath(kTestTrace),
+                           &trace_data);
+  PERFETTO_CHECK(!trace_data.empty());
+
+  std::string filter;
+  static const char kFullTraceFilter[] = "test/data/full_trace_filter.bytecode";
+  perfetto::base::ReadFile(kFullTraceFilter, &filter);
+  PERFETTO_CHECK(!filter.empty());
+
+  protozero::MessageFilter filt;
+  filt.LoadFilterBytecode(filter.data(), filter.size());
+
+  for (auto _ : state) {
+    auto res = filt.FilterMessage(trace_data.data(), trace_data.size());
+    benchmark::DoNotOptimize(res);
+    benchmark::ClobberMemory();
+  }
+  state.SetBytesProcessed(
+      static_cast<int64_t>(state.iterations() * trace_data.size()));
+}
+
+BENCHMARK(BM_ProtozeroMessageFilter);
diff --git a/src/protozero/filtering/message_filter_fuzzer.cc b/src/protozero/filtering/message_filter_fuzzer.cc
new file mode 100644
index 0000000..60cfe5c
--- /dev/null
+++ b/src/protozero/filtering/message_filter_fuzzer.cc
@@ -0,0 +1,104 @@
+/*
+ * Copyright (C) 2021 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 <stddef.h>
+#include <stdint.h>
+#include <string.h>
+
+#include "perfetto/base/logging.h"
+#include "src/protozero/filtering/message_filter.h"
+
+namespace protozero {
+namespace {
+
+// A valid filter bytecode obtained from a perfetto.protos.Trace message.
+uint8_t kValidFilter[] = {
+    0x0b, 0x01, 0x12, 0x04, 0x00, 0x0b, 0x02, 0x13, 0x0f, 0x19, 0x23, 0x13,
+    0x33, 0x14, 0x3b, 0x15, 0x41, 0x4b, 0x11, 0x51, 0x5b, 0x16, 0x63, 0x3b,
+    0x69, 0x8b, 0x02, 0x21, 0x93, 0x02, 0x2a, 0x9b, 0x02, 0x2c, 0xab, 0x02,
+    0x2e, 0xb3, 0x02, 0x09, 0xc3, 0x02, 0x30, 0xca, 0x02, 0x02, 0xdb, 0x02,
+    0x31, 0xe3, 0x02, 0x26, 0xeb, 0x02, 0x32, 0xf3, 0x02, 0x07, 0xfb, 0x02,
+    0x33, 0x8b, 0x03, 0x0a, 0x9b, 0x03, 0x34, 0xb3, 0x03, 0x06, 0xc3, 0x03,
+    0x37, 0xd1, 0x03, 0xdb, 0x03, 0x3c, 0xe3, 0x03, 0x39, 0x93, 0x04, 0x38,
+    0x9b, 0x04, 0x3a, 0x00, 0x09, 0x13, 0x03, 0x19, 0x00, 0x0a, 0x02, 0x1b,
+    0x04, 0x23, 0x05, 0x5b, 0x06, 0x6b, 0x06, 0x83, 0x01, 0x07, 0x8b, 0x01,
+    0x08, 0x93, 0x01, 0x07, 0x9b, 0x01, 0x07, 0xa3, 0x01, 0x08, 0x9b, 0x02,
+    0x08, 0xcb, 0x02, 0x08, 0xd3, 0x02, 0x08, 0xdb, 0x02, 0x09, 0xe3, 0x02,
+    0x07, 0xeb, 0x02, 0x0a, 0xf3, 0x02, 0x07, 0xfb, 0x02, 0x0b, 0x83, 0x03,
+    0x06, 0x8b, 0x03, 0x0b, 0x93, 0x03, 0x05, 0x9b, 0x03, 0x0b, 0xab, 0x03,
+    0x0c, 0xb3, 0x03, 0x0c, 0xbb, 0x03, 0x0c, 0x9b, 0x04, 0x0d, 0xa3, 0x04,
+    0x07, 0xab, 0x04, 0x06, 0xb3, 0x04, 0x07, 0xc3, 0x04, 0x06, 0xcb, 0x04,
+    0x07, 0xd3, 0x04, 0x07, 0xdb, 0x04, 0x06, 0x93, 0x07, 0x08, 0xeb, 0x07,
+    0x08, 0xcb, 0x09, 0x07, 0xd3, 0x09, 0x05, 0xf3, 0x0b, 0x06, 0xdb, 0x0e,
+    0x09, 0xe3, 0x0e, 0x09, 0xeb, 0x0e, 0x07, 0xf3, 0x0e, 0x09, 0xfb, 0x0e,
+    0x09, 0x83, 0x0f, 0x07, 0x8b, 0x0f, 0x06, 0x93, 0x0f, 0x07, 0xb3, 0x0f,
+    0x0a, 0xc3, 0x0f, 0x0e, 0xfb, 0x0f, 0x0e, 0x83, 0x10, 0x08, 0xfb, 0x10,
+    0x08, 0x8b, 0x11, 0x08, 0xcb, 0x13, 0x06, 0xd3, 0x13, 0x07, 0xdb, 0x13,
+    0x07, 0xb3, 0x14, 0x07, 0x00, 0x11, 0x00, 0x0a, 0x07, 0x00, 0x0a, 0x02,
+    0x00, 0x0a, 0x03, 0x00, 0x0a, 0x05, 0x00, 0x0a, 0x04, 0x00, 0x0a, 0x06,
+    0x00, 0x09, 0x00, 0x00, 0x0a, 0x03, 0x29, 0x00, 0x0a, 0x08, 0x00, 0x0b,
+    0x10, 0x13, 0x07, 0x19, 0x00, 0x0a, 0x03, 0x23, 0x07, 0x29, 0x00, 0x0b,
+    0x12, 0x11, 0x00, 0x0a, 0x0a, 0x5b, 0x07, 0x00, 0x0a, 0x02, 0x1b, 0x07,
+    0x00, 0x0b, 0x09, 0x11, 0x00, 0x0b, 0x06, 0x13, 0x06, 0x1b, 0x0e, 0x22,
+    0x02, 0x33, 0x06, 0x39, 0x43, 0x06, 0x49, 0x00, 0x0a, 0x03, 0x2b, 0x0b,
+    0x33, 0x20, 0x42, 0x05, 0x82, 0x01, 0x02, 0xa1, 0x01, 0xc3, 0x01, 0x17,
+    0xcb, 0x01, 0x04, 0xd3, 0x01, 0x0b, 0xdb, 0x01, 0x06, 0xe3, 0x01, 0x1e,
+    0xeb, 0x01, 0x1f, 0xf2, 0x01, 0x02, 0x00, 0x0b, 0x18, 0x12, 0x0c, 0x73,
+    0x1a, 0x7b, 0x1c, 0x83, 0x01, 0x1d, 0x8b, 0x01, 0x05, 0x00, 0x0b, 0x08,
+    0x13, 0x19, 0x00, 0x0a, 0x2e, 0x00, 0x0a, 0x03, 0x23, 0x1b, 0x2b, 0x1b,
+    0x33, 0x05, 0x00, 0x0a, 0x09, 0x53, 0x09, 0x00, 0x09, 0x13, 0x1b, 0x00,
+    0x0a, 0x03, 0x23, 0x1b, 0x00, 0x09, 0x19, 0x00, 0x0a, 0x03, 0x23, 0x06,
+    0x2a, 0x02, 0x00, 0x0a, 0x04, 0x31, 0x42, 0x08, 0x92, 0x01, 0x02, 0x00,
+    0x0b, 0x22, 0x13, 0x23, 0x1a, 0x03, 0x33, 0x07, 0x3b, 0x09, 0x42, 0x03,
+    0x5b, 0x0b, 0x62, 0x03, 0x81, 0x01, 0x8b, 0x01, 0x29, 0x92, 0x01, 0x02,
+    0xa3, 0x01, 0x07, 0xab, 0x01, 0x0b, 0xb2, 0x01, 0x03, 0xcb, 0x01, 0x09,
+    0xda, 0x01, 0x02, 0x00, 0x09, 0x21, 0x00, 0x0b, 0x24, 0x11, 0x00, 0x0a,
+    0x04, 0x31, 0xa3, 0x06, 0x25, 0xbb, 0x06, 0x26, 0xc3, 0x06, 0x0a, 0xcb,
+    0x06, 0x27, 0xd3, 0x06, 0x06, 0xeb, 0x06, 0x0b, 0x00, 0x0a, 0x03, 0x52,
+    0x02, 0x00, 0x0a, 0x04, 0x32, 0x03, 0x00, 0x0a, 0x02, 0x22, 0x02, 0x33,
+    0x28, 0x3a, 0x02, 0x00, 0x2a, 0x02, 0x00, 0x09, 0x13, 0x07, 0x19, 0x00,
+    0x09, 0x13, 0x2b, 0x00, 0x0a, 0x09, 0x00, 0x0b, 0x2d, 0x12, 0x08, 0x00,
+    0x0a, 0x12, 0x00, 0x0b, 0x06, 0x13, 0x09, 0x1b, 0x06, 0x23, 0x05, 0x2b,
+    0x2f, 0x32, 0x02, 0x00, 0x09, 0x13, 0x0a, 0x00, 0x0b, 0x09, 0x13, 0x07,
+    0x00, 0x09, 0x1a, 0x03, 0x00, 0x0b, 0x09, 0x12, 0x02, 0x00, 0x0b, 0x08,
+    0x12, 0x02, 0x00, 0x0b, 0x35, 0x11, 0x00, 0x0b, 0x36, 0x13, 0x36, 0x00,
+    0x09, 0x13, 0x07, 0x1b, 0x0b, 0x00, 0x09, 0x13, 0x08, 0x1b, 0x06, 0x23,
+    0x06, 0x2a, 0x02, 0x3b, 0x06, 0x00, 0x0a, 0x05, 0x82, 0x01, 0x03, 0x00,
+    0x09, 0x1b, 0x31, 0x23, 0x26, 0x29, 0x33, 0x09, 0x3b, 0x06, 0x43, 0x31,
+    0x00, 0x0b, 0x06, 0x00, 0x0b, 0x06, 0x13, 0x06, 0x23, 0x09, 0x2b, 0x06,
+    0x33, 0x09, 0x3b, 0x06, 0x83, 0x01, 0x06, 0x8b, 0x01, 0x06, 0x93, 0x01,
+    0x06, 0x9b, 0x01, 0x05, 0x00, 0x5b, 0x3d, 0xd1, 0x03, 0x00, 0x59, 0xf9,
+    0x01, 0x00, 0x8f, 0xf8, 0xf5, 0xcb, 0x06};
+
+int FuzzMessageFilter(const uint8_t* data, size_t size) {
+  MessageFilter filter;
+  PERFETTO_CHECK(filter.LoadFilterBytecode(kValidFilter, sizeof(kValidFilter)));
+
+  auto res = filter.FilterMessage(data, size);
+
+  // Either parsing fails or if it succeeds, the output data must be <= input.
+  PERFETTO_CHECK(res.error || res.size <= size);
+  return 0;
+}
+
+}  // namespace
+}  // namespace protozero
+
+extern "C" int LLVMFuzzerTestOneInput(const uint8_t* data, size_t size);
+
+extern "C" int LLVMFuzzerTestOneInput(const uint8_t* data, size_t size) {
+  return protozero::FuzzMessageFilter(data, size);
+}
diff --git a/src/protozero/filtering/message_filter_unittest.cc b/src/protozero/filtering/message_filter_unittest.cc
new file mode 100644
index 0000000..83cb911
--- /dev/null
+++ b/src/protozero/filtering/message_filter_unittest.cc
@@ -0,0 +1,810 @@
+/*
+ * Copyright (C) 2021 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 "test/gtest_and_gmock.h"
+
+#include <random>
+
+#include "perfetto/ext/base/file_utils.h"
+#include "perfetto/ext/base/string_utils.h"
+#include "perfetto/ext/base/temp_file.h"
+#include "perfetto/protozero/proto_decoder.h"
+#include "perfetto/protozero/proto_utils.h"
+#include "perfetto/protozero/scattered_heap_buffer.h"
+#include "protos/perfetto/trace/trace.pb.h"
+#include "src/protozero/filtering/filter_util.h"
+#include "src/protozero/filtering/message_filter.h"
+
+namespace protozero {
+
+namespace {
+
+TEST(MessageFilterTest, EndToEnd) {
+  auto schema = perfetto::base::TempFile::Create();
+  static const char kSchema[] = R"(
+  syntax = "proto2";
+  message FilterSchema {
+    message Nested {
+      optional fixed32 f32 = 2;
+      repeated string ss = 5;
+    }
+    optional int32 i32 = 1;
+    optional string str = 3;
+    repeated Nested nest = 6;
+    repeated int32 f11 = 11;
+    repeated int64 f12 = 12;
+    repeated sint32 f13 = 13;
+    repeated sint64 f14 = 14;
+    repeated fixed32 f15 = 15;
+    repeated fixed32 f16 = 16;
+    repeated fixed64 f17 = 17;
+    repeated fixed64 f18 = 18;
+    repeated float f19 = 19;
+    repeated double f20 = 20;
+  };
+  )";
+
+  perfetto::base::WriteAll(*schema, kSchema, strlen(kSchema));
+  perfetto::base::FlushFile(*schema);
+
+  FilterUtil filter;
+  ASSERT_TRUE(filter.LoadMessageDefinition(schema.path(), "", ""));
+  std::string bytecode = filter.GenerateFilterBytecode();
+  ASSERT_GT(bytecode.size(), 0u);
+
+  HeapBuffered<Message> msg;
+  msg->AppendVarInt(/*field_id=*/1, -1000000000ll);
+  msg->AppendVarInt(/*field_id=*/2, 42);
+  msg->AppendString(/*field_id=*/3, "foobar");
+  msg->AppendFixed(/*field_id=*/4, 10);
+  msg->AppendVarInt(/*field_id=*/11, INT32_MIN);
+  msg->AppendVarInt(/*field_id=*/12, INT64_MIN);
+  msg->AppendSignedVarInt(/*field_id=*/13, INT32_MIN);
+  msg->AppendSignedVarInt(/*field_id=*/14, INT64_MIN);
+  msg->AppendFixed(/*field_id=*/15, static_cast<int32_t>(INT32_MIN));
+  msg->AppendFixed(/*field_id=*/16, static_cast<int32_t>(INT32_MAX));
+  msg->AppendFixed(/*field_id=*/17, static_cast<int64_t>(INT64_MIN));
+  msg->AppendFixed(/*field_id=*/18, static_cast<int64_t>(INT64_MAX));
+  msg->AppendFixed(/*field_id=*/19, FLT_EPSILON);
+  msg->AppendFixed(/*field_id=*/20, DBL_EPSILON);
+
+  auto* nest = msg->BeginNestedMessage<Message>(/*field_id=*/6);
+  nest->AppendFixed(/*field_id=*/1, 10);
+  nest->AppendFixed(/*field_id=*/2, static_cast<int32_t>(-2000000000ll));
+  nest->AppendString(/*field_id=*/4, "stripped");
+  nest->AppendString(/*field_id=*/5, "");
+  nest->Finalize();
+
+  MessageFilter flt;
+  ASSERT_TRUE(flt.LoadFilterBytecode(bytecode.data(), bytecode.size()));
+
+  std::vector<uint8_t> encoded = msg.SerializeAsArray();
+
+  for (int repetitions = 0; repetitions < 3; ++repetitions) {
+    auto filtered = flt.FilterMessage(encoded.data(), encoded.size());
+    ASSERT_LT(filtered.size, encoded.size());
+
+    ProtoDecoder dec(filtered.data.get(), filtered.size);
+    EXPECT_TRUE(dec.FindField(1).valid());
+    EXPECT_EQ(dec.FindField(1).as_int64(), -1000000000ll);
+    EXPECT_FALSE(dec.FindField(2).valid());
+    EXPECT_TRUE(dec.FindField(3).valid());
+    EXPECT_EQ(dec.FindField(3).as_std_string(), "foobar");
+    EXPECT_FALSE(dec.FindField(4).valid());
+    EXPECT_TRUE(dec.FindField(6).valid());
+    for (uint32_t i = 11; i <= 20; ++i)
+      EXPECT_TRUE(dec.FindField(i).valid());
+
+    EXPECT_EQ(dec.FindField(11).as_int32(), INT32_MIN);
+    EXPECT_EQ(dec.FindField(12).as_int64(), INT64_MIN);
+    EXPECT_EQ(dec.FindField(13).as_sint32(), INT32_MIN);
+    EXPECT_EQ(dec.FindField(14).as_sint64(), INT64_MIN);
+    EXPECT_EQ(dec.FindField(15).as_int32(), INT32_MIN);
+    EXPECT_EQ(dec.FindField(16).as_int32(), INT32_MAX);
+    EXPECT_EQ(dec.FindField(17).as_int64(), INT64_MIN);
+    EXPECT_EQ(dec.FindField(18).as_int64(), INT64_MAX);
+    EXPECT_EQ(dec.FindField(19).as_float(), FLT_EPSILON);
+    EXPECT_EQ(dec.FindField(20).as_double(), DBL_EPSILON);
+
+    ProtoDecoder nest_dec(dec.FindField(6).as_bytes());
+    EXPECT_FALSE(nest_dec.FindField(1).valid());
+    EXPECT_TRUE(nest_dec.FindField(2).valid());
+    EXPECT_EQ(nest_dec.FindField(2).as_int32(), -2000000000ll);
+    EXPECT_TRUE(nest_dec.FindField(5).valid());
+    EXPECT_EQ(nest_dec.FindField(5).as_bytes().size, 0u);
+  }
+}
+
+TEST(MessageFilterTest, ChangeRoot) {
+  auto schema = perfetto::base::TempFile::Create();
+  static const char kSchema[] = R"(
+  syntax = "proto2";
+  message FilterSchema {
+    message Nested {
+      message Nested2 {
+        optional int32 e = 5;
+      }
+      optional int32 c = 3;
+      repeated Nested2 d = 4;
+    }
+    optional int32 a = 1;
+    optional Nested b = 2;
+  };
+  )";
+
+  perfetto::base::WriteAll(*schema, kSchema, strlen(kSchema));
+  perfetto::base::FlushFile(*schema);
+
+  FilterUtil filter;
+  ASSERT_TRUE(filter.LoadMessageDefinition(schema.path(), "", ""));
+  std::string bytecode = filter.GenerateFilterBytecode();
+  ASSERT_GT(bytecode.size(), 0u);
+
+  HeapBuffered<Message> msg;
+  msg->AppendVarInt(/*field_id=*/1, 101);
+  msg->AppendVarInt(/*field_id=*/3, 103);
+  msg->AppendVarInt(/*field_id=*/5, 105);
+  auto* nest = msg->BeginNestedMessage<Message>(/*field_id=*/4);  // Nested b.
+  nest->AppendVarInt(/*field_id=*/5, 205);
+  nest->Finalize();
+  std::vector<uint8_t> encoded = msg.SerializeAsArray();
+
+  MessageFilter flt;
+  ASSERT_TRUE(flt.LoadFilterBytecode(bytecode.data(), bytecode.size()));
+  uint32_t roots[2]{2, 4};
+
+  // First set the root to field id ".2" (.b). The fliter should happen treating
+  // |Nested| as rot, so allowing only field 3 and 4 (Nested2) through.
+  {
+    flt.SetFilterRoot(roots, 1);
+    auto filtered = flt.FilterMessage(encoded.data(), encoded.size());
+    ASSERT_LT(filtered.size, encoded.size());
+    ProtoDecoder dec(filtered.data.get(), filtered.size);
+    EXPECT_FALSE(dec.FindField(1).valid());
+    EXPECT_TRUE(dec.FindField(3).valid());
+    EXPECT_EQ(dec.FindField(3).as_int32(), 103);
+    EXPECT_FALSE(dec.FindField(5).valid());
+    EXPECT_TRUE(dec.FindField(4).valid());
+    EXPECT_EQ(dec.FindField(4).as_std_string(), "(\xCD\x01");
+  }
+
+  // Now set the root to ".2.4" (.b.d). This should allow only the field "e"
+  // to pass through.
+  {
+    flt.SetFilterRoot(roots, 2);
+    auto filtered = flt.FilterMessage(encoded.data(), encoded.size());
+    ASSERT_LT(filtered.size, encoded.size());
+    ProtoDecoder dec(filtered.data.get(), filtered.size);
+    EXPECT_FALSE(dec.FindField(1).valid());
+    EXPECT_FALSE(dec.FindField(3).valid());
+    EXPECT_FALSE(dec.FindField(4).valid());
+    EXPECT_TRUE(dec.FindField(5).valid());
+    EXPECT_EQ(dec.FindField(5).as_int32(), 105);
+  }
+}
+
+TEST(MessageFilterTest, MalformedInput) {
+  // Create and load a simple filter.
+  auto schema = perfetto::base::TempFile::Create();
+  static const char kSchema[] = R"(
+  syntax = "proto2";
+  message FilterSchema {
+    message Nested {
+      optional fixed32 f32 = 4;
+      repeated string ss = 5;
+    }
+    optional int32 i32 = 1;
+    optional string str = 2;
+    repeated Nested nest = 3;
+  };
+  )";
+  perfetto::base::WriteAll(*schema, kSchema, strlen(kSchema));
+  perfetto::base::FlushFile(*schema);
+  FilterUtil filter;
+  ASSERT_TRUE(filter.LoadMessageDefinition(schema.path(), "", ""));
+  std::string bytecode = filter.GenerateFilterBytecode();
+  ASSERT_GT(bytecode.size(), 0u);
+  MessageFilter flt;
+  ASSERT_TRUE(flt.LoadFilterBytecode(bytecode.data(), bytecode.size()));
+
+  {
+    // A malformed message found by the fuzzer.
+    static const uint8_t kData[]{
+        0x52, 0x21,  // ID=10, type=len-delimited, len=33.
+        0xa0, 0xa4,  // Early terminating payload.
+    };
+    auto res = flt.FilterMessage(kData, sizeof(kData));
+    EXPECT_TRUE(res.error);
+  }
+
+  {
+    // A malformed message which contains a non-terminated varint.
+    static const uint8_t kData[]{
+        0x08, 0x2A,  // A valid varint field id=1 value=42 (0x2A).
+        0x08, 0xFF,  // An unterminated varint.
+    };
+    auto res = flt.FilterMessage(kData, sizeof(kData));
+    EXPECT_TRUE(res.error);
+  }
+
+  {
+    // A malformed message which contains a sub-message with a field that brings
+    // it out of the outer size.
+    static const uint8_t kData[]{
+        0x08, 0x2A,  // A valid varint field id=1 value=42 (0x2A).
+        0x1A, 0x04,  // A len-delim field, id=3, length=4.
+        // The nested message |nest| starts here.
+        0x25, 0x0, 0x0, 0x0, 0x01,  // A fixed32 field, id=4.
+        // Note that the fixed32 field has an expected length of 4 but that
+        // overflows the size of the |nest| method, because with its 0x25
+        // preamble it becomes 5 bytes. At this point this should cause a
+        // persistent failure.
+    };
+    auto res = flt.FilterMessage(kData, sizeof(kData));
+    EXPECT_TRUE(res.error);
+  }
+
+  // A parsing failure shoulnd't affect the ability to filter the following
+  // message. Try again but this time with a valid message.
+  {
+    static const uint8_t kData[]{
+        0x08, 0x2A,  // A valid varint field id=1 value=42 (0x2A).
+        0x1A, 0x05,  // A len-delim field, id=3, length=5.
+        0x25, 0x0,  0x0, 0x0, 0x01,  // A fixed32 field, id=4.
+        0x38, 0x42,  // A valid but not allowed varint field id=7.
+    };
+    auto res = flt.FilterMessage(kData, sizeof(kData));
+    EXPECT_FALSE(res.error);
+    EXPECT_EQ(res.size, sizeof(kData) - 2);  // last 2 bytes should be skipped.
+    EXPECT_EQ(memcmp(kData, res.data.get(), res.size), 0);
+  }
+}
+
+// It processes a real test trace with a real filter. The filter has been
+// obtained from the full upstream perfetto proto (+ re-adding the for_testing
+// field which got removed after adding most test traces). This covers the most
+// complex case of filtering a real trace with a filter that allows all possible
+// fields, hence re-entering deeply in most nested fields.
+TEST(MessageFilterTest, RealTracePassthrough) {
+  // This is test/data/android_log_ring_buffer_mode.pb. It's re-encoded as a
+  // constant because unittests cannot depend on test/data/, only integration
+  // tests can.
+  static const uint8_t kTraceData[]{
+      0x0a, 0x16, 0x18, 0x8f, 0x4e, 0xa2, 0x02, 0x10, 0x82, 0x47, 0x7a, 0x76,
+      0xb2, 0x8d, 0x42, 0xba, 0x81, 0xdc, 0x33, 0x32, 0x6d, 0x57, 0xa0, 0x79,
+      0x0a, 0x5f, 0x18, 0x8f, 0x4e, 0x32, 0x5a, 0x0a, 0x09, 0x08, 0x06, 0x10,
+      0xf4, 0xd3, 0xea, 0xbb, 0xba, 0x55, 0x0a, 0x0c, 0x08, 0x02, 0x10, 0xf9,
+      0xcc, 0xb4, 0xd1, 0xe8, 0xdc, 0xa5, 0xbc, 0x15, 0x0a, 0x09, 0x08, 0x04,
+      0x10, 0x86, 0xb9, 0x9c, 0xba, 0xba, 0x55, 0x0a, 0x0c, 0x08, 0x01, 0x10,
+      0xeb, 0xe9, 0x82, 0xd3, 0xe8, 0xdc, 0xa5, 0xbc, 0x15, 0x0a, 0x09, 0x08,
+      0x03, 0x10, 0xac, 0xd6, 0xea, 0xbb, 0xba, 0x55, 0x0a, 0x09, 0x08, 0x05,
+      0x10, 0x9b, 0xe1, 0xd8, 0xbb, 0xba, 0x55, 0x0a, 0x07, 0x08, 0x07, 0x10,
+      0xf5, 0xe6, 0xd9, 0x55, 0x0a, 0x07, 0x08, 0x08, 0x10, 0xc1, 0xcc, 0xa7,
+      0x41, 0x0a, 0x27, 0x18, 0x8f, 0x4e, 0x9a, 0x02, 0x21, 0x0a, 0x13, 0x08,
+      0xf0, 0x1f, 0x10, 0x01, 0x18, 0x00, 0x20, 0x00, 0x28, 0x00, 0x30, 0x00,
+      0x38, 0x00, 0x40, 0x00, 0x48, 0x00, 0x10, 0x01, 0x18, 0x07, 0x20, 0x06,
+      0x28, 0x0b, 0x30, 0x01, 0x38, 0x01, 0x0a, 0xd5, 0x01, 0x18, 0x8f, 0x4e,
+      0x8a, 0x02, 0xce, 0x01, 0x0a, 0x06, 0x08, 0x80, 0x80, 0x02, 0x20, 0x00,
+      0x12, 0xa5, 0x01, 0x0a, 0xa2, 0x01, 0x0a, 0x0b, 0x61, 0x6e, 0x64, 0x72,
+      0x6f, 0x69, 0x64, 0x2e, 0x6c, 0x6f, 0x67, 0x10, 0x00, 0x18, 0x00, 0x20,
+      0x00, 0xa2, 0x06, 0x04, 0x50, 0x00, 0x58, 0x00, 0xaa, 0x06, 0x02, 0x0a,
+      0x00, 0xb2, 0x06, 0x08, 0x08, 0x00, 0x10, 0x00, 0x18, 0x00, 0x20, 0x00,
+      0xba, 0x06, 0x06, 0x10, 0x00, 0x18, 0x00, 0x20, 0x00, 0xc2, 0x06, 0x06,
+      0x08, 0x00, 0x18, 0x00, 0x28, 0x00, 0xca, 0x06, 0x0a, 0x08, 0x00, 0x28,
+      0x00, 0x32, 0x04, 0x28, 0x00, 0x30, 0x00, 0xd2, 0x06, 0x02, 0x08, 0x00,
+      0xda, 0x06, 0x02, 0x18, 0x00, 0xc2, 0x3e, 0x00, 0xfa, 0xff, 0xff, 0xff,
+      0x07, 0x46, 0x08, 0x00, 0x10, 0x00, 0x18, 0x00, 0x20, 0x00, 0x28, 0x00,
+      0x32, 0x3a, 0x08, 0x00, 0x10, 0x00, 0x18, 0x00, 0x20, 0x00, 0x29, 0x00,
+      0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x31, 0x00, 0x00, 0x00, 0x00,
+      0x00, 0x00, 0x00, 0x00, 0x3d, 0x00, 0x00, 0x00, 0x00, 0x45, 0x00, 0x00,
+      0x00, 0x00, 0x49, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x55,
+      0x00, 0x00, 0x00, 0x00, 0x58, 0x00, 0x60, 0x00, 0x6a, 0x00, 0x72, 0x00,
+      0x18, 0xe0, 0xd4, 0x03, 0x20, 0x00, 0x28, 0x00, 0x3a, 0x06, 0x08, 0x00,
+      0x10, 0x00, 0x18, 0x00, 0x40, 0x00, 0x48, 0x00, 0x50, 0x00, 0x5a, 0x02,
+      0x08, 0x00, 0x60, 0x00, 0x68, 0x00, 0x0a, 0x94, 0x01, 0x40, 0xd9, 0xf4,
+      0x98, 0x96, 0xbe, 0x54, 0xba, 0x02, 0x84, 0x81, 0x80, 0x00, 0x0a, 0xff,
+      0x80, 0x80, 0x00, 0x38, 0x04, 0x32, 0x08, 0x70, 0x65, 0x72, 0x66, 0x65,
+      0x74, 0x74, 0x6f, 0x42, 0x5d, 0x61, 0x6e, 0x64, 0x72, 0x6f, 0x69, 0x64,
+      0x5f, 0x6c, 0x6f, 0x67, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x73, 0x6f,
+      0x75, 0x72, 0x63, 0x65, 0x2e, 0x63, 0x63, 0x3a, 0x31, 0x35, 0x35, 0x20,
+      0x53, 0x74, 0x61, 0x72, 0x74, 0x69, 0x6e, 0x67, 0x20, 0x41, 0x6e, 0x64,
+      0x72, 0x6f, 0x69, 0x64, 0x20, 0x6c, 0x6f, 0x67, 0x20, 0x64, 0x61, 0x74,
+      0x61, 0x20, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x3a, 0x20, 0x73, 0x74,
+      0x72, 0x65, 0x61, 0x6d, 0x20, 0x74, 0x61, 0x69, 0x6c, 0x3d, 0x31, 0x20,
+      0x6c, 0x69, 0x64, 0x73, 0x3d, 0x30, 0x2c, 0x32, 0x2c, 0x33, 0x2c, 0x34,
+      0x2c, 0x37, 0x28, 0xdb, 0xe6, 0x9b, 0xfb, 0xeb, 0xdb, 0xa5, 0xbc, 0x15,
+      0x08, 0x00, 0x10, 0xb5, 0x58, 0x18, 0xb5, 0x58, 0x20, 0x00, 0x18, 0x8f,
+      0x4e, 0x0a, 0xf4, 0x0a, 0x40, 0xb2, 0x84, 0xde, 0xdd, 0x8f, 0x55, 0xba,
+      0x02, 0xe4, 0x8a, 0x80, 0x00, 0x0a, 0xe4, 0x80, 0x80, 0x00, 0x38, 0x04,
+      0x32, 0x12, 0x76, 0x65, 0x6e, 0x64, 0x6f, 0x72, 0x2e, 0x72, 0x6d, 0x74,
+      0x5f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x42, 0x38, 0x72, 0x6d,
+      0x74, 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x63, 0x6f,
+      0x6e, 0x6e, 0x65, 0x63, 0x74, 0x5f, 0x63, 0x62, 0x3a, 0x20, 0x63, 0x6c,
+      0x6e, 0x74, 0x5f, 0x68, 0x3d, 0x30, 0x78, 0x31, 0x34, 0x20, 0x63, 0x6f,
+      0x6e, 0x6e, 0x5f, 0x68, 0x3d, 0x30, 0x78, 0x37, 0x32, 0x31, 0x64, 0x61,
+      0x30, 0x37, 0x31, 0x30, 0x30, 0x0a, 0x28, 0x92, 0xad, 0xf6, 0xf0, 0xbd,
+      0xdc, 0xa5, 0xbc, 0x15, 0x08, 0x00, 0x10, 0xe9, 0x06, 0x18, 0xe9, 0x06,
+      0x20, 0x00, 0x0a, 0xf3, 0x80, 0x80, 0x00, 0x38, 0x06, 0x32, 0x0b, 0x72,
+      0x6d, 0x74, 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x42, 0x4e,
+      0x49, 0x4e, 0x46, 0x4f, 0x3a, 0x72, 0x6d, 0x74, 0x5f, 0x73, 0x74, 0x6f,
+      0x72, 0x61, 0x67, 0x65, 0x5f, 0x72, 0x77, 0x5f, 0x69, 0x6f, 0x76, 0x65,
+      0x63, 0x5f, 0x63, 0x62, 0x3a, 0x20, 0x57, 0x72, 0x69, 0x74, 0x65, 0x20,
+      0x69, 0x6f, 0x76, 0x65, 0x63, 0x20, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73,
+      0x74, 0x20, 0x72, 0x65, 0x63, 0x65, 0x69, 0x76, 0x65, 0x64, 0x20, 0x66,
+      0x6f, 0x72, 0x20, 0x2f, 0x62, 0x6f, 0x6f, 0x74, 0x2f, 0x6d, 0x6f, 0x64,
+      0x65, 0x6d, 0x5f, 0x66, 0x73, 0x63, 0x28, 0x9e, 0xa1, 0xea, 0xf0, 0xbd,
+      0xdc, 0xa5, 0xbc, 0x15, 0x08, 0x07, 0x10, 0xe9, 0x06, 0x18, 0xe9, 0x06,
+      0x20, 0x00, 0x0a, 0xfd, 0x80, 0x80, 0x00, 0x38, 0x06, 0x32, 0x0b, 0x72,
+      0x6d, 0x74, 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x42, 0x58,
+      0x49, 0x4e, 0x46, 0x4f, 0x3a, 0x72, 0x6d, 0x74, 0x5f, 0x73, 0x74, 0x6f,
+      0x72, 0x61, 0x67, 0x65, 0x5f, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f,
+      0x74, 0x68, 0x72, 0x65, 0x61, 0x64, 0x3a, 0x20, 0x43, 0x61, 0x6c, 0x6c,
+      0x69, 0x6e, 0x67, 0x20, 0x57, 0x72, 0x69, 0x74, 0x65, 0x20, 0x5b, 0x6f,
+      0x66, 0x66, 0x73, 0x65, 0x74, 0x3d, 0x30, 0x2c, 0x20, 0x73, 0x69, 0x7a,
+      0x65, 0x3d, 0x36, 0x35, 0x35, 0x33, 0x36, 0x5d, 0x66, 0x6f, 0x72, 0x20,
+      0x2f, 0x62, 0x6f, 0x6f, 0x74, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6d, 0x5f,
+      0x66, 0x73, 0x63, 0x21, 0x28, 0xe6, 0xc4, 0x80, 0xf1, 0xbd, 0xdc, 0xa5,
+      0xbc, 0x15, 0x08, 0x07, 0x10, 0xf6, 0x0b, 0x18, 0xf6, 0x0b, 0x20, 0x00,
+      0x0a, 0x80, 0x81, 0x80, 0x00, 0x38, 0x04, 0x32, 0x12, 0x76, 0x65, 0x6e,
+      0x64, 0x6f, 0x72, 0x2e, 0x72, 0x6d, 0x74, 0x5f, 0x73, 0x74, 0x6f, 0x72,
+      0x61, 0x67, 0x65, 0x42, 0x54, 0x72, 0x6d, 0x74, 0x5f, 0x73, 0x74, 0x6f,
+      0x72, 0x61, 0x67, 0x65, 0x5f, 0x72, 0x77, 0x5f, 0x69, 0x6f, 0x76, 0x65,
+      0x63, 0x5f, 0x63, 0x62, 0x3a, 0x20, 0x2f, 0x62, 0x6f, 0x6f, 0x74, 0x2f,
+      0x6d, 0x6f, 0x64, 0x65, 0x6d, 0x5f, 0x66, 0x73, 0x63, 0x3a, 0x20, 0x72,
+      0x65, 0x71, 0x5f, 0x68, 0x3d, 0x30, 0x78, 0x31, 0x34, 0x20, 0x6d, 0x73,
+      0x67, 0x5f, 0x69, 0x64, 0x3d, 0x33, 0x3a, 0x20, 0x52, 0x2f, 0x57, 0x20,
+      0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x20, 0x72, 0x65, 0x63, 0x65,
+      0x69, 0x76, 0x65, 0x64, 0x0a, 0x28, 0xc3, 0xe0, 0xf9, 0xf0, 0xbd, 0xdc,
+      0xa5, 0xbc, 0x15, 0x08, 0x00, 0x10, 0xe9, 0x06, 0x18, 0xe9, 0x06, 0x20,
+      0x00, 0x0a, 0xc1, 0x80, 0x80, 0x00, 0x38, 0x04, 0x32, 0x12, 0x76, 0x65,
+      0x6e, 0x64, 0x6f, 0x72, 0x2e, 0x72, 0x6d, 0x74, 0x5f, 0x73, 0x74, 0x6f,
+      0x72, 0x61, 0x67, 0x65, 0x42, 0x15, 0x77, 0x61, 0x6b, 0x65, 0x6c, 0x6f,
+      0x63, 0x6b, 0x20, 0x61, 0x63, 0x71, 0x75, 0x69, 0x72, 0x65, 0x64, 0x3a,
+      0x20, 0x31, 0x0a, 0x28, 0xb4, 0xc7, 0x8a, 0xf1, 0xbd, 0xdc, 0xa5, 0xbc,
+      0x15, 0x08, 0x00, 0x10, 0xe9, 0x06, 0x18, 0xe9, 0x06, 0x20, 0x00, 0x0a,
+      0x84, 0x81, 0x80, 0x00, 0x38, 0x04, 0x32, 0x12, 0x76, 0x65, 0x6e, 0x64,
+      0x6f, 0x72, 0x2e, 0x72, 0x6d, 0x74, 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x61,
+      0x67, 0x65, 0x42, 0x58, 0x72, 0x6d, 0x74, 0x5f, 0x73, 0x74, 0x6f, 0x72,
+      0x61, 0x67, 0x65, 0x5f, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x74,
+      0x68, 0x72, 0x65, 0x61, 0x64, 0x3a, 0x20, 0x2f, 0x62, 0x6f, 0x6f, 0x74,
+      0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6d, 0x5f, 0x66, 0x73, 0x63, 0x3a, 0x20,
+      0x55, 0x6e, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x20, 0x77, 0x6f, 0x72, 0x6b,
+      0x65, 0x72, 0x20, 0x74, 0x68, 0x72, 0x65, 0x61, 0x64, 0x20, 0x28, 0x74,
+      0x68, 0x5f, 0x69, 0x64, 0x3a, 0x20, 0x34, 0x39, 0x30, 0x31, 0x31, 0x34,
+      0x39, 0x35, 0x34, 0x34, 0x36, 0x34, 0x29, 0x0a, 0x28, 0x8e, 0xe0, 0x8e,
+      0xf1, 0xbd, 0xdc, 0xa5, 0xbc, 0x15, 0x08, 0x00, 0x10, 0xe9, 0x06, 0x18,
+      0xf6, 0x0b, 0x20, 0x00, 0x0a, 0x83, 0x81, 0x80, 0x00, 0x38, 0x04, 0x32,
+      0x12, 0x76, 0x65, 0x6e, 0x64, 0x6f, 0x72, 0x2e, 0x72, 0x6d, 0x74, 0x5f,
+      0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x42, 0x57, 0x72, 0x6d, 0x74,
+      0x5f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x63, 0x6c, 0x69,
+      0x65, 0x6e, 0x74, 0x5f, 0x74, 0x68, 0x72, 0x65, 0x61, 0x64, 0x3a, 0x20,
+      0x2f, 0x62, 0x6f, 0x6f, 0x74, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6d, 0x5f,
+      0x66, 0x73, 0x63, 0x3a, 0x20, 0x72, 0x65, 0x71, 0x5f, 0x68, 0x3d, 0x30,
+      0x78, 0x31, 0x34, 0x20, 0x6d, 0x73, 0x67, 0x5f, 0x69, 0x64, 0x3d, 0x33,
+      0x3a, 0x20, 0x42, 0x79, 0x74, 0x65, 0x73, 0x20, 0x77, 0x72, 0x69, 0x74,
+      0x74, 0x65, 0x6e, 0x20, 0x3d, 0x20, 0x36, 0x35, 0x35, 0x33, 0x36, 0x0a,
+      0x28, 0x8a, 0xe1, 0xa0, 0xf2, 0xbd, 0xdc, 0xa5, 0xbc, 0x15, 0x08, 0x00,
+      0x10, 0xe9, 0x06, 0x18, 0xf6, 0x0b, 0x20, 0x00, 0x0a, 0x88, 0x81, 0x80,
+      0x00, 0x38, 0x04, 0x32, 0x12, 0x76, 0x65, 0x6e, 0x64, 0x6f, 0x72, 0x2e,
+      0x72, 0x6d, 0x74, 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x42,
+      0x5c, 0x72, 0x6d, 0x74, 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65,
+      0x5f, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x68, 0x72, 0x65,
+      0x61, 0x64, 0x3a, 0x20, 0x2f, 0x62, 0x6f, 0x6f, 0x74, 0x2f, 0x6d, 0x6f,
+      0x64, 0x65, 0x6d, 0x5f, 0x66, 0x73, 0x63, 0x3a, 0x20, 0x72, 0x65, 0x71,
+      0x5f, 0x68, 0x3d, 0x30, 0x78, 0x31, 0x34, 0x20, 0x6d, 0x73, 0x67, 0x5f,
+      0x69, 0x64, 0x3d, 0x33, 0x3a, 0x20, 0x53, 0x65, 0x6e, 0x64, 0x20, 0x72,
+      0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x3a, 0x20, 0x72, 0x65, 0x73,
+      0x3d, 0x30, 0x20, 0x65, 0x72, 0x72, 0x3d, 0x30, 0x0a, 0x28, 0xf8, 0x89,
+      0xa2, 0xf2, 0xbd, 0xdc, 0xa5, 0xbc, 0x15, 0x08, 0x00, 0x10, 0xe9, 0x06,
+      0x18, 0xf6, 0x0b, 0x20, 0x00, 0x0a, 0xae, 0x81, 0x80, 0x00, 0x38, 0x04,
+      0x32, 0x12, 0x76, 0x65, 0x6e, 0x64, 0x6f, 0x72, 0x2e, 0x72, 0x6d, 0x74,
+      0x5f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x42, 0x81, 0x01, 0x72,
+      0x6d, 0x74, 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x63,
+      0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x68, 0x72, 0x65, 0x61, 0x64,
+      0x3a, 0x20, 0x2f, 0x62, 0x6f, 0x6f, 0x74, 0x2f, 0x6d, 0x6f, 0x64, 0x65,
+      0x6d, 0x5f, 0x66, 0x73, 0x63, 0x3a, 0x20, 0x41, 0x62, 0x6f, 0x75, 0x74,
+      0x20, 0x74, 0x6f, 0x20, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x20, 0x72, 0x6d,
+      0x74, 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x20, 0x63, 0x6c,
+      0x69, 0x65, 0x6e, 0x74, 0x20, 0x74, 0x68, 0x72, 0x65, 0x61, 0x64, 0x20,
+      0x28, 0x74, 0x68, 0x5f, 0x69, 0x64, 0x3a, 0x20, 0x34, 0x39, 0x30, 0x31,
+      0x31, 0x34, 0x39, 0x35, 0x34, 0x34, 0x36, 0x34, 0x29, 0x20, 0x77, 0x61,
+      0x6b, 0x65, 0x6c, 0x6f, 0x63, 0x6b, 0x20, 0x72, 0x65, 0x6c, 0x65, 0x61,
+      0x73, 0x65, 0x64, 0x3a, 0x20, 0x31, 0x0a, 0x0a, 0x28, 0xb9, 0xcb, 0xa9,
+      0xf2, 0xbd, 0xdc, 0xa5, 0xbc, 0x15, 0x08, 0x00, 0x10, 0xe9, 0x06, 0x18,
+      0xf6, 0x0b, 0x20, 0x00, 0x0a, 0xf4, 0x80, 0x80, 0x00, 0x38, 0x06, 0x32,
+      0x0b, 0x72, 0x6d, 0x74, 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65,
+      0x42, 0x4f, 0x49, 0x4e, 0x46, 0x4f, 0x3a, 0x72, 0x6d, 0x74, 0x5f, 0x73,
+      0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x63, 0x6c, 0x69, 0x65, 0x6e,
+      0x74, 0x5f, 0x74, 0x68, 0x72, 0x65, 0x61, 0x64, 0x3a, 0x20, 0x44, 0x6f,
+      0x6e, 0x65, 0x20, 0x57, 0x72, 0x69, 0x74, 0x65, 0x20, 0x28, 0x62, 0x79,
+      0x74, 0x65, 0x73, 0x20, 0x3d, 0x20, 0x36, 0x35, 0x35, 0x33, 0x36, 0x29,
+      0x20, 0x66, 0x6f, 0x72, 0x20, 0x2f, 0x62, 0x6f, 0x6f, 0x74, 0x2f, 0x6d,
+      0x6f, 0x64, 0x65, 0x6d, 0x5f, 0x66, 0x73, 0x63, 0x21, 0x28, 0xd6, 0xbd,
+      0x8f, 0xf2, 0xbd, 0xdc, 0xa5, 0xbc, 0x15, 0x08, 0x07, 0x10, 0xf6, 0x0b,
+      0x18, 0xf6, 0x0b, 0x20, 0x00, 0x0a, 0xe7, 0x80, 0x80, 0x00, 0x38, 0x04,
+      0x32, 0x12, 0x76, 0x65, 0x6e, 0x64, 0x6f, 0x72, 0x2e, 0x72, 0x6d, 0x74,
+      0x5f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x42, 0x3b, 0x72, 0x6d,
+      0x74, 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x64, 0x69,
+      0x73, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x5f, 0x63, 0x62, 0x3a,
+      0x20, 0x63, 0x6c, 0x6e, 0x74, 0x5f, 0x68, 0x3d, 0x30, 0x78, 0x31, 0x34,
+      0x20, 0x63, 0x6f, 0x6e, 0x6e, 0x5f, 0x68, 0x3d, 0x30, 0x78, 0x37, 0x32,
+      0x31, 0x64, 0x61, 0x30, 0x37, 0x31, 0x30, 0x30, 0x0a, 0x28, 0x92, 0xd5,
+      0xc7, 0xf2, 0xbd, 0xdc, 0xa5, 0xbc, 0x15, 0x08, 0x00, 0x10, 0xe9, 0x06,
+      0x18, 0xe9, 0x06, 0x20, 0x00, 0x18, 0x8f, 0x4e, 0x0a, 0x70, 0x40, 0x92,
+      0xaf, 0xd5, 0x8d, 0x90, 0x55, 0xba, 0x02, 0xe0, 0x80, 0x80, 0x00, 0x0a,
+      0xdb, 0x80, 0x80, 0x00, 0x38, 0x04, 0x32, 0x08, 0x70, 0x65, 0x72, 0x66,
+      0x65, 0x74, 0x74, 0x6f, 0x42, 0x39, 0x61, 0x6e, 0x64, 0x72, 0x6f, 0x69,
+      0x64, 0x5f, 0x6c, 0x6f, 0x67, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x73,
+      0x6f, 0x75, 0x72, 0x63, 0x65, 0x2e, 0x63, 0x63, 0x3a, 0x32, 0x39, 0x31,
+      0x20, 0x53, 0x65, 0x65, 0x6e, 0x20, 0x31, 0x31, 0x20, 0x41, 0x6e, 0x64,
+      0x72, 0x6f, 0x69, 0x64, 0x20, 0x6c, 0x6f, 0x67, 0x20, 0x65, 0x76, 0x65,
+      0x6e, 0x74, 0x73, 0x28, 0xd0, 0x8f, 0xfa, 0xf4, 0xbd, 0xdc, 0xa5, 0xbc,
+      0x15, 0x08, 0x00, 0x10, 0xb5, 0x58, 0x18, 0xb5, 0x58, 0x20, 0x00, 0x18,
+      0x8f, 0x4e, 0x0a, 0xfa, 0x0a, 0x40, 0x9c, 0xc7, 0xf8, 0xbc, 0x90, 0x55,
+      0xba, 0x02, 0xea, 0x8a, 0x80, 0x00, 0x0a, 0xe4, 0x80, 0x80, 0x00, 0x38,
+      0x04, 0x32, 0x12, 0x76, 0x65, 0x6e, 0x64, 0x6f, 0x72, 0x2e, 0x72, 0x6d,
+      0x74, 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x42, 0x38, 0x72,
+      0x6d, 0x74, 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x63,
+      0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x5f, 0x63, 0x62, 0x3a, 0x20, 0x63,
+      0x6c, 0x6e, 0x74, 0x5f, 0x68, 0x3d, 0x30, 0x78, 0x31, 0x35, 0x20, 0x63,
+      0x6f, 0x6e, 0x6e, 0x5f, 0x68, 0x3d, 0x30, 0x78, 0x37, 0x32, 0x31, 0x64,
+      0x61, 0x30, 0x37, 0x31, 0x30, 0x30, 0x0a, 0x28, 0xb4, 0xa2, 0x8b, 0xa5,
+      0xbe, 0xdc, 0xa5, 0xbc, 0x15, 0x08, 0x00, 0x10, 0xe9, 0x06, 0x18, 0xe9,
+      0x06, 0x20, 0x00, 0x0a, 0x80, 0x81, 0x80, 0x00, 0x38, 0x04, 0x32, 0x12,
+      0x76, 0x65, 0x6e, 0x64, 0x6f, 0x72, 0x2e, 0x72, 0x6d, 0x74, 0x5f, 0x73,
+      0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x42, 0x54, 0x72, 0x6d, 0x74, 0x5f,
+      0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x72, 0x77, 0x5f, 0x69,
+      0x6f, 0x76, 0x65, 0x63, 0x5f, 0x63, 0x62, 0x3a, 0x20, 0x2f, 0x62, 0x6f,
+      0x6f, 0x74, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6d, 0x5f, 0x66, 0x73, 0x31,
+      0x3a, 0x20, 0x72, 0x65, 0x71, 0x5f, 0x68, 0x3d, 0x30, 0x78, 0x31, 0x35,
+      0x20, 0x6d, 0x73, 0x67, 0x5f, 0x69, 0x64, 0x3d, 0x33, 0x3a, 0x20, 0x52,
+      0x2f, 0x57, 0x20, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x20, 0x72,
+      0x65, 0x63, 0x65, 0x69, 0x76, 0x65, 0x64, 0x0a, 0x28, 0x87, 0xf1, 0x8e,
+      0xa5, 0xbe, 0xdc, 0xa5, 0xbc, 0x15, 0x08, 0x00, 0x10, 0xe9, 0x06, 0x18,
+      0xe9, 0x06, 0x20, 0x00, 0x0a, 0xc1, 0x80, 0x80, 0x00, 0x38, 0x04, 0x32,
+      0x12, 0x76, 0x65, 0x6e, 0x64, 0x6f, 0x72, 0x2e, 0x72, 0x6d, 0x74, 0x5f,
+      0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x42, 0x15, 0x77, 0x61, 0x6b,
+      0x65, 0x6c, 0x6f, 0x63, 0x6b, 0x20, 0x61, 0x63, 0x71, 0x75, 0x69, 0x72,
+      0x65, 0x64, 0x3a, 0x20, 0x31, 0x0a, 0x28, 0x8e, 0x8e, 0x9e, 0xa5, 0xbe,
+      0xdc, 0xa5, 0xbc, 0x15, 0x08, 0x00, 0x10, 0xe9, 0x06, 0x18, 0xe9, 0x06,
+      0x20, 0x00, 0x0a, 0x84, 0x81, 0x80, 0x00, 0x38, 0x04, 0x32, 0x12, 0x76,
+      0x65, 0x6e, 0x64, 0x6f, 0x72, 0x2e, 0x72, 0x6d, 0x74, 0x5f, 0x73, 0x74,
+      0x6f, 0x72, 0x61, 0x67, 0x65, 0x42, 0x58, 0x72, 0x6d, 0x74, 0x5f, 0x73,
+      0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x63, 0x6c, 0x69, 0x65, 0x6e,
+      0x74, 0x5f, 0x74, 0x68, 0x72, 0x65, 0x61, 0x64, 0x3a, 0x20, 0x2f, 0x62,
+      0x6f, 0x6f, 0x74, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6d, 0x5f, 0x66, 0x73,
+      0x31, 0x3a, 0x20, 0x55, 0x6e, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x20, 0x77,
+      0x6f, 0x72, 0x6b, 0x65, 0x72, 0x20, 0x74, 0x68, 0x72, 0x65, 0x61, 0x64,
+      0x20, 0x28, 0x74, 0x68, 0x5f, 0x69, 0x64, 0x3a, 0x20, 0x34, 0x39, 0x30,
+      0x31, 0x32, 0x38, 0x37, 0x30, 0x30, 0x36, 0x34, 0x30, 0x29, 0x0a, 0x28,
+      0x9a, 0xa4, 0xa1, 0xa5, 0xbe, 0xdc, 0xa5, 0xbc, 0x15, 0x08, 0x00, 0x10,
+      0xe9, 0x06, 0x18, 0xf1, 0x0b, 0x20, 0x00, 0x0a, 0xf3, 0x80, 0x80, 0x00,
+      0x38, 0x06, 0x32, 0x0b, 0x72, 0x6d, 0x74, 0x5f, 0x73, 0x74, 0x6f, 0x72,
+      0x61, 0x67, 0x65, 0x42, 0x4e, 0x49, 0x4e, 0x46, 0x4f, 0x3a, 0x72, 0x6d,
+      0x74, 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x72, 0x77,
+      0x5f, 0x69, 0x6f, 0x76, 0x65, 0x63, 0x5f, 0x63, 0x62, 0x3a, 0x20, 0x57,
+      0x72, 0x69, 0x74, 0x65, 0x20, 0x69, 0x6f, 0x76, 0x65, 0x63, 0x20, 0x72,
+      0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x20, 0x72, 0x65, 0x63, 0x65, 0x69,
+      0x76, 0x65, 0x64, 0x20, 0x66, 0x6f, 0x72, 0x20, 0x2f, 0x62, 0x6f, 0x6f,
+      0x74, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6d, 0x5f, 0x66, 0x73, 0x31, 0x28,
+      0x8e, 0xb1, 0xff, 0xa4, 0xbe, 0xdc, 0xa5, 0xbc, 0x15, 0x08, 0x07, 0x10,
+      0xe9, 0x06, 0x18, 0xe9, 0x06, 0x20, 0x00, 0x0a, 0xff, 0x80, 0x80, 0x00,
+      0x38, 0x06, 0x32, 0x0b, 0x72, 0x6d, 0x74, 0x5f, 0x73, 0x74, 0x6f, 0x72,
+      0x61, 0x67, 0x65, 0x42, 0x5a, 0x49, 0x4e, 0x46, 0x4f, 0x3a, 0x72, 0x6d,
+      0x74, 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x63, 0x6c,
+      0x69, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x68, 0x72, 0x65, 0x61, 0x64, 0x3a,
+      0x20, 0x43, 0x61, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x20, 0x57, 0x72, 0x69,
+      0x74, 0x65, 0x20, 0x5b, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x3d, 0x30,
+      0x2c, 0x20, 0x73, 0x69, 0x7a, 0x65, 0x3d, 0x32, 0x30, 0x39, 0x37, 0x31,
+      0x35, 0x32, 0x5d, 0x66, 0x6f, 0x72, 0x20, 0x2f, 0x62, 0x6f, 0x6f, 0x74,
+      0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6d, 0x5f, 0x66, 0x73, 0x31, 0x21, 0x28,
+      0xe6, 0xae, 0x92, 0xa5, 0xbe, 0xdc, 0xa5, 0xbc, 0x15, 0x08, 0x07, 0x10,
+      0xf1, 0x0b, 0x18, 0xf1, 0x0b, 0x20, 0x00, 0x0a, 0x85, 0x81, 0x80, 0x00,
+      0x38, 0x04, 0x32, 0x12, 0x76, 0x65, 0x6e, 0x64, 0x6f, 0x72, 0x2e, 0x72,
+      0x6d, 0x74, 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x42, 0x59,
+      0x72, 0x6d, 0x74, 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f,
+      0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x68, 0x72, 0x65, 0x61,
+      0x64, 0x3a, 0x20, 0x2f, 0x62, 0x6f, 0x6f, 0x74, 0x2f, 0x6d, 0x6f, 0x64,
+      0x65, 0x6d, 0x5f, 0x66, 0x73, 0x31, 0x3a, 0x20, 0x72, 0x65, 0x71, 0x5f,
+      0x68, 0x3d, 0x30, 0x78, 0x31, 0x35, 0x20, 0x6d, 0x73, 0x67, 0x5f, 0x69,
+      0x64, 0x3d, 0x33, 0x3a, 0x20, 0x42, 0x79, 0x74, 0x65, 0x73, 0x20, 0x77,
+      0x72, 0x69, 0x74, 0x74, 0x65, 0x6e, 0x20, 0x3d, 0x20, 0x32, 0x30, 0x39,
+      0x37, 0x31, 0x35, 0x32, 0x0a, 0x28, 0x96, 0x87, 0xd7, 0xb3, 0xbe, 0xdc,
+      0xa5, 0xbc, 0x15, 0x08, 0x00, 0x10, 0xe9, 0x06, 0x18, 0xf1, 0x0b, 0x20,
+      0x00, 0x0a, 0x88, 0x81, 0x80, 0x00, 0x38, 0x04, 0x32, 0x12, 0x76, 0x65,
+      0x6e, 0x64, 0x6f, 0x72, 0x2e, 0x72, 0x6d, 0x74, 0x5f, 0x73, 0x74, 0x6f,
+      0x72, 0x61, 0x67, 0x65, 0x42, 0x5c, 0x72, 0x6d, 0x74, 0x5f, 0x73, 0x74,
+      0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74,
+      0x5f, 0x74, 0x68, 0x72, 0x65, 0x61, 0x64, 0x3a, 0x20, 0x2f, 0x62, 0x6f,
+      0x6f, 0x74, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6d, 0x5f, 0x66, 0x73, 0x31,
+      0x3a, 0x20, 0x72, 0x65, 0x71, 0x5f, 0x68, 0x3d, 0x30, 0x78, 0x31, 0x35,
+      0x20, 0x6d, 0x73, 0x67, 0x5f, 0x69, 0x64, 0x3d, 0x33, 0x3a, 0x20, 0x53,
+      0x65, 0x6e, 0x64, 0x20, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
+      0x3a, 0x20, 0x72, 0x65, 0x73, 0x3d, 0x30, 0x20, 0x65, 0x72, 0x72, 0x3d,
+      0x30, 0x0a, 0x28, 0xda, 0xb3, 0xd8, 0xb3, 0xbe, 0xdc, 0xa5, 0xbc, 0x15,
+      0x08, 0x00, 0x10, 0xe9, 0x06, 0x18, 0xf1, 0x0b, 0x20, 0x00, 0x0a, 0xae,
+      0x81, 0x80, 0x00, 0x38, 0x04, 0x32, 0x12, 0x76, 0x65, 0x6e, 0x64, 0x6f,
+      0x72, 0x2e, 0x72, 0x6d, 0x74, 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67,
+      0x65, 0x42, 0x81, 0x01, 0x72, 0x6d, 0x74, 0x5f, 0x73, 0x74, 0x6f, 0x72,
+      0x61, 0x67, 0x65, 0x5f, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x74,
+      0x68, 0x72, 0x65, 0x61, 0x64, 0x3a, 0x20, 0x2f, 0x62, 0x6f, 0x6f, 0x74,
+      0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6d, 0x5f, 0x66, 0x73, 0x31, 0x3a, 0x20,
+      0x41, 0x62, 0x6f, 0x75, 0x74, 0x20, 0x74, 0x6f, 0x20, 0x62, 0x6c, 0x6f,
+      0x63, 0x6b, 0x20, 0x72, 0x6d, 0x74, 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x61,
+      0x67, 0x65, 0x20, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x20, 0x74, 0x68,
+      0x72, 0x65, 0x61, 0x64, 0x20, 0x28, 0x74, 0x68, 0x5f, 0x69, 0x64, 0x3a,
+      0x20, 0x34, 0x39, 0x30, 0x31, 0x32, 0x38, 0x37, 0x30, 0x30, 0x36, 0x34,
+      0x30, 0x29, 0x20, 0x77, 0x61, 0x6b, 0x65, 0x6c, 0x6f, 0x63, 0x6b, 0x20,
+      0x72, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x64, 0x3a, 0x20, 0x31, 0x0a,
+      0x0a, 0x28, 0xe4, 0xa2, 0xe0, 0xb3, 0xbe, 0xdc, 0xa5, 0xbc, 0x15, 0x08,
+      0x00, 0x10, 0xe9, 0x06, 0x18, 0xf1, 0x0b, 0x20, 0x00, 0x0a, 0xe7, 0x80,
+      0x80, 0x00, 0x38, 0x04, 0x32, 0x12, 0x76, 0x65, 0x6e, 0x64, 0x6f, 0x72,
+      0x2e, 0x72, 0x6d, 0x74, 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65,
+      0x42, 0x3b, 0x72, 0x6d, 0x74, 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67,
+      0x65, 0x5f, 0x64, 0x69, 0x73, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,
+      0x5f, 0x63, 0x62, 0x3a, 0x20, 0x63, 0x6c, 0x6e, 0x74, 0x5f, 0x68, 0x3d,
+      0x30, 0x78, 0x31, 0x35, 0x20, 0x63, 0x6f, 0x6e, 0x6e, 0x5f, 0x68, 0x3d,
+      0x30, 0x78, 0x37, 0x32, 0x31, 0x64, 0x61, 0x30, 0x37, 0x31, 0x30, 0x30,
+      0x0a, 0x28, 0xeb, 0xea, 0x8f, 0xb4, 0xbe, 0xdc, 0xa5, 0xbc, 0x15, 0x08,
+      0x00, 0x10, 0xe9, 0x06, 0x18, 0xe9, 0x06, 0x20, 0x00, 0x0a, 0xf6, 0x80,
+      0x80, 0x00, 0x38, 0x06, 0x32, 0x0b, 0x72, 0x6d, 0x74, 0x5f, 0x73, 0x74,
+      0x6f, 0x72, 0x61, 0x67, 0x65, 0x42, 0x51, 0x49, 0x4e, 0x46, 0x4f, 0x3a,
+      0x72, 0x6d, 0x74, 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f,
+      0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x68, 0x72, 0x65, 0x61,
+      0x64, 0x3a, 0x20, 0x44, 0x6f, 0x6e, 0x65, 0x20, 0x57, 0x72, 0x69, 0x74,
+      0x65, 0x20, 0x28, 0x62, 0x79, 0x74, 0x65, 0x73, 0x20, 0x3d, 0x20, 0x32,
+      0x30, 0x39, 0x37, 0x31, 0x35, 0x32, 0x29, 0x20, 0x66, 0x6f, 0x72, 0x20,
+      0x2f, 0x62, 0x6f, 0x6f, 0x74, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6d, 0x5f,
+      0x66, 0x73, 0x31, 0x21, 0x28, 0x9e, 0xa9, 0xc5, 0xb3, 0xbe, 0xdc, 0xa5,
+      0xbc, 0x15, 0x08, 0x07, 0x10, 0xf1, 0x0b, 0x18, 0xf1, 0x0b, 0x20, 0x00,
+      0x18, 0x8f, 0x4e, 0x0a, 0x70, 0x40, 0xd2, 0x9f, 0x8f, 0xed, 0x90, 0x55,
+      0xba, 0x02, 0xe0, 0x80, 0x80, 0x00, 0x0a, 0xdb, 0x80, 0x80, 0x00, 0x38,
+      0x04, 0x32, 0x08, 0x70, 0x65, 0x72, 0x66, 0x65, 0x74, 0x74, 0x6f, 0x42,
+      0x39, 0x61, 0x6e, 0x64, 0x72, 0x6f, 0x69, 0x64, 0x5f, 0x6c, 0x6f, 0x67,
+      0x5f, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65,
+      0x2e, 0x63, 0x63, 0x3a, 0x32, 0x39, 0x31, 0x20, 0x53, 0x65, 0x65, 0x6e,
+      0x20, 0x31, 0x31, 0x20, 0x41, 0x6e, 0x64, 0x72, 0x6f, 0x69, 0x64, 0x20,
+      0x6c, 0x6f, 0x67, 0x20, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x28, 0xf4,
+      0x9d, 0x96, 0xd4, 0xbe, 0xdc, 0xa5, 0xbc, 0x15, 0x08, 0x00, 0x10, 0xb5,
+      0x58, 0x18, 0xb5, 0x58, 0x20, 0x00, 0x18, 0x8f, 0x4e, 0x0a, 0x4e, 0x40,
+      0xfe, 0xf3, 0x83, 0xd1, 0x9e, 0x55, 0xba, 0x02, 0xbe, 0x80, 0x80, 0x00,
+      0x0a, 0xb9, 0x80, 0x80, 0x00, 0x38, 0x06, 0x32, 0x00, 0x42, 0x1f, 0x74,
+      0x75, 0x69, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x3a, 0x73, 0x75, 0x69, 0x73,
+      0x76, 0x63, 0x20, 0x20, 0x65, 0x78, 0x69, 0x73, 0x20, 0x61, 0x6c, 0x72,
+      0x65, 0x61, 0x64, 0x79, 0x20, 0x30, 0x28, 0x9f, 0x92, 0xdf, 0xdf, 0xcc,
+      0xdc, 0xa5, 0xbc, 0x15, 0x08, 0x00, 0x10, 0xd9, 0x06, 0x18, 0xd9, 0x06,
+      0x20, 0x00, 0x18, 0x8f, 0x4e, 0x0a, 0x1b, 0x40, 0x9b, 0x83, 0xa7, 0xba,
+      0xba, 0x55, 0xba, 0x02, 0x8b, 0x80, 0x80, 0x00, 0x12, 0x86, 0x80, 0x80,
+      0x00, 0x08, 0x1a, 0x18, 0x00, 0x10, 0x00, 0x18, 0x8f, 0x4e};
+
+  static const uint8_t kFilterBytecode[]{
+      0x0b, 0x01, 0x00, 0x0b, 0x60, 0x13, 0x02, 0x19, 0x23, 0x08, 0x2b, 0x09,
+      0x33, 0x0f, 0x3b, 0x10, 0x41, 0x4b, 0x05, 0x51, 0x5b, 0x12, 0x63, 0x72,
+      0x69, 0x8b, 0x02, 0x21, 0x93, 0x02, 0x33, 0x9b, 0x02, 0x35, 0xa1, 0x02,
+      0xab, 0x02, 0x37, 0xb3, 0x02, 0x07, 0xbb, 0x02, 0x3e, 0xc3, 0x02, 0x3d,
+      0xca, 0x02, 0x02, 0xdb, 0x02, 0x1b, 0xe3, 0x02, 0x11, 0xeb, 0x02, 0x40,
+      0xf3, 0x02, 0x04, 0xfb, 0x02, 0x41, 0x83, 0x03, 0x34, 0x8b, 0x03, 0x42,
+      0x91, 0x03, 0x9b, 0x03, 0x43, 0xa3, 0x03, 0x46, 0xab, 0x03, 0x49, 0xb3,
+      0x03, 0x04, 0xbb, 0x03, 0x07, 0xc3, 0x03, 0x4b, 0xcb, 0x03, 0x4c, 0xd1,
+      0x03, 0xdb, 0x03, 0x73, 0xe3, 0x03, 0x5f, 0xeb, 0x03, 0x5b, 0xf3, 0x03,
+      0x4d, 0xfb, 0x03, 0x04, 0x83, 0x04, 0x5d, 0x8b, 0x04, 0x4e, 0x93, 0x04,
+      0x4f, 0x9b, 0x04, 0x50, 0xa3, 0x04, 0x51, 0xab, 0x04, 0x3c, 0xb3, 0x04,
+      0x0e, 0xbb, 0x04, 0x04, 0xc3, 0x04, 0x69, 0xcb, 0x04, 0x53, 0xd3, 0x04,
+      0x3c, 0xdb, 0x04, 0x04, 0xe3, 0x04, 0x56, 0xeb, 0x04, 0x58, 0xf3, 0x04,
+      0x5a, 0xa3, 0x38, 0x70, 0x00, 0x0b, 0x03, 0x13, 0x04, 0x19, 0x00, 0x0a,
+      0x03, 0x23, 0x04, 0x29, 0x00, 0x0a, 0x03, 0x00, 0x0b, 0x06, 0x11, 0x00,
+      0x0a, 0x0a, 0x5b, 0x07, 0x62, 0x03, 0x00, 0x0a, 0x04, 0x00, 0x0a, 0x02,
+      0x1b, 0x04, 0x00, 0x0b, 0x0a, 0x13, 0x0d, 0x1b, 0x0e, 0x21, 0x2b, 0x0e,
+      0x00, 0x0a, 0x0d, 0x73, 0x0b, 0x7a, 0x02, 0x00, 0x0a, 0x09, 0x53, 0x0c,
+      0x00, 0x0a, 0x02, 0x1b, 0x0c, 0x23, 0x0c, 0x2a, 0x04, 0x00, 0x0a, 0x05,
+      0x00, 0x0a, 0x02, 0x00, 0x0b, 0x07, 0x11, 0x00, 0x0b, 0x0e, 0x13, 0x0e,
+      0x1b, 0x11, 0x22, 0x02, 0x33, 0x0e, 0x39, 0x43, 0x0e, 0x49, 0x53, 0x0e,
+      0x00, 0x0a, 0x08, 0x00, 0x0a, 0x03, 0x23, 0x13, 0x2b, 0x15, 0x33, 0x20,
+      0x42, 0x05, 0x82, 0x01, 0x02, 0xa1, 0x01, 0xab, 0x01, 0x0e, 0xb2, 0x01,
+      0x02, 0xc3, 0x01, 0x16, 0xcb, 0x01, 0x0e, 0xd3, 0x01, 0x15, 0xdb, 0x01,
+      0x0e, 0xe3, 0x01, 0x07, 0xeb, 0x01, 0x1e, 0xf2, 0x01, 0x02, 0x83, 0x02,
+      0x1f, 0x8b, 0x02, 0x07, 0x91, 0x02, 0x9b, 0x02, 0x0e, 0xa1, 0x02, 0xb3,
+      0x02, 0x04, 0xbb, 0x02, 0x15, 0xc3, 0x02, 0x15, 0xcb, 0x02, 0x04, 0xd1,
+      0x02, 0xdb, 0x02, 0x15, 0xe2, 0x02, 0x03, 0x00, 0x0a, 0x07, 0x43, 0x14,
+      0x4a, 0x02, 0x5b, 0x13, 0x63, 0x13, 0x00, 0x0a, 0x02, 0x1b, 0x14, 0x23,
+      0x14, 0x2a, 0x04, 0x00, 0x09, 0x00, 0x0b, 0x17, 0x12, 0x0c, 0x73, 0x19,
+      0x7b, 0x1c, 0x83, 0x01, 0x1d, 0x8b, 0x01, 0x1b, 0x00, 0x0b, 0x0d, 0x13,
+      0x18, 0x00, 0x0a, 0x2e, 0x00, 0x0a, 0x03, 0x23, 0x1a, 0x2b, 0x1a, 0x33,
+      0x1b, 0x00, 0x0a, 0x09, 0x53, 0x07, 0x00, 0x0a, 0x07, 0x00, 0x09, 0x13,
+      0x1a, 0x00, 0x0a, 0x03, 0x23, 0x1a, 0x00, 0x0a, 0x03, 0x23, 0x0e, 0x2a,
+      0x02, 0x00, 0x0a, 0x0a, 0x00, 0x0a, 0x04, 0x32, 0x0a, 0x92, 0x01, 0x02,
+      0x00, 0x0b, 0x22, 0x13, 0x23, 0x1a, 0x03, 0x33, 0x04, 0x3b, 0x07, 0x42,
+      0x03, 0x5b, 0x15, 0x62, 0x03, 0x81, 0x01, 0x8b, 0x01, 0x32, 0x92, 0x01,
+      0x02, 0xa3, 0x01, 0x1b, 0xab, 0x01, 0x15, 0xb2, 0x01, 0x03, 0xcb, 0x01,
+      0x0d, 0xda, 0x01, 0x05, 0x83, 0x02, 0x15, 0x00, 0x09, 0x21, 0x00, 0x0b,
+      0x24, 0x12, 0x02, 0x00, 0x0a, 0x04, 0x32, 0x03, 0xa3, 0x06, 0x25, 0xab,
+      0x06, 0x0d, 0xb3, 0x06, 0x26, 0xbb, 0x06, 0x27, 0xc3, 0x06, 0x1b, 0xcb,
+      0x06, 0x28, 0xd3, 0x06, 0x07, 0xdb, 0x06, 0x2b, 0xe3, 0x06, 0x07, 0xeb,
+      0x06, 0x15, 0xf3, 0x06, 0x2a, 0xfb, 0x06, 0x2c, 0x83, 0x07, 0x0e, 0x8b,
+      0x07, 0x07, 0x93, 0x07, 0x15, 0x9b, 0x07, 0x2f, 0xc1, 0x3e, 0xcb, 0x3e,
+      0x30, 0xf9, 0xff, 0xff, 0xff, 0x07, 0x00, 0x0a, 0x03, 0x52, 0x02, 0x63,
+      0x15, 0x6a, 0x02, 0x00, 0x0a, 0x05, 0x33, 0x0e, 0x00, 0x0a, 0x04, 0x32,
+      0x03, 0x00, 0x0a, 0x02, 0x22, 0x02, 0x33, 0x29, 0x3a, 0x05, 0x6a, 0x0e,
+      0x00, 0x2a, 0x02, 0x00, 0x0a, 0x02, 0x1b, 0x0e, 0x22, 0x04, 0x00, 0x09,
+      0x1a, 0x02, 0x00, 0x0a, 0x0d, 0x7b, 0x2d, 0x83, 0x01, 0x2e, 0x8a, 0x01,
+      0x02, 0x00, 0x0a, 0x02, 0x1b, 0x0e, 0x21, 0x00, 0x0b, 0x0d, 0x11, 0x00,
+      0x09, 0xa3, 0x06, 0x0e, 0x00, 0x0a, 0x05, 0x33, 0x31, 0x00, 0x0a, 0x0e,
+      0x00, 0x09, 0x13, 0x0d, 0x19, 0x00, 0x09, 0x13, 0x34, 0x1a, 0x02, 0x00,
+      0x0a, 0x09, 0x00, 0x0b, 0x36, 0x12, 0x09, 0x5b, 0x0d, 0x00, 0x0a, 0x13,
+      0x00, 0x0b, 0x0e, 0x13, 0x07, 0x1b, 0x0e, 0x23, 0x11, 0x2b, 0x38, 0x32,
+      0x02, 0x00, 0x09, 0x13, 0x3b, 0x1a, 0x02, 0x2b, 0x39, 0x32, 0x09, 0x00,
+      0x0a, 0x03, 0x23, 0x3a, 0x2a, 0x02, 0x00, 0x0b, 0x04, 0x00, 0x0a, 0x06,
+      0x42, 0x02, 0x00, 0x0a, 0x06, 0x00, 0x0b, 0x07, 0x13, 0x04, 0x00, 0x0b,
+      0x3f, 0x13, 0x04, 0x00, 0x0a, 0x08, 0x4b, 0x07, 0x00, 0x0b, 0x07, 0x12,
+      0x02, 0x00, 0x0b, 0x0d, 0x12, 0x02, 0x00, 0x0a, 0x06, 0x3b, 0x0e, 0x42,
+      0x02, 0x00, 0x0b, 0x44, 0x12, 0x02, 0x00, 0x0b, 0x45, 0x13, 0x45, 0x00,
+      0x09, 0x13, 0x04, 0x1b, 0x0e, 0x00, 0x0b, 0x47, 0x13, 0x04, 0x19, 0x00,
+      0x0b, 0x48, 0x13, 0x0d, 0x1a, 0x03, 0x00, 0x0a, 0x03, 0x2a, 0x06, 0x00,
+      0x0a, 0x05, 0x33, 0x0e, 0x3b, 0x4a, 0x42, 0x08, 0x00, 0x0b, 0x0e, 0x13,
+      0x0e, 0x1b, 0x0e, 0x00, 0x09, 0x13, 0x1b, 0x23, 0x0e, 0x2a, 0x02, 0x3b,
+      0x0e, 0x43, 0x0e, 0x4b, 0x11, 0x00, 0x0b, 0x0d, 0x00, 0x0a, 0x08, 0x4b,
+      0x07, 0x82, 0x01, 0x05, 0x00, 0x0b, 0x0d, 0x13, 0x3c, 0x00, 0x0a, 0x06,
+      0x82, 0x01, 0x03, 0x9b, 0x01, 0x15, 0x00, 0x0b, 0x0e, 0x00, 0x09, 0x13,
+      0x52, 0x00, 0x0a, 0x0f, 0x00, 0x0a, 0x02, 0x1b, 0x54, 0x00, 0x09, 0x13,
+      0x55, 0x1b, 0x07, 0x00, 0x0a, 0x04, 0x2b, 0x07, 0x00, 0x0b, 0x04, 0x13,
+      0x11, 0x1b, 0x0d, 0x23, 0x57, 0x2b, 0x15, 0x00, 0x0a, 0x0b, 0x00, 0x0b,
+      0x59, 0x12, 0x02, 0x23, 0x0e, 0x00, 0x0b, 0x07, 0x00, 0x0a, 0x02, 0x1b,
+      0x0e, 0x00, 0x0a, 0x02, 0x1b, 0x5c, 0x00, 0x09, 0x13, 0x04, 0x00, 0x0a,
+      0x02, 0x1b, 0x5e, 0x00, 0x0a, 0x02, 0x1b, 0x0e, 0x23, 0x0e, 0x00, 0x0a,
+      0x02, 0x1b, 0x1b, 0x23, 0x11, 0x29, 0x33, 0x0d, 0x3b, 0x0e, 0x43, 0x3c,
+      0x00, 0x09, 0x13, 0x61, 0x19, 0x23, 0x57, 0x00, 0x0a, 0x02, 0x1b, 0x0e,
+      0x23, 0x1b, 0x5b, 0x0e, 0x63, 0x04, 0x6b, 0x0e, 0x73, 0x04, 0x7b, 0x04,
+      0x83, 0x01, 0x04, 0x8b, 0x01, 0x0d, 0x93, 0x01, 0x04, 0x9b, 0x01, 0x04,
+      0xa3, 0x01, 0x0d, 0xab, 0x01, 0x15, 0xb3, 0x01, 0x15, 0xbb, 0x01, 0x0e,
+      0xc3, 0x01, 0x15, 0xcb, 0x01, 0x15, 0xd3, 0x01, 0x15, 0xdb, 0x01, 0x0d,
+      0xe3, 0x01, 0x3c, 0xeb, 0x01, 0x04, 0xf3, 0x01, 0x3c, 0xfb, 0x01, 0x0d,
+      0x83, 0x02, 0x3c, 0x8b, 0x02, 0x1b, 0x93, 0x02, 0x3c, 0x9b, 0x02, 0x0d,
+      0xa3, 0x02, 0x04, 0xab, 0x02, 0x0e, 0xb3, 0x02, 0x0e, 0xbb, 0x02, 0x0e,
+      0xc3, 0x02, 0x04, 0xcb, 0x02, 0x0d, 0xd3, 0x02, 0x0d, 0xdb, 0x02, 0x07,
+      0xe3, 0x02, 0x04, 0xeb, 0x02, 0x1b, 0xf3, 0x02, 0x04, 0xfb, 0x02, 0x15,
+      0x83, 0x03, 0x0e, 0x8b, 0x03, 0x15, 0x93, 0x03, 0x1b, 0x9b, 0x03, 0x15,
+      0xa3, 0x03, 0x0d, 0xab, 0x03, 0x15, 0xb3, 0x03, 0x15, 0xbb, 0x03, 0x15,
+      0xc3, 0x03, 0x15, 0xcb, 0x03, 0x15, 0xd3, 0x03, 0x0e, 0xdb, 0x03, 0x0d,
+      0xe3, 0x03, 0x15, 0xeb, 0x03, 0x15, 0xf3, 0x03, 0x15, 0xfb, 0x03, 0x15,
+      0x83, 0x04, 0x15, 0x8b, 0x04, 0x04, 0x93, 0x04, 0x0e, 0x9b, 0x04, 0x0d,
+      0xa3, 0x04, 0x04, 0xab, 0x04, 0x04, 0xb3, 0x04, 0x04, 0xbb, 0x04, 0x0d,
+      0xc3, 0x04, 0x04, 0xcb, 0x04, 0x04, 0xd3, 0x04, 0x04, 0xdb, 0x04, 0x04,
+      0xe3, 0x04, 0x0e, 0xeb, 0x04, 0x07, 0xf3, 0x04, 0x07, 0xfb, 0x04, 0x62,
+      0x83, 0x05, 0x04, 0x8b, 0x05, 0x07, 0x93, 0x05, 0x11, 0x9b, 0x05, 0x0d,
+      0xa3, 0x05, 0x62, 0xab, 0x05, 0x0e, 0xb3, 0x05, 0x04, 0xbb, 0x05, 0x1b,
+      0xc3, 0x05, 0x04, 0xcb, 0x05, 0x15, 0xd3, 0x05, 0x15, 0xdb, 0x05, 0x11,
+      0xe3, 0x05, 0x0e, 0xeb, 0x05, 0x0e, 0xf3, 0x05, 0x1f, 0xfb, 0x05, 0x04,
+      0x83, 0x06, 0x15, 0x8b, 0x06, 0x0d, 0x93, 0x06, 0x0d, 0x9b, 0x06, 0x0d,
+      0xa3, 0x06, 0x3c, 0xab, 0x06, 0x3c, 0xb3, 0x06, 0x3c, 0xbb, 0x06, 0x3c,
+      0xc3, 0x06, 0x07, 0xcb, 0x06, 0x07, 0xd3, 0x06, 0x07, 0xdb, 0x06, 0x15,
+      0xe3, 0x06, 0x04, 0xeb, 0x06, 0x0e, 0xf3, 0x06, 0x07, 0xfb, 0x06, 0x04,
+      0x83, 0x07, 0x04, 0x8b, 0x07, 0x04, 0x93, 0x07, 0x0d, 0x9b, 0x07, 0x0d,
+      0xa3, 0x07, 0x0d, 0xab, 0x07, 0x0d, 0xb3, 0x07, 0x0d, 0xbb, 0x07, 0x0d,
+      0xc3, 0x07, 0x3c, 0xcb, 0x07, 0x04, 0xd3, 0x07, 0x0d, 0xdb, 0x07, 0x15,
+      0xe3, 0x07, 0x3c, 0xeb, 0x07, 0x3c, 0xf3, 0x07, 0x1b, 0x83, 0x08, 0x1b,
+      0x8b, 0x08, 0x3c, 0x93, 0x08, 0x0d, 0x9b, 0x08, 0x0d, 0xa3, 0x08, 0x04,
+      0xab, 0x08, 0x0e, 0xb3, 0x08, 0x07, 0xbb, 0x08, 0x57, 0xc3, 0x08, 0x07,
+      0xcb, 0x08, 0x04, 0xd3, 0x08, 0x07, 0xdb, 0x08, 0x11, 0xe3, 0x08, 0x1b,
+      0xeb, 0x08, 0x34, 0xf3, 0x08, 0x1f, 0xfb, 0x08, 0x0d, 0x83, 0x09, 0x0d,
+      0x8b, 0x09, 0x3c, 0x93, 0x09, 0x04, 0x9b, 0x09, 0x0e, 0xa3, 0x09, 0x04,
+      0xab, 0x09, 0x3c, 0xb3, 0x09, 0x04, 0xbb, 0x09, 0x3c, 0xc3, 0x09, 0x3c,
+      0xcb, 0x09, 0x04, 0xd3, 0x09, 0x1b, 0xdb, 0x09, 0x07, 0xe3, 0x09, 0x0d,
+      0xeb, 0x09, 0x04, 0xf3, 0x09, 0x04, 0xfb, 0x09, 0x04, 0x83, 0x0a, 0x04,
+      0x8b, 0x0a, 0x1b, 0x93, 0x0a, 0x1f, 0x9b, 0x0a, 0x11, 0xa3, 0x0a, 0x07,
+      0xab, 0x0a, 0x07, 0xb3, 0x0a, 0x0d, 0xbb, 0x0a, 0x11, 0xc3, 0x0a, 0x0d,
+      0xcb, 0x0a, 0x0d, 0xd3, 0x0a, 0x1b, 0xdb, 0x0a, 0x04, 0xe3, 0x0a, 0x1b,
+      0xeb, 0x0a, 0x0d, 0xf3, 0x0a, 0x3c, 0xfb, 0x0a, 0x0d, 0x83, 0x0b, 0x1b,
+      0x8b, 0x0b, 0x0d, 0x93, 0x0b, 0x3c, 0x9b, 0x0b, 0x3c, 0xa3, 0x0b, 0x3c,
+      0xab, 0x0b, 0x07, 0xb3, 0x0b, 0x0d, 0xbb, 0x0b, 0x11, 0xc3, 0x0b, 0x07,
+      0xcb, 0x0b, 0x0d, 0xd3, 0x0b, 0x0d, 0xdb, 0x0b, 0x04, 0xe3, 0x0b, 0x0d,
+      0xeb, 0x0b, 0x0d, 0xf3, 0x0b, 0x0e, 0xfb, 0x0b, 0x0e, 0x83, 0x0c, 0x04,
+      0x8b, 0x0c, 0x0e, 0x93, 0x0c, 0x0e, 0x9b, 0x0c, 0x0e, 0xa3, 0x0c, 0x0d,
+      0xab, 0x0c, 0x0d, 0xb3, 0x0c, 0x04, 0xbb, 0x0c, 0x07, 0xc3, 0x0c, 0x63,
+      0xcb, 0x0c, 0x0d, 0xd3, 0x0c, 0x0d, 0xdb, 0x0c, 0x1f, 0xe3, 0x0c, 0x3c,
+      0xeb, 0x0c, 0x0d, 0xf3, 0x0c, 0x0e, 0xfb, 0x0c, 0x04, 0x83, 0x0d, 0x04,
+      0x8b, 0x0d, 0x11, 0x93, 0x0d, 0x1f, 0x9b, 0x0d, 0x04, 0xa3, 0x0d, 0x0e,
+      0xab, 0x0d, 0x0d, 0xb3, 0x0d, 0x0d, 0xbb, 0x0d, 0x04, 0xc3, 0x0d, 0x04,
+      0xcb, 0x0d, 0x07, 0xd3, 0x0d, 0x04, 0xdb, 0x0d, 0x0d, 0xb3, 0x0e, 0x0d,
+      0xbb, 0x0e, 0x04, 0xc3, 0x0e, 0x1f, 0xcb, 0x0e, 0x1b, 0xd3, 0x0e, 0x0d,
+      0xdb, 0x0e, 0x07, 0xe3, 0x0e, 0x07, 0xeb, 0x0e, 0x04, 0xf3, 0x0e, 0x07,
+      0xfb, 0x0e, 0x07, 0x83, 0x0f, 0x04, 0x8b, 0x0f, 0x0e, 0x93, 0x0f, 0x04,
+      0x9b, 0x0f, 0x0d, 0xa3, 0x0f, 0x11, 0xab, 0x0f, 0x11, 0xb3, 0x0f, 0x3c,
+      0xbb, 0x0f, 0x1f, 0xc3, 0x0f, 0x11, 0xcb, 0x0f, 0x04, 0xd3, 0x0f, 0x04,
+      0xdb, 0x0f, 0x0d, 0xe3, 0x0f, 0x04, 0xeb, 0x0f, 0x3c, 0xf3, 0x0f, 0x0d,
+      0xfb, 0x0f, 0x11, 0x83, 0x10, 0x0d, 0x8b, 0x10, 0x04, 0x93, 0x10, 0x11,
+      0x9b, 0x10, 0x0d, 0xa3, 0x10, 0x04, 0xab, 0x10, 0x0d, 0xb3, 0x10, 0x0d,
+      0xbb, 0x10, 0x04, 0xc3, 0x10, 0x07, 0xcb, 0x10, 0x07, 0xd3, 0x10, 0x04,
+      0xdb, 0x10, 0x0d, 0xe3, 0x10, 0x0d, 0xeb, 0x10, 0x04, 0xf3, 0x10, 0x3c,
+      0xfb, 0x10, 0x0d, 0x83, 0x11, 0x04, 0x8b, 0x11, 0x0d, 0x93, 0x11, 0x0e,
+      0x9b, 0x11, 0x0e, 0xa3, 0x11, 0x0e, 0xab, 0x11, 0x0e, 0xb3, 0x11, 0x0e,
+      0xbb, 0x11, 0x0e, 0xc3, 0x11, 0x15, 0xcb, 0x11, 0x07, 0xd3, 0x11, 0x0d,
+      0xdb, 0x11, 0x0d, 0xe3, 0x11, 0x0d, 0xeb, 0x11, 0x3c, 0xf3, 0x11, 0x3c,
+      0xfb, 0x11, 0x0d, 0x83, 0x12, 0x15, 0x8b, 0x12, 0x07, 0x93, 0x12, 0x07,
+      0x9b, 0x12, 0x15, 0xa3, 0x12, 0x04, 0xab, 0x12, 0x04, 0xb3, 0x12, 0x07,
+      0xbb, 0x12, 0x07, 0xc3, 0x12, 0x0e, 0xcb, 0x12, 0x0e, 0xd3, 0x12, 0x0e,
+      0xdb, 0x12, 0x0d, 0xe3, 0x12, 0x3c, 0xeb, 0x12, 0x0d, 0xf3, 0x12, 0x3c,
+      0xfb, 0x12, 0x0e, 0x83, 0x13, 0x15, 0x8b, 0x13, 0x15, 0x93, 0x13, 0x15,
+      0x9b, 0x13, 0x0d, 0xa3, 0x13, 0x1b, 0xab, 0x13, 0x07, 0xb3, 0x13, 0x04,
+      0xbb, 0x13, 0x04, 0xc3, 0x13, 0x07, 0xcb, 0x13, 0x07, 0xd3, 0x13, 0x04,
+      0xdb, 0x13, 0x04, 0xe3, 0x13, 0x07, 0xeb, 0x13, 0x07, 0xf3, 0x13, 0x07,
+      0xfb, 0x13, 0x07, 0x83, 0x14, 0x15, 0x8b, 0x14, 0x15, 0x93, 0x14, 0x0e,
+      0x9b, 0x14, 0x04, 0xa3, 0x14, 0x04, 0xab, 0x14, 0x3c, 0xb3, 0x14, 0x04,
+      0xbb, 0x14, 0x64, 0xc3, 0x14, 0x07, 0xcb, 0x14, 0x15, 0xd3, 0x14, 0x0e,
+      0xdb, 0x14, 0x07, 0xe3, 0x14, 0x0e, 0xeb, 0x14, 0x0d, 0xf3, 0x14, 0x15,
+      0xfb, 0x14, 0x04, 0x83, 0x15, 0x0e, 0x8b, 0x15, 0x0e, 0x93, 0x15, 0x04,
+      0x9b, 0x15, 0x66, 0xa3, 0x15, 0x04, 0xab, 0x15, 0x07, 0xb3, 0x15, 0x0e,
+      0xbb, 0x15, 0x07, 0xc3, 0x15, 0x07, 0xcb, 0x15, 0x07, 0xd3, 0x15, 0x07,
+      0xdb, 0x15, 0x04, 0xe3, 0x15, 0x3c, 0xeb, 0x15, 0x67, 0xf3, 0x15, 0x07,
+      0xfb, 0x15, 0x04, 0x83, 0x16, 0x07, 0x8b, 0x16, 0x07, 0x93, 0x16, 0x04,
+      0x9b, 0x16, 0x11, 0xa3, 0x16, 0x68, 0xab, 0x16, 0x3c, 0xb3, 0x16, 0x07,
+      0x00, 0x0a, 0x10, 0x00, 0x0a, 0x14, 0x00, 0x09, 0x13, 0x65, 0x00, 0x09,
+      0x1a, 0x03, 0x00, 0x00, 0x09, 0x22, 0x03, 0x00, 0x0a, 0x0c, 0x00, 0x0b,
+      0x6a, 0x00, 0x0b, 0x6b, 0x00, 0x0a, 0x03, 0x23, 0x6c, 0x2b, 0x6e, 0x3b,
+      0x6d, 0x52, 0x02, 0x00, 0x09, 0x13, 0x6d, 0x1b, 0x6c, 0x23, 0x6e, 0x33,
+      0x6d, 0x43, 0x6f, 0x4b, 0x0e, 0x51, 0x00, 0x0a, 0x07, 0x49, 0x00, 0x09,
+      0x13, 0x0e, 0x29, 0x00, 0x09, 0x13, 0x66, 0x00, 0x0a, 0x04, 0x2b, 0x71,
+      0x00, 0x09, 0x13, 0x71, 0x1a, 0x04, 0x00, 0x0b, 0x0e, 0x13, 0x0e, 0x1b,
+      0x0e, 0x23, 0x07, 0x2b, 0x0e, 0x33, 0x07, 0x3b, 0x0e, 0x83, 0x01, 0x0e,
+      0x8b, 0x01, 0x0e, 0x93, 0x01, 0x0e, 0x9b, 0x01, 0x11, 0xa3, 0x01, 0x0e,
+      0xab, 0x01, 0x07, 0xb3, 0x01, 0x0e, 0xbb, 0x01, 0x04, 0xc3, 0x01, 0x07,
+      0xcb, 0x01, 0x0e, 0xd3, 0x01, 0x0e, 0x00, 0x5b, 0x74, 0x63, 0x75, 0xd1,
+      0x03, 0x00, 0x59, 0xf9, 0x01, 0xe9, 0x02, 0x00, 0x0b, 0x2d, 0x00, 0x89,
+      0x8c, 0xb0, 0x80, 0x08};
+
+  MessageFilter filt;
+  ASSERT_TRUE(
+      filt.LoadFilterBytecode(kFilterBytecode, sizeof(kFilterBytecode)));
+
+  // Pass the trace in input splitting it in slices of arbitrary size.
+  std::vector<MessageFilter::InputSlice> input_slices;
+  for (size_t i = 0; i < sizeof(kTraceData);) {
+    std::minstd_rand0 rnd_engine(0);
+    size_t slice_size = rnd_engine() % 4096;
+    slice_size = std::min(slice_size, sizeof(kTraceData) - i);
+    input_slices.emplace_back(
+        MessageFilter::InputSlice{kTraceData + i, slice_size});
+    i += slice_size;
+  }
+
+  auto filtered_data =
+      filt.FilterMessageFragments(input_slices.data(), input_slices.size());
+
+  EXPECT_GT(filtered_data.size, 0u);
+  EXPECT_LE(filtered_data.size, sizeof(kTraceData));
+
+  perfetto::protos::Trace original_trace;
+  ASSERT_TRUE(original_trace.ParseFromArray(kTraceData, sizeof(kTraceData)));
+
+  perfetto::protos::Trace filtered_trace;
+  ASSERT_TRUE(filtered_trace.ParseFromArray(
+      filtered_data.data.get(), static_cast<int>(filtered_data.size)));
+
+  // Check that the re-serialized traces are identical.
+  std::string original_ser = original_trace.SerializeAsString();
+  std::string filter_ser = filtered_trace.SerializeAsString();
+
+  EXPECT_EQ(filtered_trace.packet_size(), original_trace.packet_size());
+
+  // Don't use EXPECT_EQ, the string is too big. If this check fails, the gtest
+  // diffing algorithm will take several minutes to compute the diff.
+  // That would mistakenly look like a CI or timing-related issue as the gtest
+  // would fail due to timeout.
+  // If this check fails, use base::HexDump() to investigate.
+  EXPECT_TRUE(original_ser == filter_ser);
+}
+
+}  // namespace
+}  // namespace protozero
diff --git a/tools/BUILD.gn b/tools/BUILD.gn
index a8342f9..772f6a2 100644
--- a/tools/BUILD.gn
+++ b/tools/BUILD.gn
@@ -24,6 +24,7 @@
     ":copy_protoc",
     "compact_reencode",
     "ftrace_proto_gen",
+    "proto_filter",
     "protoprofile",
   ]
   if (is_linux || is_android) {
diff --git a/tools/proto_filter/BUILD.gn b/tools/proto_filter/BUILD.gn
new file mode 100644
index 0000000..b2b6997
--- /dev/null
+++ b/tools/proto_filter/BUILD.gn
@@ -0,0 +1,29 @@
+# Copyright (C) 2021 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_host_executable.gni")
+
+perfetto_host_executable("proto_filter") {
+  testonly = true
+  deps = [
+    "../../gn:default_deps",
+    "../../gn:protobuf_full",
+    "../../src/base",
+    "../../src/protozero",
+    "../../src/protozero/filtering:bytecode_generator",
+    "../../src/protozero/filtering:filter_util",
+    "../../src/protozero/filtering:message_filter",
+  ]
+  sources = [ "proto_filter.cc" ]
+}
diff --git a/tools/proto_filter/proto_filter.cc b/tools/proto_filter/proto_filter.cc
new file mode 100644
index 0000000..76bd7de
--- /dev/null
+++ b/tools/proto_filter/proto_filter.cc
@@ -0,0 +1,287 @@
+/*
+ * Copyright (C) 2021 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/ext/base/file_utils.h"
+#include "perfetto/ext/base/getopt.h"
+#include "perfetto/ext/base/scoped_file.h"
+#include "perfetto/ext/base/string_utils.h"
+#include "perfetto/ext/base/version.h"
+#include "src/protozero/filtering/filter_util.h"
+#include "src/protozero/filtering/message_filter.h"
+
+namespace perfetto {
+namespace proto_filter {
+namespace {
+
+const char kUsage[] =
+    R"(Usage: proto_filter [-s schema_in] [-i message in] [-o message out] [-f filter in] [-F filter out] [-T filter_oct_out] [--no_dedupe] [-I proto include path] [-r root message]
+
+-s --schema-in:      Path to the root .proto file. Required for most operations
+-I --proto_path:     Extra include directory for proto includes. If omitted assumed CWD.
+-r --root_message:   Fully qualified name for the root proto message (e.g. perfetto.protos.Trace)
+                     If omitted the first message defined in the schema will be used.
+-i --msg_in:         Path of a binary-encoded proto message which will be filtered.
+-o --msg_out:        Path of the binary-encoded filtered proto message written in output.
+-f --filter_in:      Path of a filter bytecode file previously generated by this tool.
+-F --filter_out:     Path of the filter bytecode file generated from the --schema-in definition.
+-T --filter_oct_out: Like --filter_out, but emits a octal-escaped C string suitable for .pbtx.
+
+Example usage:
+
+# Convert a .proto schema file into a diff-friendly list of messages/fields>
+
+  proto_filter -r perfetto.protos.Trace -s protos/perfetto/trace/trace.proto
+
+# Generate the filter bytecode from a .proto schema
+
+  proto_filter -r perfetto.protos.Trace -s protos/perfetto/trace/trace.proto \
+               -F /tmp/bytecode [--no-dedupe]
+
+# List the used/filtered fields from a trace file
+
+  proto_filter -r perfetto.protos.Trace -s protos/perfetto/trace/trace.proto \
+               -i test/data/example_android_trace_30s.pb -f /tmp/bytecode
+
+# Filter a trace using a filter bytecode
+
+  proto_filter -i test/data/example_android_trace_30s.pb -f /tmp/bytecode \
+               -o /tmp/filtered_trace
+)";
+
+int Main(int argc, char** argv) {
+  static const option long_options[] = {
+      {"help", no_argument, nullptr, 'h'},
+      {"version", no_argument, nullptr, 'v'},
+      {"no_dedupe", no_argument, nullptr, 'd'},
+      {"proto_path", no_argument, nullptr, 'I'},
+      {"schema_in", no_argument, nullptr, 's'},
+      {"root_message", no_argument, nullptr, 'r'},
+      {"msg_in", no_argument, nullptr, 'i'},
+      {"msg_out", no_argument, nullptr, 'o'},
+      {"filter_in", no_argument, nullptr, 'f'},
+      {"filter_out", no_argument, nullptr, 'F'},
+      {"filter_oct_out", no_argument, nullptr, 'T'},
+      {nullptr, 0, nullptr, 0}};
+
+  std::string msg_in;
+  std::string msg_out;
+  std::string filter_in;
+  std::string schema_in;
+  std::string filter_out;
+  std::string filter_oct_out;
+  std::string proto_path;
+  std::string root_message_arg;
+  bool dedupe = true;
+
+  for (;;) {
+    int option =
+        getopt_long(argc, argv, "hvdI:s:r:i:o:f:F:T:", long_options, nullptr);
+
+    if (option == -1)
+      break;  // EOF.
+
+    if (option == 'v') {
+      printf("%s\n", base::GetVersionString());
+      exit(0);
+    }
+
+    if (option == 'd') {
+      dedupe = false;
+      continue;
+    }
+
+    if (option == 'I') {
+      proto_path = optarg;
+      continue;
+    }
+
+    if (option == 's') {
+      schema_in = optarg;
+      continue;
+    }
+
+    if (option == 'r') {
+      root_message_arg = optarg;
+      continue;
+    }
+
+    if (option == 'i') {
+      msg_in = optarg;
+      continue;
+    }
+
+    if (option == 'o') {
+      msg_out = optarg;
+      continue;
+    }
+
+    if (option == 'f') {
+      filter_in = optarg;
+      continue;
+    }
+
+    if (option == 'F') {
+      filter_out = optarg;
+      continue;
+    }
+
+    if (option == 'T') {
+      filter_oct_out = optarg;
+      continue;
+    }
+
+    if (option == 'h') {
+      fprintf(stdout, kUsage);
+      exit(0);
+    }
+
+    fprintf(stderr, kUsage);
+    exit(1);
+  }
+
+  if (msg_in.empty() && filter_in.empty() && schema_in.empty()) {
+    fprintf(stderr, kUsage);
+    return 1;
+  }
+
+  std::string msg_in_data;
+  if (!msg_in.empty()) {
+    PERFETTO_LOG("Loading proto-encoded message from %s", msg_in.c_str());
+    if (!base::ReadFile(msg_in, &msg_in_data)) {
+      PERFETTO_ELOG("Could not open message file %s", msg_in.c_str());
+      return 1;
+    }
+  }
+
+  protozero::FilterUtil filter;
+  if (!schema_in.empty()) {
+    PERFETTO_LOG("Loading proto schema from %s", schema_in.c_str());
+    if (!filter.LoadMessageDefinition(schema_in, root_message_arg,
+                                      proto_path)) {
+      PERFETTO_ELOG("Failed to parse proto schema from %s", schema_in.c_str());
+      return 1;
+    }
+    if (dedupe)
+      filter.Dedupe();
+  }
+
+  protozero::MessageFilter msg_filter;
+  std::string filter_data;
+  std::string filter_data_src;
+  if (!filter_in.empty()) {
+    PERFETTO_LOG("Loading filter bytecode from %s", filter_in.c_str());
+    if (!base::ReadFile(filter_in, &filter_data)) {
+      PERFETTO_ELOG("Could not open filter file %s", filter_in.c_str());
+      return 1;
+    }
+    filter_data_src = filter_in;
+  } else if (!schema_in.empty()) {
+    PERFETTO_LOG("Generating filter bytecode from %s", schema_in.c_str());
+    filter_data = filter.GenerateFilterBytecode();
+    filter_data_src = schema_in;
+  }
+
+  if (!filter_data.empty()) {
+    const uint8_t* data = reinterpret_cast<const uint8_t*>(filter_data.data());
+    if (!msg_filter.LoadFilterBytecode(data, filter_data.size())) {
+      PERFETTO_ELOG("Failed to parse filter bytecode from %s",
+                    filter_data_src.c_str());
+      return 1;
+    }
+  }
+
+  // Write the filter bytecode in output.
+  if (!filter_out.empty()) {
+    auto fd = base::OpenFile(filter_out, O_WRONLY | O_TRUNC | O_CREAT, 0644);
+    if (!fd) {
+      PERFETTO_ELOG("Could not open filter out path %s", filter_out.c_str());
+      return 1;
+    }
+    PERFETTO_LOG("Writing filter bytecode (%zu bytes) into %s",
+                 filter_data.size(), filter_out.c_str());
+    base::WriteAll(*fd, filter_data.data(), filter_data.size());
+  }
+
+  if (!filter_oct_out.empty()) {
+    auto fd =
+        base::OpenFile(filter_oct_out, O_WRONLY | O_TRUNC | O_CREAT, 0644);
+    if (!fd) {
+      PERFETTO_ELOG("Could not open filter out path %s",
+                    filter_oct_out.c_str());
+      return 1;
+    }
+    std::string oct_str;
+    oct_str.reserve(filter_data.size() * 4 + 64);
+    oct_str.append("trace_filter{\n  bytecode: \"");
+    for (char c : filter_data) {
+      uint8_t octect = static_cast<uint8_t>(c);
+      char buf[5]{'\\', '0', '0', '0', 0};
+      for (uint8_t i = 0; i < 3; ++i) {
+        buf[3 - i] = static_cast<char>('0' + static_cast<uint8_t>(octect) % 8);
+        octect /= 8;
+      }
+      oct_str.append(buf);
+    }
+    oct_str.append("\"\n}\n");
+    PERFETTO_LOG("Writing filter bytecode (%zu bytes) into %s", oct_str.size(),
+                 filter_oct_out.c_str());
+    base::WriteAll(*fd, oct_str.data(), oct_str.size());
+  }
+
+  // Apply the filter to the input message (if any).
+  std::vector<uint8_t> msg_filtered_data;
+  if (!msg_in.empty()) {
+    PERFETTO_LOG("Applying filter %s to proto message %s",
+                 filter_data_src.c_str(), msg_in.c_str());
+    msg_filter.enable_field_usage_tracking(true);
+    auto res = msg_filter.FilterMessage(msg_in_data.data(), msg_in_data.size());
+    if (res.error)
+      PERFETTO_FATAL("Filtering failed");
+    msg_filtered_data.insert(msg_filtered_data.end(), res.data.get(),
+                             res.data.get() + res.size);
+  }
+
+  // Write out the filtered message.
+  if (!msg_out.empty()) {
+    PERFETTO_LOG("Writing filtered proto bytes (%zu bytes) into %s",
+                 msg_filtered_data.size(), msg_out.c_str());
+    auto fd = base::OpenFile(msg_out, O_WRONLY | O_CREAT, 0644);
+    base::WriteAll(*fd, msg_filtered_data.data(), msg_filtered_data.size());
+  }
+
+  if (!msg_in.empty()) {
+    const auto& field_usage_map = msg_filter.field_usage();
+    for (const auto& it : field_usage_map) {
+      const std::string& field_path_varint = it.first;
+      int32_t num_occurrences = it.second;
+      std::string path_str = filter.LookupField(field_path_varint);
+      printf("%-100s %s %d\n", path_str.c_str(),
+             num_occurrences < 0 ? "DROP" : "PASS", std::abs(num_occurrences));
+    }
+  } else if (!schema_in.empty()) {
+    filter.PrintAsText();
+  }
+
+  return 0;
+}
+
+}  // namespace
+}  // namespace proto_filter
+}  // namespace perfetto
+
+int main(int argc, char** argv) {
+  return perfetto::proto_filter::Main(argc, argv);
+}
diff --git a/tools/run_android_test b/tools/run_android_test
index 64cc560..5f38951 100755
--- a/tools/run_android_test
+++ b/tools/run_android_test
@@ -143,8 +143,10 @@
   AdbCall('root')
   AdbCall('wait-for-device')
 
-  target_dir = '/data/local/tmp/' + args.test_name
-  AdbCall('shell', 'rm -rf "%s"; mkdir -p "%s"' % (2 * (target_dir,)))
+  target_dir = '/data/local/tmp/perfetto_tests'
+  if not args.no_cleanup:
+    AdbCall('shell', 'rm -rf "%s"' % target_dir)
+  AdbCall('shell', 'mkdir -p "%s"' % target_dir)
   # Some tests require the trace directory to exist, while true for android
   # devices in general some emulators might not have it set up. So we check to
   # see if it exists, and if not create it.