Rename FrameBufferProxy to VideoStreamBufferController
This makes the class concrete, and the former FrameBuffer3Proxy is the implementation. Bug: webrtc:14003 Change-Id: Ife825b9f4efc7b79d9be8b4afb03904da819958a Reviewed-on: https://webrtc-review.googlesource.com/c/src/+/265868 Reviewed-by: Philip Eliasson <philipel@webrtc.org> Reviewed-by: Erik Språng <sprang@webrtc.org> Commit-Queue: Evan Shrubsole <eshr@webrtc.org> Cr-Commit-Position: refs/heads/main@{#37793}
This commit is contained in:

committed by
WebRTC LUCI CQ

parent
f37b0016c5
commit
476f18d2a4
@ -53,12 +53,12 @@ rtc_library("video") {
|
||||
]
|
||||
|
||||
deps = [
|
||||
":frame_buffer_proxy",
|
||||
":frame_cadence_adapter",
|
||||
":frame_decode_scheduler",
|
||||
":frame_dumping_decoder",
|
||||
":unique_timestamp_counter",
|
||||
":video_receive_stream_timeout_tracker",
|
||||
":video_stream_buffer_controller",
|
||||
":video_stream_encoder_impl",
|
||||
"../api:array_view",
|
||||
"../api:fec_controller_api",
|
||||
@ -249,10 +249,10 @@ rtc_library("frame_cadence_adapter") {
|
||||
]
|
||||
}
|
||||
|
||||
rtc_library("frame_buffer_proxy") {
|
||||
rtc_library("video_stream_buffer_controller") {
|
||||
sources = [
|
||||
"frame_buffer_proxy.cc",
|
||||
"frame_buffer_proxy.h",
|
||||
"video_stream_buffer_controller.cc",
|
||||
"video_stream_buffer_controller.h",
|
||||
]
|
||||
deps = [
|
||||
":decode_synchronizer",
|
||||
@ -744,7 +744,6 @@ if (rtc_include_tests) {
|
||||
"end_to_end_tests/ssrc_tests.cc",
|
||||
"end_to_end_tests/stats_tests.cc",
|
||||
"end_to_end_tests/transport_feedback_tests.cc",
|
||||
"frame_buffer_proxy_unittest.cc",
|
||||
"frame_cadence_adapter_unittest.cc",
|
||||
"frame_decode_timing_unittest.cc",
|
||||
"frame_encode_metadata_writer_unittest.cc",
|
||||
@ -767,12 +766,12 @@ if (rtc_include_tests) {
|
||||
"video_send_stream_impl_unittest.cc",
|
||||
"video_send_stream_tests.cc",
|
||||
"video_source_sink_controller_unittest.cc",
|
||||
"video_stream_buffer_controller_unittest.cc",
|
||||
"video_stream_decoder_impl_unittest.cc",
|
||||
"video_stream_encoder_unittest.cc",
|
||||
]
|
||||
deps = [
|
||||
":decode_synchronizer",
|
||||
":frame_buffer_proxy",
|
||||
":frame_cadence_adapter",
|
||||
":frame_decode_scheduler",
|
||||
":frame_decode_timing",
|
||||
@ -781,6 +780,7 @@ if (rtc_include_tests) {
|
||||
":video",
|
||||
":video_mocks",
|
||||
":video_receive_stream_timeout_tracker",
|
||||
":video_stream_buffer_controller",
|
||||
":video_stream_decoder_impl",
|
||||
":video_stream_encoder_impl",
|
||||
"../api:create_frame_generator",
|
||||
|
@ -1,509 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2022 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 "video/frame_buffer_proxy.h"
|
||||
|
||||
#include <algorithm>
|
||||
#include <memory>
|
||||
#include <utility>
|
||||
|
||||
#include "absl/base/attributes.h"
|
||||
#include "absl/functional/bind_front.h"
|
||||
#include "api/sequence_checker.h"
|
||||
#include "api/task_queue/task_queue_base.h"
|
||||
#include "api/units/data_size.h"
|
||||
#include "api/video/encoded_frame.h"
|
||||
#include "api/video/frame_buffer.h"
|
||||
#include "api/video/video_content_type.h"
|
||||
#include "modules/video_coding/frame_helpers.h"
|
||||
#include "modules/video_coding/timing/inter_frame_delay.h"
|
||||
#include "modules/video_coding/timing/jitter_estimator.h"
|
||||
#include "rtc_base/checks.h"
|
||||
#include "rtc_base/experiments/rtt_mult_experiment.h"
|
||||
#include "rtc_base/logging.h"
|
||||
#include "rtc_base/thread_annotations.h"
|
||||
#include "video/frame_decode_timing.h"
|
||||
#include "video/task_queue_frame_decode_scheduler.h"
|
||||
#include "video/video_receive_stream_timeout_tracker.h"
|
||||
|
||||
namespace webrtc {
|
||||
|
||||
namespace {
|
||||
|
||||
// Max number of frames the buffer will hold.
|
||||
static constexpr size_t kMaxFramesBuffered = 800;
|
||||
// Max number of decoded frame info that will be saved.
|
||||
static constexpr int kMaxFramesHistory = 1 << 13;
|
||||
|
||||
// Default value for the maximum decode queue size that is used when the
|
||||
// low-latency renderer is used.
|
||||
static constexpr size_t kZeroPlayoutDelayDefaultMaxDecodeQueueSize = 8;
|
||||
|
||||
struct FrameMetadata {
|
||||
explicit FrameMetadata(const EncodedFrame& frame)
|
||||
: is_last_spatial_layer(frame.is_last_spatial_layer),
|
||||
is_keyframe(frame.is_keyframe()),
|
||||
size(frame.size()),
|
||||
contentType(frame.contentType()),
|
||||
delayed_by_retransmission(frame.delayed_by_retransmission()),
|
||||
rtp_timestamp(frame.Timestamp()),
|
||||
receive_time(frame.ReceivedTimestamp()) {}
|
||||
|
||||
const bool is_last_spatial_layer;
|
||||
const bool is_keyframe;
|
||||
const size_t size;
|
||||
const VideoContentType contentType;
|
||||
const bool delayed_by_retransmission;
|
||||
const uint32_t rtp_timestamp;
|
||||
const absl::optional<Timestamp> receive_time;
|
||||
};
|
||||
|
||||
Timestamp ReceiveTime(const EncodedFrame& frame) {
|
||||
absl::optional<Timestamp> ts = frame.ReceivedTimestamp();
|
||||
RTC_DCHECK(ts.has_value()) << "Received frame must have a timestamp set!";
|
||||
return *ts;
|
||||
}
|
||||
|
||||
// Encapsulates use of the new frame buffer for use in
|
||||
// VideoReceiveStreamInterface. This behaves the same as the FrameBuffer2Proxy
|
||||
// but uses frame_buffer instead. Responsibilities from frame_buffer2, like
|
||||
// stats, jitter and frame timing accounting are moved into this pro
|
||||
class FrameBuffer3Proxy : public FrameBufferProxy {
|
||||
public:
|
||||
FrameBuffer3Proxy(
|
||||
Clock* clock,
|
||||
TaskQueueBase* worker_queue,
|
||||
VCMTiming* timing,
|
||||
VCMReceiveStatisticsCallback* stats_proxy,
|
||||
TaskQueueBase* decode_queue,
|
||||
FrameSchedulingReceiver* receiver,
|
||||
TimeDelta max_wait_for_keyframe,
|
||||
TimeDelta max_wait_for_frame,
|
||||
std::unique_ptr<FrameDecodeScheduler> frame_decode_scheduler,
|
||||
const FieldTrialsView& field_trials)
|
||||
: field_trials_(field_trials),
|
||||
clock_(clock),
|
||||
worker_queue_(worker_queue),
|
||||
decode_queue_(decode_queue),
|
||||
stats_proxy_(stats_proxy),
|
||||
receiver_(receiver),
|
||||
timing_(timing),
|
||||
frame_decode_scheduler_(std::move(frame_decode_scheduler)),
|
||||
jitter_estimator_(clock_, field_trials),
|
||||
buffer_(std::make_unique<FrameBuffer>(kMaxFramesBuffered,
|
||||
kMaxFramesHistory,
|
||||
field_trials)),
|
||||
decode_timing_(clock_, timing_),
|
||||
timeout_tracker_(clock_,
|
||||
worker_queue_,
|
||||
VideoReceiveStreamTimeoutTracker::Timeouts{
|
||||
.max_wait_for_keyframe = max_wait_for_keyframe,
|
||||
.max_wait_for_frame = max_wait_for_frame},
|
||||
absl::bind_front(&FrameBuffer3Proxy::OnTimeout, this)),
|
||||
zero_playout_delay_max_decode_queue_size_(
|
||||
"max_decode_queue_size",
|
||||
kZeroPlayoutDelayDefaultMaxDecodeQueueSize) {
|
||||
RTC_DCHECK(decode_queue_);
|
||||
RTC_DCHECK(stats_proxy_);
|
||||
RTC_DCHECK(receiver_);
|
||||
RTC_DCHECK(timing_);
|
||||
RTC_DCHECK(worker_queue_);
|
||||
RTC_DCHECK(clock_);
|
||||
RTC_DCHECK(frame_decode_scheduler_);
|
||||
RTC_LOG(LS_WARNING) << "Using FrameBuffer3";
|
||||
|
||||
ParseFieldTrial({&zero_playout_delay_max_decode_queue_size_},
|
||||
field_trials.Lookup("WebRTC-ZeroPlayoutDelay"));
|
||||
}
|
||||
|
||||
// FrameBufferProxy implementation.
|
||||
void StopOnWorker() override {
|
||||
RTC_DCHECK_RUN_ON(&worker_sequence_checker_);
|
||||
frame_decode_scheduler_->Stop();
|
||||
timeout_tracker_.Stop();
|
||||
decoder_ready_for_new_frame_ = false;
|
||||
decode_queue_->PostTask([this] {
|
||||
RTC_DCHECK_RUN_ON(decode_queue_);
|
||||
decode_safety_->SetNotAlive();
|
||||
});
|
||||
}
|
||||
|
||||
void SetProtectionMode(VCMVideoProtection protection_mode) override {
|
||||
RTC_DCHECK_RUN_ON(&worker_sequence_checker_);
|
||||
protection_mode_ = kProtectionNackFEC;
|
||||
}
|
||||
|
||||
void Clear() override {
|
||||
RTC_DCHECK_RUN_ON(&worker_sequence_checker_);
|
||||
stats_proxy_->OnDroppedFrames(buffer_->CurrentSize());
|
||||
buffer_ = std::make_unique<FrameBuffer>(kMaxFramesBuffered,
|
||||
kMaxFramesHistory, field_trials_);
|
||||
frame_decode_scheduler_->CancelOutstanding();
|
||||
}
|
||||
|
||||
absl::optional<int64_t> InsertFrame(
|
||||
std::unique_ptr<EncodedFrame> frame) override {
|
||||
RTC_DCHECK_RUN_ON(&worker_sequence_checker_);
|
||||
FrameMetadata metadata(*frame);
|
||||
int complete_units = buffer_->GetTotalNumberOfContinuousTemporalUnits();
|
||||
if (buffer_->InsertFrame(std::move(frame))) {
|
||||
RTC_DCHECK(metadata.receive_time) << "Frame receive time must be set!";
|
||||
if (!metadata.delayed_by_retransmission && metadata.receive_time)
|
||||
timing_->IncomingTimestamp(metadata.rtp_timestamp,
|
||||
*metadata.receive_time);
|
||||
if (complete_units < buffer_->GetTotalNumberOfContinuousTemporalUnits()) {
|
||||
stats_proxy_->OnCompleteFrame(metadata.is_keyframe, metadata.size,
|
||||
metadata.contentType);
|
||||
MaybeScheduleFrameForRelease();
|
||||
}
|
||||
}
|
||||
|
||||
return buffer_->LastContinuousFrameId();
|
||||
}
|
||||
|
||||
void UpdateRtt(int64_t max_rtt_ms) override {
|
||||
RTC_DCHECK_RUN_ON(&worker_sequence_checker_);
|
||||
jitter_estimator_.UpdateRtt(TimeDelta::Millis(max_rtt_ms));
|
||||
}
|
||||
|
||||
void SetMaxWaits(TimeDelta max_wait_for_keyframe,
|
||||
TimeDelta max_wait_for_frame) override {
|
||||
RTC_DCHECK_RUN_ON(&worker_sequence_checker_);
|
||||
timeout_tracker_.SetTimeouts(
|
||||
{.max_wait_for_keyframe = max_wait_for_keyframe,
|
||||
.max_wait_for_frame = max_wait_for_frame});
|
||||
}
|
||||
|
||||
void StartNextDecode(bool keyframe_required) override {
|
||||
if (!worker_queue_->IsCurrent()) {
|
||||
worker_queue_->PostTask(SafeTask(
|
||||
worker_safety_.flag(),
|
||||
[this, keyframe_required] { StartNextDecode(keyframe_required); }));
|
||||
return;
|
||||
}
|
||||
|
||||
RTC_DCHECK_RUN_ON(&worker_sequence_checker_);
|
||||
if (!timeout_tracker_.Running())
|
||||
timeout_tracker_.Start(keyframe_required);
|
||||
keyframe_required_ = keyframe_required;
|
||||
if (keyframe_required_) {
|
||||
timeout_tracker_.SetWaitingForKeyframe();
|
||||
}
|
||||
decoder_ready_for_new_frame_ = true;
|
||||
MaybeScheduleFrameForRelease();
|
||||
}
|
||||
|
||||
int Size() override {
|
||||
RTC_DCHECK_RUN_ON(&worker_sequence_checker_);
|
||||
return buffer_->CurrentSize();
|
||||
}
|
||||
|
||||
void OnFrameReady(
|
||||
absl::InlinedVector<std::unique_ptr<EncodedFrame>, 4> frames,
|
||||
Timestamp render_time) {
|
||||
RTC_DCHECK_RUN_ON(&worker_sequence_checker_);
|
||||
RTC_DCHECK(!frames.empty());
|
||||
|
||||
timeout_tracker_.OnEncodedFrameReleased();
|
||||
|
||||
Timestamp now = clock_->CurrentTime();
|
||||
bool superframe_delayed_by_retransmission = false;
|
||||
DataSize superframe_size = DataSize::Zero();
|
||||
const EncodedFrame& first_frame = *frames.front();
|
||||
Timestamp receive_time = ReceiveTime(first_frame);
|
||||
|
||||
if (first_frame.is_keyframe())
|
||||
keyframe_required_ = false;
|
||||
|
||||
// Gracefully handle bad RTP timestamps and render time issues.
|
||||
if (FrameHasBadRenderTiming(render_time, now,
|
||||
timing_->TargetVideoDelay())) {
|
||||
jitter_estimator_.Reset();
|
||||
timing_->Reset();
|
||||
render_time = timing_->RenderTime(first_frame.Timestamp(), now);
|
||||
}
|
||||
|
||||
for (std::unique_ptr<EncodedFrame>& frame : frames) {
|
||||
frame->SetRenderTime(render_time.ms());
|
||||
|
||||
superframe_delayed_by_retransmission |=
|
||||
frame->delayed_by_retransmission();
|
||||
receive_time = std::max(receive_time, ReceiveTime(*frame));
|
||||
superframe_size += DataSize::Bytes(frame->size());
|
||||
}
|
||||
|
||||
if (!superframe_delayed_by_retransmission) {
|
||||
auto frame_delay = inter_frame_delay_.CalculateDelay(
|
||||
first_frame.Timestamp(), receive_time);
|
||||
if (frame_delay) {
|
||||
jitter_estimator_.UpdateEstimate(*frame_delay, superframe_size);
|
||||
}
|
||||
|
||||
float rtt_mult = protection_mode_ == kProtectionNackFEC ? 0.0 : 1.0;
|
||||
absl::optional<TimeDelta> rtt_mult_add_cap_ms = absl::nullopt;
|
||||
if (rtt_mult_settings_.has_value()) {
|
||||
rtt_mult = rtt_mult_settings_->rtt_mult_setting;
|
||||
rtt_mult_add_cap_ms =
|
||||
TimeDelta::Millis(rtt_mult_settings_->rtt_mult_add_cap_ms);
|
||||
}
|
||||
timing_->SetJitterDelay(
|
||||
jitter_estimator_.GetJitterEstimate(rtt_mult, rtt_mult_add_cap_ms));
|
||||
timing_->UpdateCurrentDelay(render_time, now);
|
||||
} else if (RttMultExperiment::RttMultEnabled()) {
|
||||
jitter_estimator_.FrameNacked();
|
||||
}
|
||||
|
||||
// Update stats.
|
||||
UpdateDroppedFrames();
|
||||
UpdateJitterDelay();
|
||||
UpdateTimingFrameInfo();
|
||||
|
||||
std::unique_ptr<EncodedFrame> frame =
|
||||
CombineAndDeleteFrames(std::move(frames));
|
||||
|
||||
timing_->SetLastDecodeScheduledTimestamp(now);
|
||||
|
||||
decoder_ready_for_new_frame_ = false;
|
||||
// VideoReceiveStream2 wants frames on the decoder thread.
|
||||
decode_queue_->PostTask(
|
||||
SafeTask(decode_safety_, [this, frame = std::move(frame)]() mutable {
|
||||
RTC_DCHECK_RUN_ON(decode_queue_);
|
||||
receiver_->OnEncodedFrame(std::move(frame));
|
||||
}));
|
||||
}
|
||||
|
||||
void OnTimeout(TimeDelta delay) {
|
||||
RTC_DCHECK_RUN_ON(&worker_sequence_checker_);
|
||||
// If the stream is paused then ignore the timeout.
|
||||
if (!decoder_ready_for_new_frame_) {
|
||||
timeout_tracker_.Stop();
|
||||
return;
|
||||
}
|
||||
decode_queue_->PostTask(SafeTask(decode_safety_, [this, delay]() {
|
||||
RTC_DCHECK_RUN_ON(decode_queue_);
|
||||
receiver_->OnDecodableFrameTimeout(delay);
|
||||
}));
|
||||
// Stop sending timeouts until receive starts waiting for a new frame.
|
||||
timeout_tracker_.Stop();
|
||||
decoder_ready_for_new_frame_ = false;
|
||||
}
|
||||
|
||||
private:
|
||||
void FrameReadyForDecode(uint32_t rtp_timestamp, Timestamp render_time) {
|
||||
RTC_DCHECK_RUN_ON(&worker_sequence_checker_);
|
||||
auto frames = buffer_->ExtractNextDecodableTemporalUnit();
|
||||
RTC_DCHECK(frames[0]->Timestamp() == rtp_timestamp)
|
||||
<< "Frame buffer's next decodable frame was not the one sent for "
|
||||
"extraction rtp="
|
||||
<< rtp_timestamp << " extracted rtp=" << frames[0]->Timestamp();
|
||||
OnFrameReady(std::move(frames), render_time);
|
||||
}
|
||||
|
||||
void UpdateDroppedFrames() RTC_RUN_ON(&worker_sequence_checker_) {
|
||||
const int dropped_frames = buffer_->GetTotalNumberOfDroppedFrames() -
|
||||
frames_dropped_before_last_new_frame_;
|
||||
if (dropped_frames > 0)
|
||||
stats_proxy_->OnDroppedFrames(dropped_frames);
|
||||
frames_dropped_before_last_new_frame_ =
|
||||
buffer_->GetTotalNumberOfDroppedFrames();
|
||||
}
|
||||
|
||||
void UpdateJitterDelay() {
|
||||
auto timings = timing_->GetTimings();
|
||||
if (timings.num_decoded_frames) {
|
||||
stats_proxy_->OnFrameBufferTimingsUpdated(
|
||||
timings.max_decode_duration.ms(), timings.current_delay.ms(),
|
||||
timings.target_delay.ms(), timings.jitter_buffer_delay.ms(),
|
||||
timings.min_playout_delay.ms(), timings.render_delay.ms());
|
||||
}
|
||||
}
|
||||
|
||||
void UpdateTimingFrameInfo() {
|
||||
absl::optional<TimingFrameInfo> info = timing_->GetTimingFrameInfo();
|
||||
if (info)
|
||||
stats_proxy_->OnTimingFrameInfoUpdated(*info);
|
||||
}
|
||||
|
||||
bool IsTooManyFramesQueued() const RTC_RUN_ON(&worker_sequence_checker_) {
|
||||
return buffer_->CurrentSize() > zero_playout_delay_max_decode_queue_size_;
|
||||
}
|
||||
|
||||
void ForceKeyFrameReleaseImmediately() RTC_RUN_ON(&worker_sequence_checker_) {
|
||||
RTC_DCHECK(keyframe_required_);
|
||||
// Iterate through the frame buffer until there is a complete keyframe and
|
||||
// release this right away.
|
||||
while (buffer_->DecodableTemporalUnitsInfo()) {
|
||||
auto next_frame = buffer_->ExtractNextDecodableTemporalUnit();
|
||||
if (next_frame.empty()) {
|
||||
RTC_DCHECK_NOTREACHED()
|
||||
<< "Frame buffer should always return at least 1 frame.";
|
||||
continue;
|
||||
}
|
||||
// Found keyframe - decode right away.
|
||||
if (next_frame.front()->is_keyframe()) {
|
||||
auto render_time = timing_->RenderTime(next_frame.front()->Timestamp(),
|
||||
clock_->CurrentTime());
|
||||
OnFrameReady(std::move(next_frame), render_time);
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void MaybeScheduleFrameForRelease() RTC_RUN_ON(&worker_sequence_checker_) {
|
||||
auto decodable_tu_info = buffer_->DecodableTemporalUnitsInfo();
|
||||
if (!decoder_ready_for_new_frame_ || !decodable_tu_info) {
|
||||
return;
|
||||
}
|
||||
|
||||
if (keyframe_required_) {
|
||||
return ForceKeyFrameReleaseImmediately();
|
||||
}
|
||||
|
||||
// If already scheduled then abort.
|
||||
if (frame_decode_scheduler_->ScheduledRtpTimestamp() ==
|
||||
decodable_tu_info->next_rtp_timestamp) {
|
||||
return;
|
||||
}
|
||||
|
||||
TimeDelta max_wait = timeout_tracker_.TimeUntilTimeout();
|
||||
// Ensures the frame is scheduled for decode before the stream times out.
|
||||
// This is otherwise a race condition.
|
||||
max_wait = std::max(max_wait - TimeDelta::Millis(1), TimeDelta::Zero());
|
||||
absl::optional<FrameDecodeTiming::FrameSchedule> schedule;
|
||||
while (decodable_tu_info) {
|
||||
schedule = decode_timing_.OnFrameBufferUpdated(
|
||||
decodable_tu_info->next_rtp_timestamp,
|
||||
decodable_tu_info->last_rtp_timestamp, max_wait,
|
||||
IsTooManyFramesQueued());
|
||||
if (schedule) {
|
||||
// Don't schedule if already waiting for the same frame.
|
||||
if (frame_decode_scheduler_->ScheduledRtpTimestamp() !=
|
||||
decodable_tu_info->next_rtp_timestamp) {
|
||||
frame_decode_scheduler_->CancelOutstanding();
|
||||
frame_decode_scheduler_->ScheduleFrame(
|
||||
decodable_tu_info->next_rtp_timestamp, *schedule,
|
||||
absl::bind_front(&FrameBuffer3Proxy::FrameReadyForDecode, this));
|
||||
}
|
||||
return;
|
||||
}
|
||||
// If no schedule for current rtp, drop and try again.
|
||||
buffer_->DropNextDecodableTemporalUnit();
|
||||
decodable_tu_info = buffer_->DecodableTemporalUnitsInfo();
|
||||
}
|
||||
}
|
||||
|
||||
RTC_NO_UNIQUE_ADDRESS SequenceChecker worker_sequence_checker_;
|
||||
const FieldTrialsView& field_trials_;
|
||||
const absl::optional<RttMultExperiment::Settings> rtt_mult_settings_ =
|
||||
RttMultExperiment::GetRttMultValue();
|
||||
Clock* const clock_;
|
||||
TaskQueueBase* const worker_queue_;
|
||||
TaskQueueBase* const decode_queue_;
|
||||
VCMReceiveStatisticsCallback* const stats_proxy_;
|
||||
FrameSchedulingReceiver* const receiver_ RTC_PT_GUARDED_BY(decode_queue_);
|
||||
VCMTiming* const timing_;
|
||||
const std::unique_ptr<FrameDecodeScheduler> frame_decode_scheduler_
|
||||
RTC_GUARDED_BY(&worker_sequence_checker_);
|
||||
|
||||
JitterEstimator jitter_estimator_ RTC_GUARDED_BY(&worker_sequence_checker_);
|
||||
InterFrameDelay inter_frame_delay_ RTC_GUARDED_BY(&worker_sequence_checker_);
|
||||
bool keyframe_required_ RTC_GUARDED_BY(&worker_sequence_checker_) = false;
|
||||
std::unique_ptr<FrameBuffer> buffer_
|
||||
RTC_GUARDED_BY(&worker_sequence_checker_);
|
||||
FrameDecodeTiming decode_timing_ RTC_GUARDED_BY(&worker_sequence_checker_);
|
||||
VideoReceiveStreamTimeoutTracker timeout_tracker_
|
||||
RTC_GUARDED_BY(&worker_sequence_checker_);
|
||||
int frames_dropped_before_last_new_frame_
|
||||
RTC_GUARDED_BY(&worker_sequence_checker_) = 0;
|
||||
VCMVideoProtection protection_mode_
|
||||
RTC_GUARDED_BY(&worker_sequence_checker_) = kProtectionNack;
|
||||
|
||||
// This flag guards frames from queuing in front of the decoder. Without this
|
||||
// guard, encoded frames will not wait for the decoder to finish decoding a
|
||||
// frame and just queue up, meaning frames will not be dropped or
|
||||
// fast-forwarded when the decoder is slow or hangs.
|
||||
bool decoder_ready_for_new_frame_ RTC_GUARDED_BY(&worker_sequence_checker_) =
|
||||
false;
|
||||
|
||||
// Maximum number of frames in the decode queue to allow pacing. If the
|
||||
// queue grows beyond the max limit, pacing will be disabled and frames will
|
||||
// be pushed to the decoder as soon as possible. This only has an effect
|
||||
// when the low-latency rendering path is active, which is indicated by
|
||||
// the frame's render time == 0.
|
||||
FieldTrialParameter<unsigned> zero_playout_delay_max_decode_queue_size_;
|
||||
|
||||
rtc::scoped_refptr<PendingTaskSafetyFlag> decode_safety_ =
|
||||
PendingTaskSafetyFlag::CreateDetached();
|
||||
ScopedTaskSafety worker_safety_;
|
||||
};
|
||||
|
||||
enum class FrameBufferArm {
|
||||
kFrameBuffer3,
|
||||
kSyncDecode,
|
||||
};
|
||||
|
||||
constexpr const char* kFrameBufferFieldTrial = "WebRTC-FrameBuffer3";
|
||||
|
||||
FrameBufferArm ParseFrameBufferFieldTrial(const FieldTrialsView& field_trials) {
|
||||
webrtc::FieldTrialEnum<FrameBufferArm> arm(
|
||||
"arm", FrameBufferArm::kFrameBuffer3,
|
||||
{
|
||||
{"FrameBuffer3", FrameBufferArm::kFrameBuffer3},
|
||||
{"SyncDecoding", FrameBufferArm::kSyncDecode},
|
||||
});
|
||||
ParseFieldTrial({&arm}, field_trials.Lookup(kFrameBufferFieldTrial));
|
||||
return arm.Get();
|
||||
}
|
||||
|
||||
} // namespace
|
||||
|
||||
std::unique_ptr<FrameBufferProxy> FrameBufferProxy::CreateFromFieldTrial(
|
||||
Clock* clock,
|
||||
TaskQueueBase* worker_queue,
|
||||
VCMTiming* timing,
|
||||
VCMReceiveStatisticsCallback* stats_proxy,
|
||||
TaskQueueBase* decode_queue,
|
||||
FrameSchedulingReceiver* receiver,
|
||||
TimeDelta max_wait_for_keyframe,
|
||||
TimeDelta max_wait_for_frame,
|
||||
DecodeSynchronizer* decode_sync,
|
||||
const FieldTrialsView& field_trials) {
|
||||
switch (ParseFrameBufferFieldTrial(field_trials)) {
|
||||
case FrameBufferArm::kSyncDecode: {
|
||||
std::unique_ptr<FrameDecodeScheduler> scheduler;
|
||||
if (decode_sync) {
|
||||
scheduler = decode_sync->CreateSynchronizedFrameScheduler();
|
||||
} else {
|
||||
RTC_LOG(LS_ERROR) << "In FrameBuffer with sync decode trial, but "
|
||||
"no DecodeSynchronizer was present!";
|
||||
// Crash in debug, but in production use the task queue scheduler.
|
||||
RTC_DCHECK_NOTREACHED();
|
||||
scheduler = std::make_unique<TaskQueueFrameDecodeScheduler>(
|
||||
clock, worker_queue);
|
||||
}
|
||||
return std::make_unique<FrameBuffer3Proxy>(
|
||||
clock, worker_queue, timing, stats_proxy, decode_queue, receiver,
|
||||
max_wait_for_keyframe, max_wait_for_frame, std::move(scheduler),
|
||||
field_trials);
|
||||
}
|
||||
case FrameBufferArm::kFrameBuffer3:
|
||||
ABSL_FALLTHROUGH_INTENDED;
|
||||
default: {
|
||||
auto scheduler =
|
||||
std::make_unique<TaskQueueFrameDecodeScheduler>(clock, worker_queue);
|
||||
return std::make_unique<FrameBuffer3Proxy>(
|
||||
clock, worker_queue, timing, stats_proxy, decode_queue, receiver,
|
||||
max_wait_for_keyframe, max_wait_for_frame, std::move(scheduler),
|
||||
field_trials);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
} // namespace webrtc
|
@ -1,71 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2022 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 VIDEO_FRAME_BUFFER_PROXY_H_
|
||||
#define VIDEO_FRAME_BUFFER_PROXY_H_
|
||||
|
||||
#include <memory>
|
||||
|
||||
#include "api/field_trials_view.h"
|
||||
#include "api/metronome/metronome.h"
|
||||
#include "api/task_queue/task_queue_base.h"
|
||||
#include "api/video/encoded_frame.h"
|
||||
#include "modules/video_coding/include/video_coding_defines.h"
|
||||
#include "modules/video_coding/timing/timing.h"
|
||||
#include "system_wrappers/include/clock.h"
|
||||
#include "video/decode_synchronizer.h"
|
||||
|
||||
namespace webrtc {
|
||||
|
||||
class FrameSchedulingReceiver {
|
||||
public:
|
||||
virtual ~FrameSchedulingReceiver() = default;
|
||||
|
||||
virtual void OnEncodedFrame(std::unique_ptr<EncodedFrame> frame) = 0;
|
||||
virtual void OnDecodableFrameTimeout(TimeDelta wait_time) = 0;
|
||||
};
|
||||
|
||||
// Temporary class to enable replacement of frame_buffer2 with frame_buffer.
|
||||
// Once frame_buffer has shown to work with a field trial, frame_buffer2 will
|
||||
// be removed and this class should be directly integrated into
|
||||
// video_receive_stream2. bugs.webrtc.org/13343 tracks this integration.
|
||||
class FrameBufferProxy {
|
||||
public:
|
||||
static std::unique_ptr<FrameBufferProxy> CreateFromFieldTrial(
|
||||
Clock* clock,
|
||||
TaskQueueBase* worker_queue,
|
||||
VCMTiming* timing,
|
||||
VCMReceiveStatisticsCallback* stats_proxy,
|
||||
TaskQueueBase* decode_queue,
|
||||
FrameSchedulingReceiver* receiver,
|
||||
TimeDelta max_wait_for_keyframe,
|
||||
TimeDelta max_wait_for_frame,
|
||||
DecodeSynchronizer* decode_sync,
|
||||
const FieldTrialsView& field_trials);
|
||||
virtual ~FrameBufferProxy() = default;
|
||||
|
||||
// Run on the worker thread.
|
||||
virtual void StopOnWorker() = 0;
|
||||
virtual void SetProtectionMode(VCMVideoProtection protection_mode) = 0;
|
||||
virtual void Clear() = 0;
|
||||
virtual absl::optional<int64_t> InsertFrame(
|
||||
std::unique_ptr<EncodedFrame> frame) = 0;
|
||||
virtual void UpdateRtt(int64_t max_rtt_ms) = 0;
|
||||
virtual int Size() = 0;
|
||||
virtual void SetMaxWaits(TimeDelta max_wait_for_keyframe,
|
||||
TimeDelta max_wait_for_frame) = 0;
|
||||
|
||||
// Run on either the worker thread or the decode thread.
|
||||
virtual void StartNextDecode(bool keyframe_required) = 0;
|
||||
};
|
||||
|
||||
} // namespace webrtc
|
||||
|
||||
#endif // VIDEO_FRAME_BUFFER_PROXY_H_
|
@ -261,7 +261,7 @@ VideoReceiveStream2::VideoReceiveStream2(
|
||||
|
||||
timing_->set_render_delay(TimeDelta::Millis(config_.render_delay_ms));
|
||||
|
||||
frame_buffer_ = FrameBufferProxy::CreateFromFieldTrial(
|
||||
buffer_ = VideoStreamBufferController::CreateFromFieldTrial(
|
||||
clock_, call_->worker_thread(), timing_.get(), &stats_proxy_,
|
||||
decode_queue_.Get(), this, max_wait_for_keyframe_, max_wait_for_frame_,
|
||||
decode_sync_, call_->trials());
|
||||
@ -353,7 +353,7 @@ void VideoReceiveStream2::Start() {
|
||||
rtp_video_stream_receiver_.ulpfec_payload_type() != -1;
|
||||
|
||||
if (config_.rtp.nack.rtp_history_ms > 0 && protected_by_fec) {
|
||||
frame_buffer_->SetProtectionMode(kProtectionNackFEC);
|
||||
buffer_->SetProtectionMode(kProtectionNackFEC);
|
||||
}
|
||||
|
||||
transport_adapter_.Enable();
|
||||
@ -410,7 +410,7 @@ void VideoReceiveStream2::Start() {
|
||||
RTC_DCHECK_RUN_ON(&decode_queue_);
|
||||
decoder_stopped_ = false;
|
||||
});
|
||||
frame_buffer_->StartNextDecode(true);
|
||||
buffer_->StartNextDecode(true);
|
||||
decoder_running_ = true;
|
||||
|
||||
{
|
||||
@ -433,7 +433,7 @@ void VideoReceiveStream2::Stop() {
|
||||
stats_proxy_.OnUniqueFramesCounted(
|
||||
rtp_video_stream_receiver_.GetUniqueFramesSeen());
|
||||
|
||||
frame_buffer_->StopOnWorker();
|
||||
buffer_->StopOnWorker();
|
||||
call_stats_->DeregisterStatsObserver(this);
|
||||
if (decoder_running_) {
|
||||
rtc::Event done;
|
||||
@ -534,9 +534,9 @@ void VideoReceiveStream2::SetNackHistory(TimeDelta history) {
|
||||
config_.rtp.protected_by_flexfec ||
|
||||
rtp_video_stream_receiver_.ulpfec_payload_type() != -1;
|
||||
|
||||
frame_buffer_->SetProtectionMode(history.ms() > 0 && protected_by_fec
|
||||
? kProtectionNackFEC
|
||||
: kProtectionNack);
|
||||
buffer_->SetProtectionMode(history.ms() > 0 && protected_by_fec
|
||||
? kProtectionNackFEC
|
||||
: kProtectionNack);
|
||||
|
||||
rtp_video_stream_receiver_.SetNackHistory(history);
|
||||
TimeDelta max_wait_for_keyframe = DetermineMaxWaitForFrame(history, true);
|
||||
@ -548,7 +548,7 @@ void VideoReceiveStream2::SetNackHistory(TimeDelta history) {
|
||||
max_wait_for_frame_ = max_wait_for_frame;
|
||||
});
|
||||
|
||||
frame_buffer_->SetMaxWaits(max_wait_for_keyframe, max_wait_for_frame);
|
||||
buffer_->SetMaxWaits(max_wait_for_keyframe, max_wait_for_frame);
|
||||
}
|
||||
|
||||
void VideoReceiveStream2::SetProtectionPayloadTypes(int red_payload_type,
|
||||
@ -739,7 +739,7 @@ void VideoReceiveStream2::OnCompleteFrame(std::unique_ptr<EncodedFrame> frame) {
|
||||
UpdatePlayoutDelays();
|
||||
}
|
||||
|
||||
auto last_continuous_pid = frame_buffer_->InsertFrame(std::move(frame));
|
||||
auto last_continuous_pid = buffer_->InsertFrame(std::move(frame));
|
||||
if (last_continuous_pid.has_value()) {
|
||||
{
|
||||
// TODO(bugs.webrtc.org/11993): Call on the network thread.
|
||||
@ -752,7 +752,7 @@ void VideoReceiveStream2::OnCompleteFrame(std::unique_ptr<EncodedFrame> frame) {
|
||||
void VideoReceiveStream2::OnRttUpdate(int64_t avg_rtt_ms, int64_t max_rtt_ms) {
|
||||
RTC_DCHECK_RUN_ON(&worker_sequence_checker_);
|
||||
// TODO(bugs.webrtc.org/13757): Replace with TimeDelta.
|
||||
frame_buffer_->UpdateRtt(max_rtt_ms);
|
||||
buffer_->UpdateRtt(max_rtt_ms);
|
||||
rtp_video_stream_receiver_.UpdateRtt(max_rtt_ms);
|
||||
stats_proxy_.OnRttUpdate(avg_rtt_ms);
|
||||
}
|
||||
@ -803,7 +803,7 @@ void VideoReceiveStream2::OnEncodedFrame(std::unique_ptr<EncodedFrame> frame) {
|
||||
if (decoder_stopped_)
|
||||
return;
|
||||
HandleEncodedFrame(std::move(frame));
|
||||
frame_buffer_->StartNextDecode(keyframe_required_);
|
||||
buffer_->StartNextDecode(keyframe_required_);
|
||||
}
|
||||
|
||||
void VideoReceiveStream2::OnDecodableFrameTimeout(TimeDelta wait_time) {
|
||||
@ -818,7 +818,7 @@ void VideoReceiveStream2::OnDecodableFrameTimeout(TimeDelta wait_time) {
|
||||
|
||||
decode_queue_.PostTask([this] {
|
||||
RTC_DCHECK_RUN_ON(&decode_queue_);
|
||||
frame_buffer_->StartNextDecode(keyframe_required_);
|
||||
buffer_->StartNextDecode(keyframe_required_);
|
||||
});
|
||||
}));
|
||||
}
|
||||
@ -1059,7 +1059,7 @@ void VideoReceiveStream2::UpdatePlayoutDelays() const {
|
||||
std::lrint(*frame_maximum_playout_delay_ * kFrameRate);
|
||||
// Subtract frames in buffer.
|
||||
max_composition_delay_in_frames =
|
||||
std::max(max_composition_delay_in_frames - frame_buffer_->Size(), 0);
|
||||
std::max(max_composition_delay_in_frames - buffer_->Size(), 0);
|
||||
timing_->SetMaxCompositionDelayInFrames(max_composition_delay_in_frames);
|
||||
}
|
||||
}
|
||||
|
@ -34,11 +34,11 @@
|
||||
#include "rtc_base/task_queue.h"
|
||||
#include "rtc_base/thread_annotations.h"
|
||||
#include "system_wrappers/include/clock.h"
|
||||
#include "video/frame_buffer_proxy.h"
|
||||
#include "video/receive_statistics_proxy2.h"
|
||||
#include "video/rtp_streams_synchronizer2.h"
|
||||
#include "video/rtp_video_stream_receiver2.h"
|
||||
#include "video/transport_adapter.h"
|
||||
#include "video/video_stream_buffer_controller.h"
|
||||
#include "video/video_stream_decoder2.h"
|
||||
|
||||
namespace webrtc {
|
||||
@ -262,7 +262,7 @@ class VideoReceiveStream2
|
||||
// moved to the new VideoStreamDecoder.
|
||||
std::vector<std::unique_ptr<VideoDecoder>> video_decoders_;
|
||||
|
||||
std::unique_ptr<FrameBufferProxy> frame_buffer_;
|
||||
std::unique_ptr<VideoStreamBufferController> buffer_;
|
||||
|
||||
std::unique_ptr<RtpStreamReceiverInterface> media_receiver_
|
||||
RTC_GUARDED_BY(packet_sequence_checker_);
|
||||
|
461
video/video_stream_buffer_controller.cc
Normal file
461
video/video_stream_buffer_controller.cc
Normal file
@ -0,0 +1,461 @@
|
||||
/*
|
||||
* Copyright (c) 2022 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 "video/video_stream_buffer_controller.h"
|
||||
|
||||
#include <algorithm>
|
||||
#include <memory>
|
||||
#include <utility>
|
||||
|
||||
#include "absl/base/attributes.h"
|
||||
#include "absl/functional/bind_front.h"
|
||||
#include "api/sequence_checker.h"
|
||||
#include "api/task_queue/task_queue_base.h"
|
||||
#include "api/units/data_size.h"
|
||||
#include "api/video/encoded_frame.h"
|
||||
#include "api/video/frame_buffer.h"
|
||||
#include "api/video/video_content_type.h"
|
||||
#include "modules/video_coding/frame_helpers.h"
|
||||
#include "modules/video_coding/timing/inter_frame_delay.h"
|
||||
#include "modules/video_coding/timing/jitter_estimator.h"
|
||||
#include "rtc_base/checks.h"
|
||||
#include "rtc_base/logging.h"
|
||||
#include "rtc_base/thread_annotations.h"
|
||||
#include "video/frame_decode_timing.h"
|
||||
#include "video/task_queue_frame_decode_scheduler.h"
|
||||
#include "video/video_receive_stream_timeout_tracker.h"
|
||||
|
||||
namespace webrtc {
|
||||
|
||||
namespace {
|
||||
|
||||
// Max number of frames the buffer will hold.
|
||||
static constexpr size_t kMaxFramesBuffered = 800;
|
||||
// Max number of decoded frame info that will be saved.
|
||||
static constexpr int kMaxFramesHistory = 1 << 13;
|
||||
|
||||
// Default value for the maximum decode queue size that is used when the
|
||||
// low-latency renderer is used.
|
||||
static constexpr size_t kZeroPlayoutDelayDefaultMaxDecodeQueueSize = 8;
|
||||
|
||||
struct FrameMetadata {
|
||||
explicit FrameMetadata(const EncodedFrame& frame)
|
||||
: is_last_spatial_layer(frame.is_last_spatial_layer),
|
||||
is_keyframe(frame.is_keyframe()),
|
||||
size(frame.size()),
|
||||
contentType(frame.contentType()),
|
||||
delayed_by_retransmission(frame.delayed_by_retransmission()),
|
||||
rtp_timestamp(frame.Timestamp()),
|
||||
receive_time(frame.ReceivedTimestamp()) {}
|
||||
|
||||
const bool is_last_spatial_layer;
|
||||
const bool is_keyframe;
|
||||
const size_t size;
|
||||
const VideoContentType contentType;
|
||||
const bool delayed_by_retransmission;
|
||||
const uint32_t rtp_timestamp;
|
||||
const absl::optional<Timestamp> receive_time;
|
||||
};
|
||||
|
||||
Timestamp ReceiveTime(const EncodedFrame& frame) {
|
||||
absl::optional<Timestamp> ts = frame.ReceivedTimestamp();
|
||||
RTC_DCHECK(ts.has_value()) << "Received frame must have a timestamp set!";
|
||||
return *ts;
|
||||
}
|
||||
|
||||
enum class FrameBufferArm {
|
||||
kFrameBuffer3,
|
||||
kSyncDecode,
|
||||
};
|
||||
|
||||
constexpr const char* kFrameBufferFieldTrial = "WebRTC-FrameBuffer3";
|
||||
|
||||
FrameBufferArm ParseFrameBufferFieldTrial(const FieldTrialsView& field_trials) {
|
||||
webrtc::FieldTrialEnum<FrameBufferArm> arm(
|
||||
"arm", FrameBufferArm::kFrameBuffer3,
|
||||
{
|
||||
{"FrameBuffer3", FrameBufferArm::kFrameBuffer3},
|
||||
{"SyncDecoding", FrameBufferArm::kSyncDecode},
|
||||
});
|
||||
ParseFieldTrial({&arm}, field_trials.Lookup(kFrameBufferFieldTrial));
|
||||
return arm.Get();
|
||||
}
|
||||
|
||||
} // namespace
|
||||
|
||||
std::unique_ptr<VideoStreamBufferController>
|
||||
VideoStreamBufferController::CreateFromFieldTrial(
|
||||
Clock* clock,
|
||||
TaskQueueBase* worker_queue,
|
||||
VCMTiming* timing,
|
||||
VCMReceiveStatisticsCallback* stats_proxy,
|
||||
TaskQueueBase* decode_queue,
|
||||
FrameSchedulingReceiver* receiver,
|
||||
TimeDelta max_wait_for_keyframe,
|
||||
TimeDelta max_wait_for_frame,
|
||||
DecodeSynchronizer* decode_sync,
|
||||
const FieldTrialsView& field_trials) {
|
||||
switch (ParseFrameBufferFieldTrial(field_trials)) {
|
||||
case FrameBufferArm::kSyncDecode: {
|
||||
std::unique_ptr<FrameDecodeScheduler> scheduler;
|
||||
if (decode_sync) {
|
||||
scheduler = decode_sync->CreateSynchronizedFrameScheduler();
|
||||
} else {
|
||||
RTC_LOG(LS_ERROR) << "In FrameBuffer with sync decode trial, but "
|
||||
"no DecodeSynchronizer was present!";
|
||||
// Crash in debug, but in production use the task queue scheduler.
|
||||
RTC_DCHECK_NOTREACHED();
|
||||
scheduler = std::make_unique<TaskQueueFrameDecodeScheduler>(
|
||||
clock, worker_queue);
|
||||
}
|
||||
return std::make_unique<VideoStreamBufferController>(
|
||||
clock, worker_queue, timing, stats_proxy, decode_queue, receiver,
|
||||
max_wait_for_keyframe, max_wait_for_frame, std::move(scheduler),
|
||||
field_trials);
|
||||
}
|
||||
case FrameBufferArm::kFrameBuffer3:
|
||||
ABSL_FALLTHROUGH_INTENDED;
|
||||
default: {
|
||||
auto scheduler =
|
||||
std::make_unique<TaskQueueFrameDecodeScheduler>(clock, worker_queue);
|
||||
return std::make_unique<VideoStreamBufferController>(
|
||||
clock, worker_queue, timing, stats_proxy, decode_queue, receiver,
|
||||
max_wait_for_keyframe, max_wait_for_frame, std::move(scheduler),
|
||||
field_trials);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
VideoStreamBufferController::VideoStreamBufferController(
|
||||
Clock* clock,
|
||||
TaskQueueBase* worker_queue,
|
||||
VCMTiming* timing,
|
||||
VCMReceiveStatisticsCallback* stats_proxy,
|
||||
TaskQueueBase* decode_queue,
|
||||
FrameSchedulingReceiver* receiver,
|
||||
TimeDelta max_wait_for_keyframe,
|
||||
TimeDelta max_wait_for_frame,
|
||||
std::unique_ptr<FrameDecodeScheduler> frame_decode_scheduler,
|
||||
const FieldTrialsView& field_trials)
|
||||
: field_trials_(field_trials),
|
||||
clock_(clock),
|
||||
worker_queue_(worker_queue),
|
||||
decode_queue_(decode_queue),
|
||||
stats_proxy_(stats_proxy),
|
||||
receiver_(receiver),
|
||||
timing_(timing),
|
||||
frame_decode_scheduler_(std::move(frame_decode_scheduler)),
|
||||
jitter_estimator_(clock_, field_trials),
|
||||
buffer_(std::make_unique<FrameBuffer>(kMaxFramesBuffered,
|
||||
kMaxFramesHistory,
|
||||
field_trials)),
|
||||
decode_timing_(clock_, timing_),
|
||||
timeout_tracker_(
|
||||
clock_,
|
||||
worker_queue_,
|
||||
VideoReceiveStreamTimeoutTracker::Timeouts{
|
||||
.max_wait_for_keyframe = max_wait_for_keyframe,
|
||||
.max_wait_for_frame = max_wait_for_frame},
|
||||
absl::bind_front(&VideoStreamBufferController::OnTimeout, this)),
|
||||
zero_playout_delay_max_decode_queue_size_(
|
||||
"max_decode_queue_size",
|
||||
kZeroPlayoutDelayDefaultMaxDecodeQueueSize) {
|
||||
RTC_DCHECK(decode_queue_);
|
||||
RTC_DCHECK(stats_proxy_);
|
||||
RTC_DCHECK(receiver_);
|
||||
RTC_DCHECK(timing_);
|
||||
RTC_DCHECK(worker_queue_);
|
||||
RTC_DCHECK(clock_);
|
||||
RTC_DCHECK(frame_decode_scheduler_);
|
||||
RTC_LOG(LS_WARNING) << "Using FrameBuffer3";
|
||||
|
||||
ParseFieldTrial({&zero_playout_delay_max_decode_queue_size_},
|
||||
field_trials.Lookup("WebRTC-ZeroPlayoutDelay"));
|
||||
}
|
||||
|
||||
void VideoStreamBufferController::StopOnWorker() {
|
||||
RTC_DCHECK_RUN_ON(&worker_sequence_checker_);
|
||||
frame_decode_scheduler_->Stop();
|
||||
timeout_tracker_.Stop();
|
||||
decoder_ready_for_new_frame_ = false;
|
||||
decode_queue_->PostTask([this] {
|
||||
RTC_DCHECK_RUN_ON(decode_queue_);
|
||||
decode_safety_->SetNotAlive();
|
||||
});
|
||||
}
|
||||
|
||||
void VideoStreamBufferController::SetProtectionMode(
|
||||
VCMVideoProtection protection_mode) {
|
||||
RTC_DCHECK_RUN_ON(&worker_sequence_checker_);
|
||||
protection_mode_ = protection_mode;
|
||||
}
|
||||
|
||||
void VideoStreamBufferController::Clear() {
|
||||
RTC_DCHECK_RUN_ON(&worker_sequence_checker_);
|
||||
stats_proxy_->OnDroppedFrames(buffer_->CurrentSize());
|
||||
buffer_ = std::make_unique<FrameBuffer>(kMaxFramesBuffered, kMaxFramesHistory,
|
||||
field_trials_);
|
||||
frame_decode_scheduler_->CancelOutstanding();
|
||||
}
|
||||
|
||||
absl::optional<int64_t> VideoStreamBufferController::InsertFrame(
|
||||
std::unique_ptr<EncodedFrame> frame) {
|
||||
RTC_DCHECK_RUN_ON(&worker_sequence_checker_);
|
||||
FrameMetadata metadata(*frame);
|
||||
int complete_units = buffer_->GetTotalNumberOfContinuousTemporalUnits();
|
||||
if (buffer_->InsertFrame(std::move(frame))) {
|
||||
RTC_DCHECK(metadata.receive_time) << "Frame receive time must be set!";
|
||||
if (!metadata.delayed_by_retransmission && metadata.receive_time)
|
||||
timing_->IncomingTimestamp(metadata.rtp_timestamp,
|
||||
*metadata.receive_time);
|
||||
if (complete_units < buffer_->GetTotalNumberOfContinuousTemporalUnits()) {
|
||||
stats_proxy_->OnCompleteFrame(metadata.is_keyframe, metadata.size,
|
||||
metadata.contentType);
|
||||
MaybeScheduleFrameForRelease();
|
||||
}
|
||||
}
|
||||
|
||||
return buffer_->LastContinuousFrameId();
|
||||
}
|
||||
|
||||
void VideoStreamBufferController::UpdateRtt(int64_t max_rtt_ms) {
|
||||
RTC_DCHECK_RUN_ON(&worker_sequence_checker_);
|
||||
jitter_estimator_.UpdateRtt(TimeDelta::Millis(max_rtt_ms));
|
||||
}
|
||||
|
||||
void VideoStreamBufferController::SetMaxWaits(TimeDelta max_wait_for_keyframe,
|
||||
TimeDelta max_wait_for_frame) {
|
||||
RTC_DCHECK_RUN_ON(&worker_sequence_checker_);
|
||||
timeout_tracker_.SetTimeouts({.max_wait_for_keyframe = max_wait_for_keyframe,
|
||||
.max_wait_for_frame = max_wait_for_frame});
|
||||
}
|
||||
|
||||
void VideoStreamBufferController::StartNextDecode(bool keyframe_required) {
|
||||
if (!worker_queue_->IsCurrent()) {
|
||||
worker_queue_->PostTask(SafeTask(
|
||||
worker_safety_.flag(),
|
||||
[this, keyframe_required] { StartNextDecode(keyframe_required); }));
|
||||
return;
|
||||
}
|
||||
|
||||
RTC_DCHECK_RUN_ON(&worker_sequence_checker_);
|
||||
if (!timeout_tracker_.Running())
|
||||
timeout_tracker_.Start(keyframe_required);
|
||||
keyframe_required_ = keyframe_required;
|
||||
if (keyframe_required_) {
|
||||
timeout_tracker_.SetWaitingForKeyframe();
|
||||
}
|
||||
decoder_ready_for_new_frame_ = true;
|
||||
MaybeScheduleFrameForRelease();
|
||||
}
|
||||
|
||||
int VideoStreamBufferController::Size() {
|
||||
RTC_DCHECK_RUN_ON(&worker_sequence_checker_);
|
||||
return buffer_->CurrentSize();
|
||||
}
|
||||
|
||||
void VideoStreamBufferController::OnFrameReady(
|
||||
absl::InlinedVector<std::unique_ptr<EncodedFrame>, 4> frames,
|
||||
Timestamp render_time) {
|
||||
RTC_DCHECK_RUN_ON(&worker_sequence_checker_);
|
||||
RTC_DCHECK(!frames.empty());
|
||||
|
||||
timeout_tracker_.OnEncodedFrameReleased();
|
||||
|
||||
Timestamp now = clock_->CurrentTime();
|
||||
bool superframe_delayed_by_retransmission = false;
|
||||
DataSize superframe_size = DataSize::Zero();
|
||||
const EncodedFrame& first_frame = *frames.front();
|
||||
Timestamp receive_time = ReceiveTime(first_frame);
|
||||
|
||||
if (first_frame.is_keyframe())
|
||||
keyframe_required_ = false;
|
||||
|
||||
// Gracefully handle bad RTP timestamps and render time issues.
|
||||
if (FrameHasBadRenderTiming(render_time, now, timing_->TargetVideoDelay())) {
|
||||
jitter_estimator_.Reset();
|
||||
timing_->Reset();
|
||||
render_time = timing_->RenderTime(first_frame.Timestamp(), now);
|
||||
}
|
||||
|
||||
for (std::unique_ptr<EncodedFrame>& frame : frames) {
|
||||
frame->SetRenderTime(render_time.ms());
|
||||
|
||||
superframe_delayed_by_retransmission |= frame->delayed_by_retransmission();
|
||||
receive_time = std::max(receive_time, ReceiveTime(*frame));
|
||||
superframe_size += DataSize::Bytes(frame->size());
|
||||
}
|
||||
|
||||
if (!superframe_delayed_by_retransmission) {
|
||||
auto frame_delay = inter_frame_delay_.CalculateDelay(
|
||||
first_frame.Timestamp(), receive_time);
|
||||
if (frame_delay) {
|
||||
jitter_estimator_.UpdateEstimate(*frame_delay, superframe_size);
|
||||
}
|
||||
|
||||
float rtt_mult = protection_mode_ == kProtectionNackFEC ? 0.0 : 1.0;
|
||||
absl::optional<TimeDelta> rtt_mult_add_cap_ms = absl::nullopt;
|
||||
if (rtt_mult_settings_.has_value()) {
|
||||
rtt_mult = rtt_mult_settings_->rtt_mult_setting;
|
||||
rtt_mult_add_cap_ms =
|
||||
TimeDelta::Millis(rtt_mult_settings_->rtt_mult_add_cap_ms);
|
||||
}
|
||||
timing_->SetJitterDelay(
|
||||
jitter_estimator_.GetJitterEstimate(rtt_mult, rtt_mult_add_cap_ms));
|
||||
timing_->UpdateCurrentDelay(render_time, now);
|
||||
} else if (RttMultExperiment::RttMultEnabled()) {
|
||||
jitter_estimator_.FrameNacked();
|
||||
}
|
||||
|
||||
// Update stats.
|
||||
UpdateDroppedFrames();
|
||||
UpdateJitterDelay();
|
||||
UpdateTimingFrameInfo();
|
||||
|
||||
std::unique_ptr<EncodedFrame> frame =
|
||||
CombineAndDeleteFrames(std::move(frames));
|
||||
|
||||
timing_->SetLastDecodeScheduledTimestamp(now);
|
||||
|
||||
decoder_ready_for_new_frame_ = false;
|
||||
// VideoReceiveStream2 wants frames on the decoder thread.
|
||||
decode_queue_->PostTask(
|
||||
SafeTask(decode_safety_, [this, frame = std::move(frame)]() mutable {
|
||||
RTC_DCHECK_RUN_ON(decode_queue_);
|
||||
receiver_->OnEncodedFrame(std::move(frame));
|
||||
}));
|
||||
}
|
||||
|
||||
void VideoStreamBufferController::OnTimeout(TimeDelta delay) {
|
||||
RTC_DCHECK_RUN_ON(&worker_sequence_checker_);
|
||||
// If the stream is paused then ignore the timeout.
|
||||
if (!decoder_ready_for_new_frame_) {
|
||||
timeout_tracker_.Stop();
|
||||
return;
|
||||
}
|
||||
decode_queue_->PostTask(SafeTask(decode_safety_, [this, delay]() {
|
||||
RTC_DCHECK_RUN_ON(decode_queue_);
|
||||
receiver_->OnDecodableFrameTimeout(delay);
|
||||
}));
|
||||
// Stop sending timeouts until receive starts waiting for a new frame.
|
||||
timeout_tracker_.Stop();
|
||||
decoder_ready_for_new_frame_ = false;
|
||||
}
|
||||
|
||||
void VideoStreamBufferController::FrameReadyForDecode(uint32_t rtp_timestamp,
|
||||
Timestamp render_time) {
|
||||
RTC_DCHECK_RUN_ON(&worker_sequence_checker_);
|
||||
auto frames = buffer_->ExtractNextDecodableTemporalUnit();
|
||||
RTC_DCHECK(frames[0]->Timestamp() == rtp_timestamp)
|
||||
<< "Frame buffer's next decodable frame was not the one sent for "
|
||||
"extraction rtp="
|
||||
<< rtp_timestamp << " extracted rtp=" << frames[0]->Timestamp();
|
||||
OnFrameReady(std::move(frames), render_time);
|
||||
}
|
||||
|
||||
void VideoStreamBufferController::UpdateDroppedFrames()
|
||||
RTC_RUN_ON(&worker_sequence_checker_) {
|
||||
const int dropped_frames = buffer_->GetTotalNumberOfDroppedFrames() -
|
||||
frames_dropped_before_last_new_frame_;
|
||||
if (dropped_frames > 0)
|
||||
stats_proxy_->OnDroppedFrames(dropped_frames);
|
||||
frames_dropped_before_last_new_frame_ =
|
||||
buffer_->GetTotalNumberOfDroppedFrames();
|
||||
}
|
||||
|
||||
void VideoStreamBufferController::UpdateJitterDelay() {
|
||||
auto timings = timing_->GetTimings();
|
||||
if (timings.num_decoded_frames) {
|
||||
stats_proxy_->OnFrameBufferTimingsUpdated(
|
||||
timings.max_decode_duration.ms(), timings.current_delay.ms(),
|
||||
timings.target_delay.ms(), timings.jitter_buffer_delay.ms(),
|
||||
timings.min_playout_delay.ms(), timings.render_delay.ms());
|
||||
}
|
||||
}
|
||||
|
||||
void VideoStreamBufferController::UpdateTimingFrameInfo() {
|
||||
absl::optional<TimingFrameInfo> info = timing_->GetTimingFrameInfo();
|
||||
if (info)
|
||||
stats_proxy_->OnTimingFrameInfoUpdated(*info);
|
||||
}
|
||||
|
||||
bool VideoStreamBufferController::IsTooManyFramesQueued() const
|
||||
RTC_RUN_ON(&worker_sequence_checker_) {
|
||||
return buffer_->CurrentSize() > zero_playout_delay_max_decode_queue_size_;
|
||||
}
|
||||
|
||||
void VideoStreamBufferController::ForceKeyFrameReleaseImmediately()
|
||||
RTC_RUN_ON(&worker_sequence_checker_) {
|
||||
RTC_DCHECK(keyframe_required_);
|
||||
// Iterate through the frame buffer until there is a complete keyframe and
|
||||
// release this right away.
|
||||
while (buffer_->DecodableTemporalUnitsInfo()) {
|
||||
auto next_frame = buffer_->ExtractNextDecodableTemporalUnit();
|
||||
if (next_frame.empty()) {
|
||||
RTC_DCHECK_NOTREACHED()
|
||||
<< "Frame buffer should always return at least 1 frame.";
|
||||
continue;
|
||||
}
|
||||
// Found keyframe - decode right away.
|
||||
if (next_frame.front()->is_keyframe()) {
|
||||
auto render_time = timing_->RenderTime(next_frame.front()->Timestamp(),
|
||||
clock_->CurrentTime());
|
||||
OnFrameReady(std::move(next_frame), render_time);
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void VideoStreamBufferController::MaybeScheduleFrameForRelease()
|
||||
RTC_RUN_ON(&worker_sequence_checker_) {
|
||||
auto decodable_tu_info = buffer_->DecodableTemporalUnitsInfo();
|
||||
if (!decoder_ready_for_new_frame_ || !decodable_tu_info) {
|
||||
return;
|
||||
}
|
||||
|
||||
if (keyframe_required_) {
|
||||
return ForceKeyFrameReleaseImmediately();
|
||||
}
|
||||
|
||||
// If already scheduled then abort.
|
||||
if (frame_decode_scheduler_->ScheduledRtpTimestamp() ==
|
||||
decodable_tu_info->next_rtp_timestamp) {
|
||||
return;
|
||||
}
|
||||
|
||||
TimeDelta max_wait = timeout_tracker_.TimeUntilTimeout();
|
||||
// Ensures the frame is scheduled for decode before the stream times out.
|
||||
// This is otherwise a race condition.
|
||||
max_wait = std::max(max_wait - TimeDelta::Millis(1), TimeDelta::Zero());
|
||||
absl::optional<FrameDecodeTiming::FrameSchedule> schedule;
|
||||
while (decodable_tu_info) {
|
||||
schedule = decode_timing_.OnFrameBufferUpdated(
|
||||
decodable_tu_info->next_rtp_timestamp,
|
||||
decodable_tu_info->last_rtp_timestamp, max_wait,
|
||||
IsTooManyFramesQueued());
|
||||
if (schedule) {
|
||||
// Don't schedule if already waiting for the same frame.
|
||||
if (frame_decode_scheduler_->ScheduledRtpTimestamp() !=
|
||||
decodable_tu_info->next_rtp_timestamp) {
|
||||
frame_decode_scheduler_->CancelOutstanding();
|
||||
frame_decode_scheduler_->ScheduleFrame(
|
||||
decodable_tu_info->next_rtp_timestamp, *schedule,
|
||||
absl::bind_front(&VideoStreamBufferController::FrameReadyForDecode,
|
||||
this));
|
||||
}
|
||||
return;
|
||||
}
|
||||
// If no schedule for current rtp, drop and try again.
|
||||
buffer_->DropNextDecodableTemporalUnit();
|
||||
decodable_tu_info = buffer_->DecodableTemporalUnitsInfo();
|
||||
}
|
||||
}
|
||||
|
||||
} // namespace webrtc
|
135
video/video_stream_buffer_controller.h
Normal file
135
video/video_stream_buffer_controller.h
Normal file
@ -0,0 +1,135 @@
|
||||
/*
|
||||
* Copyright (c) 2022 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 VIDEO_VIDEO_STREAM_BUFFER_CONTROLLER_H_
|
||||
#define VIDEO_VIDEO_STREAM_BUFFER_CONTROLLER_H_
|
||||
|
||||
#include <memory>
|
||||
|
||||
#include "api/field_trials_view.h"
|
||||
#include "api/task_queue/task_queue_base.h"
|
||||
#include "api/video/encoded_frame.h"
|
||||
#include "api/video/frame_buffer.h"
|
||||
#include "modules/video_coding/include/video_coding_defines.h"
|
||||
#include "modules/video_coding/timing/inter_frame_delay.h"
|
||||
#include "modules/video_coding/timing/jitter_estimator.h"
|
||||
#include "modules/video_coding/timing/timing.h"
|
||||
#include "rtc_base/experiments/rtt_mult_experiment.h"
|
||||
#include "system_wrappers/include/clock.h"
|
||||
#include "video/decode_synchronizer.h"
|
||||
#include "video/video_receive_stream_timeout_tracker.h"
|
||||
|
||||
namespace webrtc {
|
||||
|
||||
class FrameSchedulingReceiver {
|
||||
public:
|
||||
virtual ~FrameSchedulingReceiver() = default;
|
||||
|
||||
virtual void OnEncodedFrame(std::unique_ptr<EncodedFrame> frame) = 0;
|
||||
virtual void OnDecodableFrameTimeout(TimeDelta wait_time) = 0;
|
||||
};
|
||||
|
||||
class VideoStreamBufferController {
|
||||
public:
|
||||
static std::unique_ptr<VideoStreamBufferController> CreateFromFieldTrial(
|
||||
Clock* clock,
|
||||
TaskQueueBase* worker_queue,
|
||||
VCMTiming* timing,
|
||||
VCMReceiveStatisticsCallback* stats_proxy,
|
||||
TaskQueueBase* decode_queue,
|
||||
FrameSchedulingReceiver* receiver,
|
||||
TimeDelta max_wait_for_keyframe,
|
||||
TimeDelta max_wait_for_frame,
|
||||
DecodeSynchronizer* decode_sync,
|
||||
const FieldTrialsView& field_trials);
|
||||
virtual ~VideoStreamBufferController() = default;
|
||||
VideoStreamBufferController(
|
||||
Clock* clock,
|
||||
TaskQueueBase* worker_queue,
|
||||
VCMTiming* timing,
|
||||
VCMReceiveStatisticsCallback* stats_proxy,
|
||||
TaskQueueBase* decode_queue,
|
||||
FrameSchedulingReceiver* receiver,
|
||||
TimeDelta max_wait_for_keyframe,
|
||||
TimeDelta max_wait_for_frame,
|
||||
std::unique_ptr<FrameDecodeScheduler> frame_decode_scheduler,
|
||||
const FieldTrialsView& field_trials);
|
||||
|
||||
void StopOnWorker();
|
||||
void SetProtectionMode(VCMVideoProtection protection_mode);
|
||||
void Clear();
|
||||
absl::optional<int64_t> InsertFrame(std::unique_ptr<EncodedFrame> frame);
|
||||
void UpdateRtt(int64_t max_rtt_ms);
|
||||
void SetMaxWaits(TimeDelta max_wait_for_keyframe,
|
||||
TimeDelta max_wait_for_frame);
|
||||
void StartNextDecode(bool keyframe_required);
|
||||
int Size();
|
||||
|
||||
private:
|
||||
void OnFrameReady(
|
||||
absl::InlinedVector<std::unique_ptr<EncodedFrame>, 4> frames,
|
||||
Timestamp render_time);
|
||||
void OnTimeout(TimeDelta delay);
|
||||
void FrameReadyForDecode(uint32_t rtp_timestamp, Timestamp render_time);
|
||||
void UpdateDroppedFrames() RTC_RUN_ON(&worker_sequence_checker_);
|
||||
void UpdateJitterDelay();
|
||||
void UpdateTimingFrameInfo();
|
||||
bool IsTooManyFramesQueued() const RTC_RUN_ON(&worker_sequence_checker_);
|
||||
void ForceKeyFrameReleaseImmediately() RTC_RUN_ON(&worker_sequence_checker_);
|
||||
void MaybeScheduleFrameForRelease() RTC_RUN_ON(&worker_sequence_checker_);
|
||||
|
||||
RTC_NO_UNIQUE_ADDRESS SequenceChecker worker_sequence_checker_;
|
||||
const FieldTrialsView& field_trials_;
|
||||
const absl::optional<RttMultExperiment::Settings> rtt_mult_settings_ =
|
||||
RttMultExperiment::GetRttMultValue();
|
||||
Clock* const clock_;
|
||||
TaskQueueBase* const worker_queue_;
|
||||
TaskQueueBase* const decode_queue_;
|
||||
VCMReceiveStatisticsCallback* const stats_proxy_;
|
||||
FrameSchedulingReceiver* const receiver_ RTC_PT_GUARDED_BY(decode_queue_);
|
||||
VCMTiming* const timing_;
|
||||
const std::unique_ptr<FrameDecodeScheduler> frame_decode_scheduler_
|
||||
RTC_GUARDED_BY(&worker_sequence_checker_);
|
||||
|
||||
JitterEstimator jitter_estimator_ RTC_GUARDED_BY(&worker_sequence_checker_);
|
||||
InterFrameDelay inter_frame_delay_ RTC_GUARDED_BY(&worker_sequence_checker_);
|
||||
bool keyframe_required_ RTC_GUARDED_BY(&worker_sequence_checker_) = false;
|
||||
std::unique_ptr<FrameBuffer> buffer_
|
||||
RTC_GUARDED_BY(&worker_sequence_checker_);
|
||||
FrameDecodeTiming decode_timing_ RTC_GUARDED_BY(&worker_sequence_checker_);
|
||||
VideoReceiveStreamTimeoutTracker timeout_tracker_
|
||||
RTC_GUARDED_BY(&worker_sequence_checker_);
|
||||
int frames_dropped_before_last_new_frame_
|
||||
RTC_GUARDED_BY(&worker_sequence_checker_) = 0;
|
||||
VCMVideoProtection protection_mode_
|
||||
RTC_GUARDED_BY(&worker_sequence_checker_) = kProtectionNack;
|
||||
|
||||
// This flag guards frames from queuing in front of the decoder. Without this
|
||||
// guard, encoded frames will not wait for the decoder to finish decoding a
|
||||
// frame and just queue up, meaning frames will not be dropped or
|
||||
// fast-forwarded when the decoder is slow or hangs.
|
||||
bool decoder_ready_for_new_frame_ RTC_GUARDED_BY(&worker_sequence_checker_) =
|
||||
false;
|
||||
|
||||
// Maximum number of frames in the decode queue to allow pacing. If the
|
||||
// queue grows beyond the max limit, pacing will be disabled and frames will
|
||||
// be pushed to the decoder as soon as possible. This only has an effect
|
||||
// when the low-latency rendering path is active, which is indicated by
|
||||
// the frame's render time == 0.
|
||||
FieldTrialParameter<unsigned> zero_playout_delay_max_decode_queue_size_;
|
||||
|
||||
rtc::scoped_refptr<PendingTaskSafetyFlag> decode_safety_ =
|
||||
PendingTaskSafetyFlag::CreateDetached();
|
||||
ScopedTaskSafety worker_safety_;
|
||||
};
|
||||
|
||||
} // namespace webrtc
|
||||
|
||||
#endif // VIDEO_VIDEO_STREAM_BUFFER_CONTROLLER_H_
|
@ -8,7 +8,7 @@
|
||||
* be found in the AUTHORS file in the root of the source tree.
|
||||
*/
|
||||
|
||||
#include "video/frame_buffer_proxy.h"
|
||||
#include "video/video_stream_buffer_controller.h"
|
||||
|
||||
#include <stdint.h>
|
||||
|
||||
@ -107,11 +107,11 @@ class VCMReceiveStatisticsCallbackMock : public VCMReceiveStatisticsCallback {
|
||||
|
||||
constexpr auto kMaxWaitForKeyframe = TimeDelta::Millis(500);
|
||||
constexpr auto kMaxWaitForFrame = TimeDelta::Millis(1500);
|
||||
class FrameBufferProxyFixture
|
||||
class VideoStreamBufferControllerFixture
|
||||
: public ::testing::WithParamInterface<std::string>,
|
||||
public FrameSchedulingReceiver {
|
||||
public:
|
||||
FrameBufferProxyFixture()
|
||||
VideoStreamBufferControllerFixture()
|
||||
: field_trials_(GetParam()),
|
||||
time_controller_(kClockStart),
|
||||
clock_(time_controller_.GetClock()),
|
||||
@ -122,16 +122,17 @@ class FrameBufferProxyFixture
|
||||
TimeDelta::Millis(16)),
|
||||
decode_sync_(clock_, &fake_metronome_, run_loop_.task_queue()),
|
||||
timing_(clock_, field_trials_),
|
||||
proxy_(FrameBufferProxy::CreateFromFieldTrial(clock_,
|
||||
run_loop_.task_queue(),
|
||||
&timing_,
|
||||
&stats_callback_,
|
||||
decode_queue_.Get(),
|
||||
this,
|
||||
kMaxWaitForKeyframe,
|
||||
kMaxWaitForFrame,
|
||||
&decode_sync_,
|
||||
field_trials_)) {
|
||||
buffer_(VideoStreamBufferController::CreateFromFieldTrial(
|
||||
clock_,
|
||||
run_loop_.task_queue(),
|
||||
&timing_,
|
||||
&stats_callback_,
|
||||
decode_queue_.Get(),
|
||||
this,
|
||||
kMaxWaitForKeyframe,
|
||||
kMaxWaitForFrame,
|
||||
&decode_sync_,
|
||||
field_trials_)) {
|
||||
// Avoid starting with negative render times.
|
||||
timing_.set_min_playout_delay(TimeDelta::Millis(10));
|
||||
|
||||
@ -140,9 +141,9 @@ class FrameBufferProxyFixture
|
||||
[this](auto num_dropped) { dropped_frames_ += num_dropped; });
|
||||
}
|
||||
|
||||
~FrameBufferProxyFixture() override {
|
||||
if (proxy_) {
|
||||
proxy_->StopOnWorker();
|
||||
~VideoStreamBufferControllerFixture() override {
|
||||
if (buffer_) {
|
||||
buffer_->StopOnWorker();
|
||||
}
|
||||
fake_metronome_.Stop();
|
||||
time_controller_.AdvanceTime(TimeDelta::Zero());
|
||||
@ -187,13 +188,13 @@ class FrameBufferProxyFixture
|
||||
|
||||
void StartNextDecode() {
|
||||
ResetLastResult();
|
||||
proxy_->StartNextDecode(false);
|
||||
buffer_->StartNextDecode(false);
|
||||
time_controller_.AdvanceTime(TimeDelta::Zero());
|
||||
}
|
||||
|
||||
void StartNextDecodeForceKeyframe() {
|
||||
ResetLastResult();
|
||||
proxy_->StartNextDecode(true);
|
||||
buffer_->StartNextDecode(true);
|
||||
time_controller_.AdvanceTime(TimeDelta::Zero());
|
||||
}
|
||||
|
||||
@ -212,7 +213,7 @@ class FrameBufferProxyFixture
|
||||
VCMTiming timing_;
|
||||
|
||||
::testing::NiceMock<VCMReceiveStatisticsCallbackMock> stats_callback_;
|
||||
std::unique_ptr<FrameBufferProxy> proxy_;
|
||||
std::unique_ptr<VideoStreamBufferController> buffer_;
|
||||
|
||||
private:
|
||||
void SetWaitResult(WaitResult result) {
|
||||
@ -228,10 +229,12 @@ class FrameBufferProxyFixture
|
||||
absl::optional<WaitResult> wait_result_;
|
||||
};
|
||||
|
||||
class FrameBufferProxyTest : public ::testing::Test,
|
||||
public FrameBufferProxyFixture {};
|
||||
class VideoStreamBufferControllerTest
|
||||
: public ::testing::Test,
|
||||
public VideoStreamBufferControllerFixture {};
|
||||
|
||||
TEST_P(FrameBufferProxyTest, InitialTimeoutAfterKeyframeTimeoutPeriod) {
|
||||
TEST_P(VideoStreamBufferControllerTest,
|
||||
InitialTimeoutAfterKeyframeTimeoutPeriod) {
|
||||
StartNextDecodeForceKeyframe();
|
||||
// No frame inserted. Timeout expected.
|
||||
EXPECT_THAT(WaitForFrameOrTimeout(kMaxWaitForKeyframe), TimedOut());
|
||||
@ -245,22 +248,23 @@ TEST_P(FrameBufferProxyTest, InitialTimeoutAfterKeyframeTimeoutPeriod) {
|
||||
EXPECT_THAT(WaitForFrameOrTimeout(kMaxWaitForKeyframe), TimedOut());
|
||||
}
|
||||
|
||||
TEST_P(FrameBufferProxyTest, KeyFramesAreScheduled) {
|
||||
TEST_P(VideoStreamBufferControllerTest, KeyFramesAreScheduled) {
|
||||
StartNextDecodeForceKeyframe();
|
||||
time_controller_.AdvanceTime(TimeDelta::Millis(50));
|
||||
|
||||
auto frame = test::FakeFrameBuilder().Id(0).Time(0).AsLast().Build();
|
||||
proxy_->InsertFrame(std::move(frame));
|
||||
buffer_->InsertFrame(std::move(frame));
|
||||
|
||||
EXPECT_THAT(WaitForFrameOrTimeout(TimeDelta::Zero()), Frame(test::WithId(0)));
|
||||
}
|
||||
|
||||
TEST_P(FrameBufferProxyTest, DeltaFrameTimeoutAfterKeyframeExtracted) {
|
||||
TEST_P(VideoStreamBufferControllerTest,
|
||||
DeltaFrameTimeoutAfterKeyframeExtracted) {
|
||||
StartNextDecodeForceKeyframe();
|
||||
|
||||
time_controller_.AdvanceTime(TimeDelta::Millis(50));
|
||||
auto frame = test::FakeFrameBuilder().Id(0).Time(0).AsLast().Build();
|
||||
proxy_->InsertFrame(std::move(frame));
|
||||
buffer_->InsertFrame(std::move(frame));
|
||||
EXPECT_THAT(WaitForFrameOrTimeout(kMaxWaitForKeyframe),
|
||||
Frame(test::WithId(0)));
|
||||
|
||||
@ -275,45 +279,45 @@ TEST_P(FrameBufferProxyTest, DeltaFrameTimeoutAfterKeyframeExtracted) {
|
||||
}
|
||||
}
|
||||
|
||||
TEST_P(FrameBufferProxyTest, DependantFramesAreScheduled) {
|
||||
TEST_P(VideoStreamBufferControllerTest, DependantFramesAreScheduled) {
|
||||
StartNextDecodeForceKeyframe();
|
||||
proxy_->InsertFrame(test::FakeFrameBuilder().Id(0).Time(0).AsLast().Build());
|
||||
buffer_->InsertFrame(test::FakeFrameBuilder().Id(0).Time(0).AsLast().Build());
|
||||
EXPECT_THAT(WaitForFrameOrTimeout(TimeDelta::Zero()), Frame(test::WithId(0)));
|
||||
|
||||
StartNextDecode();
|
||||
|
||||
time_controller_.AdvanceTime(kFps30Delay);
|
||||
proxy_->InsertFrame(test::FakeFrameBuilder()
|
||||
.Id(1)
|
||||
.Time(kFps30Rtp)
|
||||
.AsLast()
|
||||
.Refs({0})
|
||||
.Build());
|
||||
buffer_->InsertFrame(test::FakeFrameBuilder()
|
||||
.Id(1)
|
||||
.Time(kFps30Rtp)
|
||||
.AsLast()
|
||||
.Refs({0})
|
||||
.Build());
|
||||
EXPECT_THAT(WaitForFrameOrTimeout(kFps30Delay), Frame(test::WithId(1)));
|
||||
}
|
||||
|
||||
TEST_P(FrameBufferProxyTest, SpatialLayersAreScheduled) {
|
||||
TEST_P(VideoStreamBufferControllerTest, SpatialLayersAreScheduled) {
|
||||
StartNextDecodeForceKeyframe();
|
||||
proxy_->InsertFrame(WithReceiveTimeFromRtpTimestamp(
|
||||
buffer_->InsertFrame(WithReceiveTimeFromRtpTimestamp(
|
||||
test::FakeFrameBuilder().Id(0).SpatialLayer(0).Time(0).Build()));
|
||||
proxy_->InsertFrame(WithReceiveTimeFromRtpTimestamp(
|
||||
buffer_->InsertFrame(WithReceiveTimeFromRtpTimestamp(
|
||||
test::FakeFrameBuilder().Id(1).SpatialLayer(1).Time(0).Build()));
|
||||
proxy_->InsertFrame(WithReceiveTimeFromRtpTimestamp(
|
||||
buffer_->InsertFrame(WithReceiveTimeFromRtpTimestamp(
|
||||
test::FakeFrameBuilder().Id(2).SpatialLayer(2).Time(0).AsLast().Build()));
|
||||
EXPECT_THAT(
|
||||
WaitForFrameOrTimeout(TimeDelta::Zero()),
|
||||
Frame(AllOf(test::WithId(0), test::FrameWithSize(3 * kFrameSize))));
|
||||
|
||||
proxy_->InsertFrame(WithReceiveTimeFromRtpTimestamp(
|
||||
buffer_->InsertFrame(WithReceiveTimeFromRtpTimestamp(
|
||||
test::FakeFrameBuilder().Id(3).Time(kFps30Rtp).SpatialLayer(0).Build()));
|
||||
proxy_->InsertFrame(WithReceiveTimeFromRtpTimestamp(
|
||||
buffer_->InsertFrame(WithReceiveTimeFromRtpTimestamp(
|
||||
test::FakeFrameBuilder().Id(4).Time(kFps30Rtp).SpatialLayer(1).Build()));
|
||||
proxy_->InsertFrame(WithReceiveTimeFromRtpTimestamp(test::FakeFrameBuilder()
|
||||
.Id(5)
|
||||
.Time(kFps30Rtp)
|
||||
.SpatialLayer(2)
|
||||
.AsLast()
|
||||
.Build()));
|
||||
buffer_->InsertFrame(WithReceiveTimeFromRtpTimestamp(test::FakeFrameBuilder()
|
||||
.Id(5)
|
||||
.Time(kFps30Rtp)
|
||||
.SpatialLayer(2)
|
||||
.AsLast()
|
||||
.Build()));
|
||||
|
||||
StartNextDecode();
|
||||
EXPECT_THAT(
|
||||
@ -321,41 +325,42 @@ TEST_P(FrameBufferProxyTest, SpatialLayersAreScheduled) {
|
||||
Frame(AllOf(test::WithId(3), test::FrameWithSize(3 * kFrameSize))));
|
||||
}
|
||||
|
||||
TEST_P(FrameBufferProxyTest, OutstandingFrameTasksAreCancelledAfterDeletion) {
|
||||
TEST_P(VideoStreamBufferControllerTest,
|
||||
OutstandingFrameTasksAreCancelledAfterDeletion) {
|
||||
StartNextDecodeForceKeyframe();
|
||||
proxy_->InsertFrame(WithReceiveTimeFromRtpTimestamp(
|
||||
buffer_->InsertFrame(WithReceiveTimeFromRtpTimestamp(
|
||||
test::FakeFrameBuilder().Id(0).Time(0).AsLast().Build()));
|
||||
// Get keyframe. Delta frame should now be scheduled.
|
||||
EXPECT_THAT(WaitForFrameOrTimeout(TimeDelta::Zero()), Frame(test::WithId(0)));
|
||||
|
||||
StartNextDecode();
|
||||
proxy_->InsertFrame(WithReceiveTimeFromRtpTimestamp(test::FakeFrameBuilder()
|
||||
.Id(1)
|
||||
.Time(kFps30Rtp)
|
||||
.AsLast()
|
||||
.Refs({0})
|
||||
.Build()));
|
||||
proxy_->StopOnWorker();
|
||||
buffer_->InsertFrame(WithReceiveTimeFromRtpTimestamp(test::FakeFrameBuilder()
|
||||
.Id(1)
|
||||
.Time(kFps30Rtp)
|
||||
.AsLast()
|
||||
.Refs({0})
|
||||
.Build()));
|
||||
buffer_->StopOnWorker();
|
||||
// Wait for 2x max wait time. Since we stopped, this should cause no timeouts
|
||||
// or frame-ready callbacks.
|
||||
EXPECT_THAT(WaitForFrameOrTimeout(kMaxWaitForFrame * 2), Eq(absl::nullopt));
|
||||
}
|
||||
|
||||
TEST_P(FrameBufferProxyTest, FramesWaitForDecoderToComplete) {
|
||||
TEST_P(VideoStreamBufferControllerTest, FramesWaitForDecoderToComplete) {
|
||||
StartNextDecodeForceKeyframe();
|
||||
|
||||
// Start with a keyframe.
|
||||
proxy_->InsertFrame(test::FakeFrameBuilder().Id(0).Time(0).AsLast().Build());
|
||||
buffer_->InsertFrame(test::FakeFrameBuilder().Id(0).Time(0).AsLast().Build());
|
||||
EXPECT_THAT(WaitForFrameOrTimeout(TimeDelta::Zero()), Frame(test::WithId(0)));
|
||||
|
||||
ResetLastResult();
|
||||
// Insert a delta frame.
|
||||
proxy_->InsertFrame(test::FakeFrameBuilder()
|
||||
.Id(1)
|
||||
.Time(kFps30Rtp)
|
||||
.AsLast()
|
||||
.Refs({0})
|
||||
.Build());
|
||||
buffer_->InsertFrame(test::FakeFrameBuilder()
|
||||
.Id(1)
|
||||
.Time(kFps30Rtp)
|
||||
.AsLast()
|
||||
.Refs({0})
|
||||
.Build());
|
||||
|
||||
// Advancing time should not result in a frame since the scheduler has not
|
||||
// been signalled that we are ready.
|
||||
@ -365,12 +370,12 @@ TEST_P(FrameBufferProxyTest, FramesWaitForDecoderToComplete) {
|
||||
EXPECT_THAT(WaitForFrameOrTimeout(kFps30Delay), Frame(test::WithId(1)));
|
||||
}
|
||||
|
||||
TEST_P(FrameBufferProxyTest, LateFrameDropped) {
|
||||
TEST_P(VideoStreamBufferControllerTest, LateFrameDropped) {
|
||||
StartNextDecodeForceKeyframe();
|
||||
// F1
|
||||
// /
|
||||
// F0 --> F2
|
||||
proxy_->InsertFrame(test::FakeFrameBuilder().Id(0).Time(0).AsLast().Build());
|
||||
buffer_->InsertFrame(test::FakeFrameBuilder().Id(0).Time(0).AsLast().Build());
|
||||
// Start with a keyframe.
|
||||
EXPECT_THAT(WaitForFrameOrTimeout(TimeDelta::Zero()), Frame(test::WithId(0)));
|
||||
|
||||
@ -378,50 +383,50 @@ TEST_P(FrameBufferProxyTest, LateFrameDropped) {
|
||||
|
||||
// Simulate late F1 which arrives after F2.
|
||||
time_controller_.AdvanceTime(kFps30Delay * 2);
|
||||
proxy_->InsertFrame(test::FakeFrameBuilder()
|
||||
.Id(2)
|
||||
.Time(2 * kFps30Rtp)
|
||||
.AsLast()
|
||||
.Refs({0})
|
||||
.Build());
|
||||
buffer_->InsertFrame(test::FakeFrameBuilder()
|
||||
.Id(2)
|
||||
.Time(2 * kFps30Rtp)
|
||||
.AsLast()
|
||||
.Refs({0})
|
||||
.Build());
|
||||
EXPECT_THAT(WaitForFrameOrTimeout(kFps30Delay), Frame(test::WithId(2)));
|
||||
|
||||
StartNextDecode();
|
||||
|
||||
proxy_->InsertFrame(test::FakeFrameBuilder()
|
||||
.Id(1)
|
||||
.Time(1 * kFps30Rtp)
|
||||
.AsLast()
|
||||
.Refs({0})
|
||||
.Build());
|
||||
buffer_->InsertFrame(test::FakeFrameBuilder()
|
||||
.Id(1)
|
||||
.Time(1 * kFps30Rtp)
|
||||
.AsLast()
|
||||
.Refs({0})
|
||||
.Build());
|
||||
// Confirm frame 1 is never scheduled by timing out.
|
||||
EXPECT_THAT(WaitForFrameOrTimeout(kMaxWaitForFrame), TimedOut());
|
||||
}
|
||||
|
||||
TEST_P(FrameBufferProxyTest, FramesFastForwardOnSystemHalt) {
|
||||
TEST_P(VideoStreamBufferControllerTest, FramesFastForwardOnSystemHalt) {
|
||||
StartNextDecodeForceKeyframe();
|
||||
// F1
|
||||
// /
|
||||
// F0 --> F2
|
||||
proxy_->InsertFrame(test::FakeFrameBuilder().Id(0).Time(0).AsLast().Build());
|
||||
buffer_->InsertFrame(test::FakeFrameBuilder().Id(0).Time(0).AsLast().Build());
|
||||
|
||||
// Start with a keyframe.
|
||||
EXPECT_THAT(WaitForFrameOrTimeout(TimeDelta::Zero()), Frame(test::WithId(0)));
|
||||
|
||||
time_controller_.AdvanceTime(kFps30Delay);
|
||||
proxy_->InsertFrame(test::FakeFrameBuilder()
|
||||
.Id(1)
|
||||
.Time(kFps30Rtp)
|
||||
.AsLast()
|
||||
.Refs({0})
|
||||
.Build());
|
||||
buffer_->InsertFrame(test::FakeFrameBuilder()
|
||||
.Id(1)
|
||||
.Time(kFps30Rtp)
|
||||
.AsLast()
|
||||
.Refs({0})
|
||||
.Build());
|
||||
time_controller_.AdvanceTime(kFps30Delay);
|
||||
proxy_->InsertFrame(test::FakeFrameBuilder()
|
||||
.Id(2)
|
||||
.Time(2 * kFps30Rtp)
|
||||
.AsLast()
|
||||
.Refs({0})
|
||||
.Build());
|
||||
buffer_->InsertFrame(test::FakeFrameBuilder()
|
||||
.Id(2)
|
||||
.Time(2 * kFps30Rtp)
|
||||
.AsLast()
|
||||
.Refs({0})
|
||||
.Build());
|
||||
|
||||
// Halting time should result in F1 being skipped.
|
||||
time_controller_.AdvanceTime(kFps30Delay * 2);
|
||||
@ -430,54 +435,54 @@ TEST_P(FrameBufferProxyTest, FramesFastForwardOnSystemHalt) {
|
||||
EXPECT_EQ(dropped_frames(), 1);
|
||||
}
|
||||
|
||||
TEST_P(FrameBufferProxyTest, ForceKeyFrame) {
|
||||
TEST_P(VideoStreamBufferControllerTest, ForceKeyFrame) {
|
||||
StartNextDecodeForceKeyframe();
|
||||
// Initial keyframe.
|
||||
proxy_->InsertFrame(test::FakeFrameBuilder().Id(0).Time(0).AsLast().Build());
|
||||
buffer_->InsertFrame(test::FakeFrameBuilder().Id(0).Time(0).AsLast().Build());
|
||||
EXPECT_THAT(WaitForFrameOrTimeout(TimeDelta::Zero()), Frame(test::WithId(0)));
|
||||
|
||||
StartNextDecodeForceKeyframe();
|
||||
|
||||
// F2 is the next keyframe, and should be extracted since a keyframe was
|
||||
// forced.
|
||||
proxy_->InsertFrame(test::FakeFrameBuilder()
|
||||
.Id(1)
|
||||
.Time(kFps30Rtp)
|
||||
.AsLast()
|
||||
.Refs({0})
|
||||
.Build());
|
||||
proxy_->InsertFrame(
|
||||
buffer_->InsertFrame(test::FakeFrameBuilder()
|
||||
.Id(1)
|
||||
.Time(kFps30Rtp)
|
||||
.AsLast()
|
||||
.Refs({0})
|
||||
.Build());
|
||||
buffer_->InsertFrame(
|
||||
test::FakeFrameBuilder().Id(2).Time(kFps30Rtp * 2).AsLast().Build());
|
||||
|
||||
EXPECT_THAT(WaitForFrameOrTimeout(kFps30Delay * 3), Frame(test::WithId(2)));
|
||||
}
|
||||
|
||||
TEST_P(FrameBufferProxyTest, SlowDecoderDropsTemporalLayers) {
|
||||
TEST_P(VideoStreamBufferControllerTest, SlowDecoderDropsTemporalLayers) {
|
||||
StartNextDecodeForceKeyframe();
|
||||
// 2 temporal layers, at 15fps per layer to make 30fps total.
|
||||
// Decoder is slower than 30fps, so last_frame() will be skipped.
|
||||
// F1 --> F3 --> F5
|
||||
// / / /
|
||||
// F0 --> F2 --> F4
|
||||
proxy_->InsertFrame(test::FakeFrameBuilder().Id(0).Time(0).AsLast().Build());
|
||||
buffer_->InsertFrame(test::FakeFrameBuilder().Id(0).Time(0).AsLast().Build());
|
||||
// Keyframe received.
|
||||
// Don't start next decode until slow delay.
|
||||
EXPECT_THAT(WaitForFrameOrTimeout(TimeDelta::Zero()), Frame(test::WithId(0)));
|
||||
|
||||
time_controller_.AdvanceTime(kFps30Delay);
|
||||
proxy_->InsertFrame(test::FakeFrameBuilder()
|
||||
.Id(1)
|
||||
.Time(1 * kFps30Rtp)
|
||||
.Refs({0})
|
||||
.AsLast()
|
||||
.Build());
|
||||
buffer_->InsertFrame(test::FakeFrameBuilder()
|
||||
.Id(1)
|
||||
.Time(1 * kFps30Rtp)
|
||||
.Refs({0})
|
||||
.AsLast()
|
||||
.Build());
|
||||
time_controller_.AdvanceTime(kFps30Delay);
|
||||
proxy_->InsertFrame(test::FakeFrameBuilder()
|
||||
.Id(2)
|
||||
.Time(2 * kFps30Rtp)
|
||||
.Refs({0})
|
||||
.AsLast()
|
||||
.Build());
|
||||
buffer_->InsertFrame(test::FakeFrameBuilder()
|
||||
.Id(2)
|
||||
.Time(2 * kFps30Rtp)
|
||||
.Refs({0})
|
||||
.AsLast()
|
||||
.Build());
|
||||
|
||||
// Simulate decode taking 3x FPS rate.
|
||||
time_controller_.AdvanceTime(kFps30Delay * 1.5);
|
||||
@ -487,19 +492,19 @@ TEST_P(FrameBufferProxyTest, SlowDecoderDropsTemporalLayers) {
|
||||
EXPECT_EQ(dropped_frames(), 1);
|
||||
time_controller_.AdvanceTime(kFps30Delay / 2);
|
||||
|
||||
proxy_->InsertFrame(test::FakeFrameBuilder()
|
||||
.Id(3)
|
||||
.Time(3 * kFps30Rtp)
|
||||
.Refs({1, 2})
|
||||
.AsLast()
|
||||
.Build());
|
||||
buffer_->InsertFrame(test::FakeFrameBuilder()
|
||||
.Id(3)
|
||||
.Time(3 * kFps30Rtp)
|
||||
.Refs({1, 2})
|
||||
.AsLast()
|
||||
.Build());
|
||||
time_controller_.AdvanceTime(kFps30Delay / 2);
|
||||
proxy_->InsertFrame(test::FakeFrameBuilder()
|
||||
.Id(4)
|
||||
.Time(4 * kFps30Rtp)
|
||||
.Refs({2})
|
||||
.AsLast()
|
||||
.Build());
|
||||
buffer_->InsertFrame(test::FakeFrameBuilder()
|
||||
.Id(4)
|
||||
.Time(4 * kFps30Rtp)
|
||||
.Refs({2})
|
||||
.AsLast()
|
||||
.Build());
|
||||
time_controller_.AdvanceTime(kFps30Delay / 2);
|
||||
|
||||
// F4 is the best frame since decoding was so slow that F1 is too old.
|
||||
@ -507,12 +512,12 @@ TEST_P(FrameBufferProxyTest, SlowDecoderDropsTemporalLayers) {
|
||||
StartNextDecode();
|
||||
EXPECT_THAT(WaitForFrameOrTimeout(kFps30Delay), Frame(test::WithId(4)));
|
||||
|
||||
proxy_->InsertFrame(test::FakeFrameBuilder()
|
||||
.Id(5)
|
||||
.Time(5 * kFps30Rtp)
|
||||
.Refs({3, 4})
|
||||
.AsLast()
|
||||
.Build());
|
||||
buffer_->InsertFrame(test::FakeFrameBuilder()
|
||||
.Id(5)
|
||||
.Time(5 * kFps30Rtp)
|
||||
.Refs({3, 4})
|
||||
.AsLast()
|
||||
.Build());
|
||||
time_controller_.AdvanceTime(kFps30Delay / 2);
|
||||
|
||||
// F5 is not decodable since F4 was decoded, so a timeout is expected.
|
||||
@ -526,35 +531,36 @@ TEST_P(FrameBufferProxyTest, SlowDecoderDropsTemporalLayers) {
|
||||
// EXPECT_EQ(dropped_frames(), 2);
|
||||
}
|
||||
|
||||
TEST_P(FrameBufferProxyTest, NewFrameInsertedWhileWaitingToReleaseFrame) {
|
||||
TEST_P(VideoStreamBufferControllerTest,
|
||||
NewFrameInsertedWhileWaitingToReleaseFrame) {
|
||||
StartNextDecodeForceKeyframe();
|
||||
// Initial keyframe.
|
||||
proxy_->InsertFrame(WithReceiveTimeFromRtpTimestamp(
|
||||
buffer_->InsertFrame(WithReceiveTimeFromRtpTimestamp(
|
||||
test::FakeFrameBuilder().Id(0).Time(0).AsLast().Build()));
|
||||
EXPECT_THAT(WaitForFrameOrTimeout(TimeDelta::Zero()), Frame(test::WithId(0)));
|
||||
|
||||
time_controller_.AdvanceTime(kFps30Delay / 2);
|
||||
proxy_->InsertFrame(WithReceiveTimeFromRtpTimestamp(test::FakeFrameBuilder()
|
||||
.Id(1)
|
||||
.Time(kFps30Rtp)
|
||||
.Refs({0})
|
||||
.AsLast()
|
||||
.Build()));
|
||||
buffer_->InsertFrame(WithReceiveTimeFromRtpTimestamp(test::FakeFrameBuilder()
|
||||
.Id(1)
|
||||
.Time(kFps30Rtp)
|
||||
.Refs({0})
|
||||
.AsLast()
|
||||
.Build()));
|
||||
StartNextDecode();
|
||||
EXPECT_THAT(WaitForFrameOrTimeout(TimeDelta::Zero()), Eq(absl::nullopt));
|
||||
|
||||
// Scheduler is waiting to deliver Frame 1 now. Insert Frame 2. Frame 1 should
|
||||
// be delivered still.
|
||||
proxy_->InsertFrame(WithReceiveTimeFromRtpTimestamp(test::FakeFrameBuilder()
|
||||
.Id(2)
|
||||
.Time(kFps30Rtp * 2)
|
||||
.Refs({0})
|
||||
.AsLast()
|
||||
.Build()));
|
||||
buffer_->InsertFrame(WithReceiveTimeFromRtpTimestamp(test::FakeFrameBuilder()
|
||||
.Id(2)
|
||||
.Time(kFps30Rtp * 2)
|
||||
.Refs({0})
|
||||
.AsLast()
|
||||
.Build()));
|
||||
EXPECT_THAT(WaitForFrameOrTimeout(kFps30Delay), Frame(test::WithId(1)));
|
||||
}
|
||||
|
||||
TEST_P(FrameBufferProxyTest, SameFrameNotScheduledTwice) {
|
||||
TEST_P(VideoStreamBufferControllerTest, SameFrameNotScheduledTwice) {
|
||||
// A frame could be scheduled twice if last_frame() arrive out-of-order but
|
||||
// the older frame is old enough to be fast forwarded.
|
||||
//
|
||||
@ -567,7 +573,7 @@ TEST_P(FrameBufferProxyTest, SameFrameNotScheduledTwice) {
|
||||
StartNextDecodeForceKeyframe();
|
||||
|
||||
// First keyframe.
|
||||
proxy_->InsertFrame(WithReceiveTimeFromRtpTimestamp(
|
||||
buffer_->InsertFrame(WithReceiveTimeFromRtpTimestamp(
|
||||
test::FakeFrameBuilder().Id(0).Time(0).AsLast().Build()));
|
||||
EXPECT_THAT(WaitForFrameOrTimeout(TimeDelta::Millis(15)),
|
||||
Frame(test::WithId(0)));
|
||||
@ -576,25 +582,25 @@ TEST_P(FrameBufferProxyTest, SameFrameNotScheduledTwice) {
|
||||
|
||||
// Warmup VCMTiming for 30fps.
|
||||
for (int i = 1; i <= 30; ++i) {
|
||||
proxy_->InsertFrame(WithReceiveTimeFromRtpTimestamp(
|
||||
buffer_->InsertFrame(WithReceiveTimeFromRtpTimestamp(
|
||||
test::FakeFrameBuilder().Id(i).Time(i * kFps30Rtp).AsLast().Build()));
|
||||
EXPECT_THAT(WaitForFrameOrTimeout(kFps30Delay), Frame(test::WithId(i)));
|
||||
StartNextDecode();
|
||||
}
|
||||
|
||||
// F2 arrives and is scheduled.
|
||||
proxy_->InsertFrame(WithReceiveTimeFromRtpTimestamp(
|
||||
buffer_->InsertFrame(WithReceiveTimeFromRtpTimestamp(
|
||||
test::FakeFrameBuilder().Id(32).Time(32 * kFps30Rtp).AsLast().Build()));
|
||||
|
||||
// F3 arrives before F2 is extracted.
|
||||
time_controller_.AdvanceTime(kFps30Delay);
|
||||
proxy_->InsertFrame(WithReceiveTimeFromRtpTimestamp(
|
||||
buffer_->InsertFrame(WithReceiveTimeFromRtpTimestamp(
|
||||
test::FakeFrameBuilder().Id(33).Time(33 * kFps30Rtp).AsLast().Build()));
|
||||
|
||||
// F1 arrives and is fast-forwarded since it is too late.
|
||||
// F2 is already scheduled and should not be rescheduled.
|
||||
time_controller_.AdvanceTime(kFps30Delay / 2);
|
||||
proxy_->InsertFrame(WithReceiveTimeFromRtpTimestamp(
|
||||
buffer_->InsertFrame(WithReceiveTimeFromRtpTimestamp(
|
||||
test::FakeFrameBuilder().Id(31).Time(31 * kFps30Rtp).AsLast().Build()));
|
||||
|
||||
EXPECT_THAT(WaitForFrameOrTimeout(kFps30Delay), Frame(test::WithId(32)));
|
||||
@ -606,7 +612,7 @@ TEST_P(FrameBufferProxyTest, SameFrameNotScheduledTwice) {
|
||||
EXPECT_EQ(dropped_frames(), 1);
|
||||
}
|
||||
|
||||
TEST_P(FrameBufferProxyTest, TestStatsCallback) {
|
||||
TEST_P(VideoStreamBufferControllerTest, TestStatsCallback) {
|
||||
EXPECT_CALL(stats_callback_,
|
||||
OnCompleteFrame(true, kFrameSize, VideoContentType::UNSPECIFIED));
|
||||
EXPECT_CALL(stats_callback_, OnFrameBufferTimingsUpdated);
|
||||
@ -614,33 +620,36 @@ TEST_P(FrameBufferProxyTest, TestStatsCallback) {
|
||||
// Fake timing having received decoded frame.
|
||||
timing_.StopDecodeTimer(TimeDelta::Millis(1), clock_->CurrentTime());
|
||||
StartNextDecodeForceKeyframe();
|
||||
proxy_->InsertFrame(test::FakeFrameBuilder().Id(0).Time(0).AsLast().Build());
|
||||
buffer_->InsertFrame(test::FakeFrameBuilder().Id(0).Time(0).AsLast().Build());
|
||||
EXPECT_THAT(WaitForFrameOrTimeout(TimeDelta::Zero()), Frame(test::WithId(0)));
|
||||
|
||||
// Flush stats posted on the decode queue.
|
||||
time_controller_.AdvanceTime(TimeDelta::Zero());
|
||||
}
|
||||
|
||||
TEST_P(FrameBufferProxyTest, FrameCompleteCalledOnceForDuplicateFrame) {
|
||||
TEST_P(VideoStreamBufferControllerTest,
|
||||
FrameCompleteCalledOnceForDuplicateFrame) {
|
||||
EXPECT_CALL(stats_callback_,
|
||||
OnCompleteFrame(true, kFrameSize, VideoContentType::UNSPECIFIED))
|
||||
.Times(1);
|
||||
|
||||
StartNextDecodeForceKeyframe();
|
||||
proxy_->InsertFrame(test::FakeFrameBuilder().Id(0).Time(0).AsLast().Build());
|
||||
proxy_->InsertFrame(test::FakeFrameBuilder().Id(0).Time(0).AsLast().Build());
|
||||
buffer_->InsertFrame(test::FakeFrameBuilder().Id(0).Time(0).AsLast().Build());
|
||||
buffer_->InsertFrame(test::FakeFrameBuilder().Id(0).Time(0).AsLast().Build());
|
||||
// Flush stats posted on the decode queue.
|
||||
time_controller_.AdvanceTime(TimeDelta::Zero());
|
||||
}
|
||||
|
||||
TEST_P(FrameBufferProxyTest, FrameCompleteCalledOnceForSingleTemporalUnit) {
|
||||
TEST_P(VideoStreamBufferControllerTest,
|
||||
FrameCompleteCalledOnceForSingleTemporalUnit) {
|
||||
StartNextDecodeForceKeyframe();
|
||||
|
||||
// `OnCompleteFrame` should not be called for the first two frames since they
|
||||
// do not complete the temporal layer.
|
||||
EXPECT_CALL(stats_callback_, OnCompleteFrame(_, _, _)).Times(0);
|
||||
proxy_->InsertFrame(test::FakeFrameBuilder().Id(0).Time(0).Build());
|
||||
proxy_->InsertFrame(test::FakeFrameBuilder().Id(1).Time(0).Refs({0}).Build());
|
||||
buffer_->InsertFrame(test::FakeFrameBuilder().Id(0).Time(0).Build());
|
||||
buffer_->InsertFrame(
|
||||
test::FakeFrameBuilder().Id(1).Time(0).Refs({0}).Build());
|
||||
time_controller_.AdvanceTime(TimeDelta::Zero());
|
||||
// Flush stats posted on the decode queue.
|
||||
::testing::Mock::VerifyAndClearExpectations(&stats_callback_);
|
||||
@ -650,13 +659,14 @@ TEST_P(FrameBufferProxyTest, FrameCompleteCalledOnceForSingleTemporalUnit) {
|
||||
EXPECT_CALL(stats_callback_,
|
||||
OnCompleteFrame(false, kFrameSize, VideoContentType::UNSPECIFIED))
|
||||
.Times(1);
|
||||
proxy_->InsertFrame(
|
||||
buffer_->InsertFrame(
|
||||
test::FakeFrameBuilder().Id(2).Time(0).Refs({0, 1}).AsLast().Build());
|
||||
// Flush stats posted on the decode queue.
|
||||
time_controller_.AdvanceTime(TimeDelta::Zero());
|
||||
}
|
||||
|
||||
TEST_P(FrameBufferProxyTest, FrameCompleteCalledOnceForCompleteTemporalUnit) {
|
||||
TEST_P(VideoStreamBufferControllerTest,
|
||||
FrameCompleteCalledOnceForCompleteTemporalUnit) {
|
||||
// FrameBuffer2 logs the complete frame on the arrival of the last layer.
|
||||
StartNextDecodeForceKeyframe();
|
||||
|
||||
@ -664,8 +674,8 @@ TEST_P(FrameBufferProxyTest, FrameCompleteCalledOnceForCompleteTemporalUnit) {
|
||||
// do not complete the temporal layer. Frame 1 arrives later, at which time
|
||||
// this frame can finally be considered complete.
|
||||
EXPECT_CALL(stats_callback_, OnCompleteFrame(_, _, _)).Times(0);
|
||||
proxy_->InsertFrame(test::FakeFrameBuilder().Id(0).Time(0).Build());
|
||||
proxy_->InsertFrame(
|
||||
buffer_->InsertFrame(test::FakeFrameBuilder().Id(0).Time(0).Build());
|
||||
buffer_->InsertFrame(
|
||||
test::FakeFrameBuilder().Id(2).Time(0).Refs({0, 1}).AsLast().Build());
|
||||
time_controller_.AdvanceTime(TimeDelta::Zero());
|
||||
// Flush stats posted on the decode queue.
|
||||
@ -674,7 +684,8 @@ TEST_P(FrameBufferProxyTest, FrameCompleteCalledOnceForCompleteTemporalUnit) {
|
||||
EXPECT_CALL(stats_callback_,
|
||||
OnCompleteFrame(false, kFrameSize, VideoContentType::UNSPECIFIED))
|
||||
.Times(1);
|
||||
proxy_->InsertFrame(test::FakeFrameBuilder().Id(1).Time(0).Refs({0}).Build());
|
||||
buffer_->InsertFrame(
|
||||
test::FakeFrameBuilder().Id(1).Time(0).Refs({0}).Build());
|
||||
// Flush stats posted on the decode queue.
|
||||
time_controller_.AdvanceTime(TimeDelta::Zero());
|
||||
}
|
||||
@ -683,7 +694,7 @@ TEST_P(FrameBufferProxyTest, FrameCompleteCalledOnceForCompleteTemporalUnit) {
|
||||
// Since the test needs to wait for the timestamp to rollover, it has a fake
|
||||
// delay of around 6.5 hours. Even though time is simulated, this will be
|
||||
// around 1,500,000 metronome tick invocations.
|
||||
TEST_P(FrameBufferProxyTest, NextFrameWithOldTimestamp) {
|
||||
TEST_P(VideoStreamBufferControllerTest, NextFrameWithOldTimestamp) {
|
||||
// Test inserting 31 frames and pause the stream for a long time before
|
||||
// frame 32.
|
||||
StartNextDecodeForceKeyframe();
|
||||
@ -692,22 +703,22 @@ TEST_P(FrameBufferProxyTest, NextFrameWithOldTimestamp) {
|
||||
// First keyframe. The receive time must be explicitly set in this test since
|
||||
// the RTP derived time used in all tests does not work when the long pause
|
||||
// happens later in the test.
|
||||
proxy_->InsertFrame(test::FakeFrameBuilder()
|
||||
.Id(0)
|
||||
.Time(kBaseRtp)
|
||||
.ReceivedTime(clock_->CurrentTime())
|
||||
.AsLast()
|
||||
.Build());
|
||||
buffer_->InsertFrame(test::FakeFrameBuilder()
|
||||
.Id(0)
|
||||
.Time(kBaseRtp)
|
||||
.ReceivedTime(clock_->CurrentTime())
|
||||
.AsLast()
|
||||
.Build());
|
||||
EXPECT_THAT(WaitForFrameOrTimeout(kFps30Delay), Frame(test::WithId(0)));
|
||||
|
||||
// 1 more frame to warmup VCMTiming for 30fps.
|
||||
StartNextDecode();
|
||||
proxy_->InsertFrame(test::FakeFrameBuilder()
|
||||
.Id(1)
|
||||
.Time(kBaseRtp + kFps30Rtp)
|
||||
.ReceivedTime(clock_->CurrentTime())
|
||||
.AsLast()
|
||||
.Build());
|
||||
buffer_->InsertFrame(test::FakeFrameBuilder()
|
||||
.Id(1)
|
||||
.Time(kBaseRtp + kFps30Rtp)
|
||||
.ReceivedTime(clock_->CurrentTime())
|
||||
.AsLast()
|
||||
.Build());
|
||||
EXPECT_THAT(WaitForFrameOrTimeout(kFps30Delay), Frame(test::WithId(1)));
|
||||
|
||||
// Pause the stream for such a long time it incurs an RTP timestamp rollover
|
||||
@ -727,26 +738,27 @@ TEST_P(FrameBufferProxyTest, NextFrameWithOldTimestamp) {
|
||||
EXPECT_THAT(WaitForFrameOrTimeout(kMaxWaitForFrame), Eq(absl::nullopt));
|
||||
time_controller_.AdvanceTime(kRolloverDelay - kMaxWaitForFrame);
|
||||
StartNextDecode();
|
||||
proxy_->InsertFrame(test::FakeFrameBuilder()
|
||||
.Id(2)
|
||||
.Time(kRolloverRtp)
|
||||
.ReceivedTime(clock_->CurrentTime())
|
||||
.AsLast()
|
||||
.Build());
|
||||
buffer_->InsertFrame(test::FakeFrameBuilder()
|
||||
.Id(2)
|
||||
.Time(kRolloverRtp)
|
||||
.ReceivedTime(clock_->CurrentTime())
|
||||
.AsLast()
|
||||
.Build());
|
||||
// FrameBuffer2 drops the frame, while FrameBuffer3 will continue the stream.
|
||||
EXPECT_THAT(WaitForFrameOrTimeout(kFps30Delay), Frame(test::WithId(2)));
|
||||
}
|
||||
|
||||
INSTANTIATE_TEST_SUITE_P(
|
||||
FrameBufferProxy,
|
||||
FrameBufferProxyTest,
|
||||
VideoStreamBufferController,
|
||||
VideoStreamBufferControllerTest,
|
||||
::testing::Values("WebRTC-FrameBuffer3/arm:FrameBuffer3/",
|
||||
"WebRTC-FrameBuffer3/arm:SyncDecoding/"));
|
||||
|
||||
class LowLatencyFrameBufferProxyTest : public ::testing::Test,
|
||||
public FrameBufferProxyFixture {};
|
||||
class LowLatencyVideoStreamBufferControllerTest
|
||||
: public ::testing::Test,
|
||||
public VideoStreamBufferControllerFixture {};
|
||||
|
||||
TEST_P(LowLatencyFrameBufferProxyTest,
|
||||
TEST_P(LowLatencyVideoStreamBufferControllerTest,
|
||||
FramesDecodedInstantlyWithLowLatencyRendering) {
|
||||
// Initial keyframe.
|
||||
StartNextDecodeForceKeyframe();
|
||||
@ -755,7 +767,7 @@ TEST_P(LowLatencyFrameBufferProxyTest,
|
||||
auto frame = test::FakeFrameBuilder().Id(0).Time(0).AsLast().Build();
|
||||
// Playout delay of 0 implies low-latency rendering.
|
||||
frame->SetPlayoutDelay({0, 10});
|
||||
proxy_->InsertFrame(std::move(frame));
|
||||
buffer_->InsertFrame(std::move(frame));
|
||||
EXPECT_THAT(WaitForFrameOrTimeout(TimeDelta::Zero()), Frame(test::WithId(0)));
|
||||
|
||||
// Delta frame would normally wait here, but should decode at the pacing rate
|
||||
@ -763,14 +775,14 @@ TEST_P(LowLatencyFrameBufferProxyTest,
|
||||
StartNextDecode();
|
||||
frame = test::FakeFrameBuilder().Id(1).Time(kFps30Rtp).AsLast().Build();
|
||||
frame->SetPlayoutDelay({0, 10});
|
||||
proxy_->InsertFrame(std::move(frame));
|
||||
buffer_->InsertFrame(std::move(frame));
|
||||
// Pacing is set to 16ms in the field trial so we should not decode yet.
|
||||
EXPECT_THAT(WaitForFrameOrTimeout(TimeDelta::Zero()), Eq(absl::nullopt));
|
||||
time_controller_.AdvanceTime(TimeDelta::Millis(16));
|
||||
EXPECT_THAT(WaitForFrameOrTimeout(TimeDelta::Zero()), Frame(test::WithId(1)));
|
||||
}
|
||||
|
||||
TEST_P(LowLatencyFrameBufferProxyTest, ZeroPlayoutDelayFullQueue) {
|
||||
TEST_P(LowLatencyVideoStreamBufferControllerTest, ZeroPlayoutDelayFullQueue) {
|
||||
// Initial keyframe.
|
||||
StartNextDecodeForceKeyframe();
|
||||
timing_.set_min_playout_delay(TimeDelta::Zero());
|
||||
@ -778,7 +790,7 @@ TEST_P(LowLatencyFrameBufferProxyTest, ZeroPlayoutDelayFullQueue) {
|
||||
auto frame = test::FakeFrameBuilder().Id(0).Time(0).AsLast().Build();
|
||||
// Playout delay of 0 implies low-latency rendering.
|
||||
frame->SetPlayoutDelay({0, 10});
|
||||
proxy_->InsertFrame(std::move(frame));
|
||||
buffer_->InsertFrame(std::move(frame));
|
||||
EXPECT_THAT(WaitForFrameOrTimeout(TimeDelta::Zero()), Frame(test::WithId(0)));
|
||||
|
||||
// Queue up 5 frames (configured max queue size for 0-playout delay pacing).
|
||||
@ -786,7 +798,7 @@ TEST_P(LowLatencyFrameBufferProxyTest, ZeroPlayoutDelayFullQueue) {
|
||||
frame =
|
||||
test::FakeFrameBuilder().Id(id).Time(kFps30Rtp * id).AsLast().Build();
|
||||
frame->SetPlayoutDelay({0, 10});
|
||||
proxy_->InsertFrame(std::move(frame));
|
||||
buffer_->InsertFrame(std::move(frame));
|
||||
}
|
||||
|
||||
// The queue is at its max size for zero playout delay pacing, so the pacing
|
||||
@ -795,7 +807,8 @@ TEST_P(LowLatencyFrameBufferProxyTest, ZeroPlayoutDelayFullQueue) {
|
||||
EXPECT_THAT(WaitForFrameOrTimeout(TimeDelta::Zero()), Frame(test::WithId(1)));
|
||||
}
|
||||
|
||||
TEST_P(LowLatencyFrameBufferProxyTest, MinMaxDelayZeroLowLatencyMode) {
|
||||
TEST_P(LowLatencyVideoStreamBufferControllerTest,
|
||||
MinMaxDelayZeroLowLatencyMode) {
|
||||
// Initial keyframe.
|
||||
StartNextDecodeForceKeyframe();
|
||||
timing_.set_min_playout_delay(TimeDelta::Zero());
|
||||
@ -803,7 +816,7 @@ TEST_P(LowLatencyFrameBufferProxyTest, MinMaxDelayZeroLowLatencyMode) {
|
||||
auto frame = test::FakeFrameBuilder().Id(0).Time(0).AsLast().Build();
|
||||
// Playout delay of 0 implies low-latency rendering.
|
||||
frame->SetPlayoutDelay({0, 0});
|
||||
proxy_->InsertFrame(std::move(frame));
|
||||
buffer_->InsertFrame(std::move(frame));
|
||||
EXPECT_THAT(WaitForFrameOrTimeout(TimeDelta::Zero()), Frame(test::WithId(0)));
|
||||
|
||||
// Delta frame would normally wait here, but should decode at the pacing rate
|
||||
@ -811,15 +824,15 @@ TEST_P(LowLatencyFrameBufferProxyTest, MinMaxDelayZeroLowLatencyMode) {
|
||||
StartNextDecode();
|
||||
frame = test::FakeFrameBuilder().Id(1).Time(kFps30Rtp).AsLast().Build();
|
||||
frame->SetPlayoutDelay({0, 0});
|
||||
proxy_->InsertFrame(std::move(frame));
|
||||
buffer_->InsertFrame(std::move(frame));
|
||||
// The min/max=0 version of low-latency rendering will result in a large
|
||||
// negative decode wait time, so the frame should be ready right away.
|
||||
EXPECT_THAT(WaitForFrameOrTimeout(TimeDelta::Zero()), Frame(test::WithId(1)));
|
||||
}
|
||||
|
||||
INSTANTIATE_TEST_SUITE_P(
|
||||
FrameBufferProxy,
|
||||
LowLatencyFrameBufferProxyTest,
|
||||
VideoStreamBufferController,
|
||||
LowLatencyVideoStreamBufferControllerTest,
|
||||
::testing::Values(
|
||||
"WebRTC-FrameBuffer3/arm:FrameBuffer3/"
|
||||
"WebRTC-ZeroPlayoutDelay/min_pacing:16ms,max_decode_queue_size:5/",
|
Reference in New Issue
Block a user