Add log based GoogCC simulation to visualizer.

This CL adds  a mode to simulate roughly what GoogCC could have been
doing during the recording of an rtc event log by using the logged
events as input to GoogCC and visualizing the resulting target rate.

This is similar to the existing simulated_sendside_bwe mode, but uses
the new NetworkControllerInterface to ensure more reliable GoogCC
simulation.

Bug: None
Change-Id: I57894aa666151efc8405407d928b5257fb9b7d61
Reviewed-on: https://webrtc-review.googlesource.com/c/src/+/123924
Reviewed-by: Björn Terelius <terelius@webrtc.org>
Cr-Commit-Position: refs/heads/master@{#27095}
diff --git a/logging/rtc_event_log/logged_events.h b/logging/rtc_event_log/logged_events.h
index 4bcd937..afe5127 100644
--- a/logging/rtc_event_log/logged_events.h
+++ b/logging/rtc_event_log/logged_events.h
@@ -491,6 +491,8 @@
 };
 
 struct LoggedRouteChangeEvent {
+  int64_t log_time_ms() const { return log_time.ms(); }
+  int64_t log_time_us() const { return log_time.us(); }
   uint32_t route_id;
   Timestamp log_time = Timestamp::MinusInfinity();
   uint16_t send_overhead;
@@ -506,6 +508,8 @@
                    Timestamp capture_time);
   LoggedPacketInfo(const LoggedPacketInfo&);
   ~LoggedPacketInfo();
+  int64_t log_time_ms() const { return log_packet_time.ms(); }
+  int64_t log_time_us() const { return log_packet_time.us(); }
   uint32_t ssrc;
   uint16_t stream_seq_no;
   uint16_t size;
diff --git a/rtc_tools/BUILD.gn b/rtc_tools/BUILD.gn
index 08e9c27..7bdda7d 100644
--- a/rtc_tools/BUILD.gn
+++ b/rtc_tools/BUILD.gn
@@ -285,6 +285,8 @@
       sources = [
         "event_log_visualizer/analyzer.cc",
         "event_log_visualizer/analyzer.h",
+        "event_log_visualizer/log_simulation.cc",
+        "event_log_visualizer/log_simulation.h",
         "event_log_visualizer/plot_base.cc",
         "event_log_visualizer/plot_base.h",
         "event_log_visualizer/plot_protobuf.cc",
@@ -302,6 +304,7 @@
         "../api/audio_codecs:audio_codecs_api",
         "../api/transport:field_trial_based_config",
         "../api/transport:goog_cc",
+        "../api/transport:network_control",
         "../call:call_interfaces",
         "../call:video_stream_api",
         "../logging:rtc_event_log_api",
diff --git a/rtc_tools/event_log_visualizer/analyzer.cc b/rtc_tools/event_log_visualizer/analyzer.cc
index b8eff1d..7f07329 100644
--- a/rtc_tools/event_log_visualizer/analyzer.cc
+++ b/rtc_tools/event_log_visualizer/analyzer.cc
@@ -59,6 +59,7 @@
 #include "rtc_base/numerics/sequence_number_util.h"
 #include "rtc_base/rate_statistics.h"
 #include "rtc_base/strings/string_builder.h"
+#include "rtc_tools/event_log_visualizer/log_simulation.h"
 
 #ifndef BWE_TEST_LOGGING_COMPILE_TIME_ENABLE
 #define BWE_TEST_LOGGING_COMPILE_TIME_ENABLE 0
@@ -1156,6 +1157,51 @@
     plot->SetTitle("Target bitrate per outgoing layer");
 }
 
+void EventLogAnalyzer::CreateGoogCcSimulationGraph(Plot* plot) {
+  TimeSeries target_rates("Simulated target rate", LineStyle::kStep,
+                          PointStyle::kHighlight);
+  TimeSeries delay_based("Logged delay-based estimate", LineStyle::kStep,
+                         PointStyle::kHighlight);
+  TimeSeries loss_based("Logged loss-based estimate", LineStyle::kStep,
+                        PointStyle::kHighlight);
+  TimeSeries probe_results("Logged probe success", LineStyle::kNone,
+                           PointStyle::kHighlight);
+
+  RtcEventLogNullImpl null_event_log;
+  LogBasedNetworkControllerSimulation simulation(
+      absl::make_unique<GoogCcNetworkControllerFactory>(&null_event_log),
+      [&](const NetworkControlUpdate& update, Timestamp at_time) {
+        if (update.target_rate) {
+          target_rates.points.emplace_back(
+              config_.GetCallTimeSec(at_time.us()),
+              update.target_rate->target_rate.kbps<float>());
+        }
+      });
+
+  simulation.ProcessEventsInLog(parsed_log_);
+  for (const auto& logged : parsed_log_.bwe_delay_updates())
+    delay_based.points.emplace_back(
+        config_.GetCallTimeSec(logged.log_time_us()),
+        logged.bitrate_bps / 1000);
+  for (const auto& logged : parsed_log_.bwe_probe_success_events())
+    probe_results.points.emplace_back(
+        config_.GetCallTimeSec(logged.log_time_us()),
+        logged.bitrate_bps / 1000);
+  for (const auto& logged : parsed_log_.bwe_loss_updates())
+    loss_based.points.emplace_back(config_.GetCallTimeSec(logged.log_time_us()),
+                                   logged.bitrate_bps / 1000);
+
+  plot->AppendTimeSeries(std::move(delay_based));
+  plot->AppendTimeSeries(std::move(loss_based));
+  plot->AppendTimeSeries(std::move(probe_results));
+  plot->AppendTimeSeries(std::move(target_rates));
+
+  plot->SetXAxis(config_.CallBeginTimeSec(), config_.CallEndTimeSec(),
+                 "Time (s)", kLeftMargin, kRightMargin);
+  plot->SetSuggestedYAxis(0, 10, "Bitrate (kbps)", kBottomMargin, kTopMargin);
+  plot->SetTitle("Simulated BWE behavior");
+}
+
 void EventLogAnalyzer::CreateSendSideBweSimulationGraph(Plot* plot) {
   using RtpPacketType = LoggedRtpPacketOutgoing;
   using TransportFeedbackType = LoggedRtcpPacketTransportFeedback;
diff --git a/rtc_tools/event_log_visualizer/analyzer.h b/rtc_tools/event_log_visualizer/analyzer.h
index e0ef0c3..2432b4d 100644
--- a/rtc_tools/event_log_visualizer/analyzer.h
+++ b/rtc_tools/event_log_visualizer/analyzer.h
@@ -83,6 +83,7 @@
   void CreateStreamBitrateGraph(PacketDirection direction, Plot* plot);
   void CreateBitrateAllocationGraph(PacketDirection direction, Plot* plot);
 
+  void CreateGoogCcSimulationGraph(Plot* plot);
   void CreateSendSideBweSimulationGraph(Plot* plot);
   void CreateReceiveSideBweSimulationGraph(Plot* plot);
 
diff --git a/rtc_tools/event_log_visualizer/log_simulation.cc b/rtc_tools/event_log_visualizer/log_simulation.cc
new file mode 100644
index 0000000..4778cc5
--- /dev/null
+++ b/rtc_tools/event_log_visualizer/log_simulation.cc
@@ -0,0 +1,199 @@
+/*
+ *  Copyright 2019 The WebRTC project authors. All Rights Reserved.
+ *
+ *  Use of this source code is governed by a BSD-style license
+ *  that can be found in the LICENSE file in the root of the source
+ *  tree. An additional intellectual property rights grant can be found
+ *  in the file PATENTS.  All contributing project authors may
+ *  be found in the AUTHORS file in the root of the source tree.
+ */
+#include "rtc_tools/event_log_visualizer/log_simulation.h"
+
+#include <algorithm>
+#include <utility>
+
+#include "logging/rtc_event_log/rtc_event_processor.h"
+#include "modules/rtp_rtcp/source/time_util.h"
+
+namespace webrtc {
+
+LogBasedNetworkControllerSimulation::LogBasedNetworkControllerSimulation(
+    std::unique_ptr<NetworkControllerFactoryInterface> factory,
+    std::function<void(const NetworkControlUpdate&, Timestamp)> update_handler)
+    : update_handler_(update_handler), factory_(std::move(factory)) {}
+
+LogBasedNetworkControllerSimulation::~LogBasedNetworkControllerSimulation() {}
+
+void LogBasedNetworkControllerSimulation::HandleStateUpdate(
+    const NetworkControlUpdate& update) {
+  update_handler_(update, current_time_);
+}
+
+void LogBasedNetworkControllerSimulation::ProcessUntil(Timestamp to_time) {
+  if (last_process_.IsInfinite()) {
+    NetworkControllerConfig config;
+    config.constraints.at_time = to_time;
+    config.constraints.min_data_rate = DataRate::kbps(30);
+    config.constraints.starting_rate = DataRate::kbps(300);
+    controller_ = factory_->Create(config);
+  }
+  if (last_process_.IsInfinite() ||
+      to_time - last_process_ > TimeDelta::seconds(1)) {
+    last_process_ = to_time;
+    current_time_ = to_time;
+    ProcessInterval msg;
+    msg.at_time = to_time;
+    HandleStateUpdate(controller_->OnProcessInterval(msg));
+  } else {
+    while (last_process_ + factory_->GetProcessInterval() <= to_time) {
+      last_process_ += factory_->GetProcessInterval();
+      current_time_ = last_process_;
+      ProcessInterval msg;
+      msg.at_time = current_time_;
+      HandleStateUpdate(controller_->OnProcessInterval(msg));
+    }
+    current_time_ = to_time;
+  }
+}
+
+void LogBasedNetworkControllerSimulation::OnProbeCreated(
+    const LoggedBweProbeClusterCreatedEvent& probe_cluster) {
+  pending_probes_.push_back({probe_cluster, 0, 0});
+}
+
+void LogBasedNetworkControllerSimulation::OnPacketSent(
+    const LoggedPacketInfo& packet) {
+  ProcessUntil(packet.log_packet_time);
+  if (packet.has_transport_seq_no) {
+    PacedPacketInfo probe_info;
+    if (!pending_probes_.empty() &&
+        packet.media_type == LoggedMediaType::kVideo) {
+      auto& probe = pending_probes_.front();
+      probe_info.probe_cluster_id = probe.event.id;
+      probe_info.send_bitrate_bps = probe.event.bitrate_bps;
+      probe_info.probe_cluster_min_bytes = probe.event.min_bytes;
+      probe_info.probe_cluster_min_probes = probe.event.min_packets;
+      probe.packets_sent++;
+      probe.bytes_sent += packet.size + packet.overhead;
+      if (probe.bytes_sent >= probe.event.min_bytes &&
+          probe.packets_sent >= probe.event.min_packets) {
+        pending_probes_.pop_front();
+      }
+    }
+    transport_feedback_.AddPacket(packet.ssrc, packet.transport_seq_no,
+                                  packet.size + packet.overhead, probe_info,
+                                  packet.log_packet_time);
+  }
+  rtc::SentPacket sent_packet;
+  sent_packet.send_time_ms = packet.log_packet_time.ms();
+  sent_packet.info.included_in_allocation = true;
+  sent_packet.info.packet_size_bytes = packet.size + packet.overhead;
+  if (packet.has_transport_seq_no) {
+    sent_packet.packet_id = packet.transport_seq_no;
+    sent_packet.info.included_in_feedback = true;
+  }
+  auto msg = transport_feedback_.ProcessSentPacket(sent_packet);
+  if (msg)
+    HandleStateUpdate(controller_->OnSentPacket(*msg));
+}
+
+void LogBasedNetworkControllerSimulation::OnFeedback(
+    const LoggedRtcpPacketTransportFeedback& feedback) {
+  auto feedback_time = Timestamp::ms(feedback.log_time_ms());
+  ProcessUntil(feedback_time);
+  auto msg = transport_feedback_.ProcessTransportFeedback(
+      feedback.transport_feedback, feedback_time);
+  if (msg)
+    HandleStateUpdate(controller_->OnTransportPacketsFeedback(*msg));
+}
+
+void LogBasedNetworkControllerSimulation::OnReceiverReport(
+    const LoggedRtcpPacketReceiverReport& report) {
+  if (report.rr.report_blocks().empty())
+    return;
+  auto report_time = Timestamp::ms(report.log_time_ms());
+  ProcessUntil(report_time);
+  int packets_delta = 0;
+  int lost_delta = 0;
+  for (auto& block : report.rr.report_blocks()) {
+    auto it = last_report_blocks_.find(block.source_ssrc());
+    if (it != last_report_blocks_.end()) {
+      packets_delta +=
+          block.extended_high_seq_num() - it->second.extended_high_seq_num();
+      lost_delta += block.cumulative_lost() - it->second.cumulative_lost();
+    }
+    last_report_blocks_[block.source_ssrc()] = block;
+  }
+  if (packets_delta > lost_delta) {
+    TransportLossReport msg;
+    msg.packets_lost_delta = lost_delta;
+    msg.packets_received_delta = packets_delta - lost_delta;
+    msg.receive_time = report_time;
+    msg.start_time = last_report_block_time_;
+    msg.end_time = report_time;
+    last_report_block_time_ = report_time;
+    HandleStateUpdate(controller_->OnTransportLossReport(msg));
+  }
+
+  TimeDelta rtt = TimeDelta::PlusInfinity();
+  for (auto& rb : report.rr.report_blocks()) {
+    if (rb.last_sr()) {
+      uint32_t receive_time_ntp =
+          CompactNtp(TimeMicrosToNtp(report.log_time_us()));
+      uint32_t rtt_ntp =
+          receive_time_ntp - rb.delay_since_last_sr() - rb.last_sr();
+      rtt = std::min(rtt, TimeDelta::ms(CompactNtpRttToMs(rtt_ntp)));
+    }
+  }
+  if (rtt.IsFinite()) {
+    RoundTripTimeUpdate msg;
+    msg.receive_time = report_time;
+    msg.round_trip_time = rtt;
+    HandleStateUpdate(controller_->OnRoundTripTimeUpdate(msg));
+  }
+}
+
+void LogBasedNetworkControllerSimulation::OnIceConfig(
+    const LoggedIceCandidatePairConfig& candidate) {
+  if (candidate.type == IceCandidatePairConfigType::kSelected) {
+    auto log_time = Timestamp::us(candidate.log_time_us());
+    ProcessUntil(log_time);
+    NetworkRouteChange msg;
+    msg.at_time = log_time;
+    msg.constraints.min_data_rate = DataRate::kbps(30);
+    msg.constraints.starting_rate = DataRate::kbps(300);
+    msg.constraints.at_time = log_time;
+    HandleStateUpdate(controller_->OnNetworkRouteChange(msg));
+  }
+}
+
+void LogBasedNetworkControllerSimulation::ProcessEventsInLog(
+    const ParsedRtcEventLog& parsed_log_) {
+  auto packet_infos = parsed_log_.GetOutgoingPacketInfos();
+  RtcEventProcessor processor;
+  processor.AddEvents(
+      parsed_log_.bwe_probe_cluster_created_events(),
+      [this](const LoggedBweProbeClusterCreatedEvent& probe_cluster) {
+        OnProbeCreated(probe_cluster);
+      });
+  processor.AddEvents(packet_infos, [this](const LoggedPacketInfo& packet) {
+    OnPacketSent(packet);
+  });
+  processor.AddEvents(
+      parsed_log_.transport_feedbacks(PacketDirection::kIncomingPacket),
+      [this](const LoggedRtcpPacketTransportFeedback& feedback) {
+        OnFeedback(feedback);
+      });
+  processor.AddEvents(
+      parsed_log_.receiver_reports(PacketDirection::kIncomingPacket),
+      [this](const LoggedRtcpPacketReceiverReport& report) {
+        OnReceiverReport(report);
+      });
+  processor.AddEvents(parsed_log_.ice_candidate_pair_configs(),
+                      [this](const LoggedIceCandidatePairConfig& candidate) {
+                        OnIceConfig(candidate);
+                      });
+  processor.ProcessEventsInOrder();
+}
+
+}  // namespace webrtc
diff --git a/rtc_tools/event_log_visualizer/log_simulation.h b/rtc_tools/event_log_visualizer/log_simulation.h
new file mode 100644
index 0000000..f891db6
--- /dev/null
+++ b/rtc_tools/event_log_visualizer/log_simulation.h
@@ -0,0 +1,63 @@
+/*
+ *  Copyright 2019 The WebRTC project authors. All Rights Reserved.
+ *
+ *  Use of this source code is governed by a BSD-style license
+ *  that can be found in the LICENSE file in the root of the source
+ *  tree. An additional intellectual property rights grant can be found
+ *  in the file PATENTS.  All contributing project authors may
+ *  be found in the AUTHORS file in the root of the source tree.
+ */
+#ifndef RTC_TOOLS_EVENT_LOG_VISUALIZER_LOG_SIMULATION_H_
+#define RTC_TOOLS_EVENT_LOG_VISUALIZER_LOG_SIMULATION_H_
+
+#include <deque>
+#include <functional>
+#include <map>
+#include <memory>
+#include <vector>
+
+#include "api/transport/network_control.h"
+#include "logging/rtc_event_log/rtc_event_log_parser.h"
+#include "modules/congestion_controller/rtp/transport_feedback_adapter.h"
+
+namespace webrtc {
+
+class LogBasedNetworkControllerSimulation {
+ public:
+  explicit LogBasedNetworkControllerSimulation(
+      std::unique_ptr<NetworkControllerFactoryInterface> factory,
+      std::function<void(const NetworkControlUpdate&, Timestamp)>
+          update_handler);
+  ~LogBasedNetworkControllerSimulation();
+  void ProcessEventsInLog(const ParsedRtcEventLog& parsed_log_);
+
+ private:
+  struct ProbingStatus {
+    const LoggedBweProbeClusterCreatedEvent event;
+    size_t bytes_sent;
+    size_t packets_sent;
+  };
+  void HandleStateUpdate(const NetworkControlUpdate& update);
+  void ProcessUntil(Timestamp to_time);
+
+  void OnProbeCreated(const LoggedBweProbeClusterCreatedEvent& probe_cluster);
+  void OnPacketSent(const LoggedPacketInfo& packet);
+  void OnFeedback(const LoggedRtcpPacketTransportFeedback& feedback);
+  void OnReceiverReport(const LoggedRtcpPacketReceiverReport& report);
+  void OnIceConfig(const LoggedIceCandidatePairConfig& candidate);
+
+  const std::function<void(const NetworkControlUpdate&, Timestamp)>
+      update_handler_;
+  std::unique_ptr<NetworkControllerFactoryInterface> factory_;
+  std::unique_ptr<NetworkControllerInterface> controller_;
+
+  Timestamp current_time_ = Timestamp::MinusInfinity();
+  Timestamp last_process_ = Timestamp::MinusInfinity();
+  TransportFeedbackAdapter transport_feedback_;
+  std::deque<ProbingStatus> pending_probes_;
+  std::map<uint32_t, rtcp::ReportBlock> last_report_blocks_;
+  Timestamp last_report_block_time_ = Timestamp::MinusInfinity();
+};
+}  // namespace webrtc
+
+#endif  // RTC_TOOLS_EVENT_LOG_VISUALIZER_LOG_SIMULATION_H_
diff --git a/rtc_tools/event_log_visualizer/main.cc b/rtc_tools/event_log_visualizer/main.cc
index 252f630..1e23c86 100644
--- a/rtc_tools/event_log_visualizer/main.cc
+++ b/rtc_tools/event_log_visualizer/main.cc
@@ -112,6 +112,10 @@
     false,
     "Run the send-side bandwidth estimator with the outgoing rtp and "
     "incoming rtcp and plot the resulting estimate.");
+WEBRTC_DEFINE_bool(plot_simulated_goog_cc,
+                   false,
+                   "Run the GoogCC congestion controller based on the logged "
+                   "events and plot the target bitrate.");
 WEBRTC_DEFINE_bool(
     plot_network_delay_feedback,
     true,
@@ -365,6 +369,9 @@
   if (FLAG_plot_simulated_sendside_bwe) {
     analyzer.CreateSendSideBweSimulationGraph(collection->AppendNewPlot());
   }
+  if (FLAG_plot_simulated_goog_cc) {
+    analyzer.CreateGoogCcSimulationGraph(collection->AppendNewPlot());
+  }
   if (FLAG_plot_network_delay_feedback) {
     analyzer.CreateNetworkDelayFeedbackGraph(collection->AppendNewPlot());
   }
@@ -542,6 +549,7 @@
   FLAG_plot_outgoing_layer_bitrate_allocation = setting;
   FLAG_plot_simulated_receiveside_bwe = setting;
   FLAG_plot_simulated_sendside_bwe = setting;
+  FLAG_plot_simulated_goog_cc = setting;
   FLAG_plot_network_delay_feedback = setting;
   FLAG_plot_fraction_loss_feedback = setting;
   FLAG_plot_timestamps = setting;