Switch encoder on init failure

Currently if encoder initialization fails WebRTC doesn't send any video.
This CL adds functionality that changes encoder type in such case and
restores the video. If encoder selector is available we switch to
encoder it recommends. Otherwise, VP8 is used as the default fallback
encoder.

Bug: webrtc:13572
Change-Id: Ifcdf707a575711f5ff81f9451caf30140c9171dc
Reviewed-on: https://webrtc-review.googlesource.com/c/src/+/246960
Reviewed-by: Philip Eliasson <philipel@webrtc.org>
Reviewed-by: Erik Språng <sprang@webrtc.org>
Reviewed-by: Rasmus Brandt <brandtr@webrtc.org>
Commit-Queue: Sergey Silkin <ssilkin@webrtc.org>
Cr-Commit-Position: refs/heads/main@{#35761}
This commit is contained in:
Sergey Silkin 2022-01-21 11:35:04 +01:00 committed by WebRTC LUCI CQ
parent 6d2fe89b7e
commit e1cd3ad4f5
7 changed files with 213 additions and 196 deletions

View File

@ -23,19 +23,13 @@ class EncoderSwitchRequestCallback {
public:
virtual ~EncoderSwitchRequestCallback() {}
struct Config {
std::string codec_name;
absl::optional<std::string> param;
absl::optional<std::string> value;
};
// Requests that encoder fallback is performed.
// Requests switch to next negotiated encoder.
virtual void RequestEncoderFallback() = 0;
// Requests that a switch to a specific encoder is performed.
virtual void RequestEncoderSwitch(const Config& conf) = 0;
virtual void RequestEncoderSwitch(const SdpVideoFormat& format) = 0;
// Requests switch to a specific encoder. If the encoder is not available and
// `allow_default_fallback` is `true` the default fallback is invoked.
virtual void RequestEncoderSwitch(const SdpVideoFormat& format,
bool allow_default_fallback) = 0;
};
struct VideoStreamEncoderSettings {

View File

@ -899,56 +899,13 @@ void WebRtcVideoChannel::RequestEncoderFallback() {
}
void WebRtcVideoChannel::RequestEncoderSwitch(
const EncoderSwitchRequestCallback::Config& conf) {
const webrtc::SdpVideoFormat& format,
bool allow_default_fallback) {
if (!worker_thread_->IsCurrent()) {
worker_thread_->PostTask(ToQueuedTask(
task_safety_, [this, conf] { RequestEncoderSwitch(conf); }));
return;
}
RTC_DCHECK_RUN_ON(&thread_checker_);
if (!allow_codec_switching_) {
RTC_LOG(LS_INFO) << "Encoder switch requested but codec switching has"
" not been enabled yet.";
requested_encoder_switch_ = conf;
return;
}
for (const VideoCodecSettings& codec_setting : negotiated_codecs_) {
if (codec_setting.codec.name == conf.codec_name) {
if (conf.param) {
auto it = codec_setting.codec.params.find(*conf.param);
if (it == codec_setting.codec.params.end())
continue;
if (conf.value && it->second != *conf.value)
continue;
}
if (send_codec_ == codec_setting) {
// Already using this codec, no switch required.
return;
}
ChangedSendParameters params;
params.send_codec = codec_setting;
ApplyChangedParams(params);
return;
}
}
RTC_LOG(LS_WARNING) << "Requested encoder with codec_name:" << conf.codec_name
<< ", param:" << conf.param.value_or("none")
<< " and value:" << conf.value.value_or("none")
<< "not found. No switch performed.";
}
void WebRtcVideoChannel::RequestEncoderSwitch(
const webrtc::SdpVideoFormat& format) {
if (!worker_thread_->IsCurrent()) {
worker_thread_->PostTask(ToQueuedTask(
task_safety_, [this, format] { RequestEncoderSwitch(format); }));
worker_thread_->PostTask(
ToQueuedTask(task_safety_, [this, format, allow_default_fallback] {
RequestEncoderSwitch(format, allow_default_fallback);
}));
return;
}
@ -974,8 +931,13 @@ void WebRtcVideoChannel::RequestEncoderSwitch(
}
}
RTC_LOG(LS_WARNING) << "Encoder switch failed: SdpVideoFormat "
<< format.ToString() << " not negotiated.";
RTC_LOG(LS_WARNING) << "Failed to switch encoder to: " << format.ToString()
<< ". Is default fallback allowed: "
<< allow_default_fallback;
if (allow_default_fallback) {
RequestEncoderFallback();
}
}
bool WebRtcVideoChannel::ApplyChangedParams(
@ -1956,11 +1918,6 @@ void WebRtcVideoChannel::SetVideoCodecSwitchingEnabled(bool enabled) {
allow_codec_switching_ = enabled;
if (allow_codec_switching_) {
RTC_LOG(LS_INFO) << "Encoder switching enabled.";
if (requested_encoder_switch_) {
RTC_LOG(LS_INFO) << "Executing cached video encoder switch request.";
RequestEncoderSwitch(*requested_encoder_switch_);
requested_encoder_switch_.reset();
}
}
}

View File

@ -224,11 +224,8 @@ class WebRtcVideoChannel : public VideoMediaChannel,
// Implements webrtc::EncoderSwitchRequestCallback.
void RequestEncoderFallback() override;
// TODO(bugs.webrtc.org/11341) : Remove this version of RequestEncoderSwitch.
void RequestEncoderSwitch(
const EncoderSwitchRequestCallback::Config& conf) override;
void RequestEncoderSwitch(const webrtc::SdpVideoFormat& format) override;
void RequestEncoderSwitch(const webrtc::SdpVideoFormat& format,
bool allow_default_fallback) override;
void SetRecordableEncodedFrameCallback(
uint32_t ssrc,
@ -638,9 +635,10 @@ class WebRtcVideoChannel : public VideoMediaChannel,
std::unique_ptr<UnhandledPacketsBuffer> unknown_ssrc_packet_buffer_
RTC_GUARDED_BY(thread_checker_);
// TODO(bugs.webrtc.org/11341): Remove this and relevant PC API. Presence
// of multiple negotiated codecs allows generic encoder fallback on failures.
// Presence of EncoderSelector allows switching to specific encoders.
bool allow_codec_switching_ = false;
absl::optional<EncoderSwitchRequestCallback::Config>
requested_encoder_switch_;
};
class EncoderStreamFactory

View File

@ -2405,116 +2405,59 @@ TEST_F(WebRtcVideoChannelBaseTest, RequestEncoderFallback) {
EXPECT_EQ("VP8", codec.name);
}
TEST_F(WebRtcVideoChannelBaseTest, RequestEncoderSwitchWithConfig) {
const std::string kParam = "the-param";
const std::string kPing = "ping";
const std::string kPong = "pong";
TEST_F(WebRtcVideoChannelBaseTest, RequestEncoderSwitchDefaultFallback) {
cricket::VideoSendParameters parameters;
VideoCodec vp9 = GetEngineCodec("VP9");
vp9.params[kParam] = kPong;
parameters.codecs.push_back(vp9);
VideoCodec vp8 = GetEngineCodec("VP8");
vp8.params[kParam] = kPing;
parameters.codecs.push_back(vp8);
parameters.codecs.push_back(GetEngineCodec("VP9"));
parameters.codecs.push_back(GetEngineCodec("VP8"));
EXPECT_TRUE(channel_->SetSendParameters(parameters));
channel_->SetVideoCodecSwitchingEnabled(true);
VideoCodec codec;
ASSERT_TRUE(channel_->GetSendCodec(&codec));
EXPECT_THAT(codec.name, Eq("VP9"));
EXPECT_EQ("VP9", codec.name);
// RequestEncoderSwitch will post a task to the worker thread (which is also
// the current thread), hence the ProcessMessages call.
webrtc::EncoderSwitchRequestCallback::Config conf1{"VP8", kParam, kPing};
channel_->RequestEncoderSwitch(conf1);
channel_->RequestEncoderSwitch(webrtc::SdpVideoFormat("UnavailableCodec"),
/*allow_default_fallback=*/true);
rtc::Thread::Current()->ProcessMessages(30);
ASSERT_TRUE(channel_->GetSendCodec(&codec));
EXPECT_THAT(codec.name, Eq("VP8"));
EXPECT_THAT(codec.params, Contains(Pair(kParam, kPing)));
webrtc::EncoderSwitchRequestCallback::Config conf2{"VP9", kParam, kPong};
channel_->RequestEncoderSwitch(conf2);
rtc::Thread::Current()->ProcessMessages(30);
// Requested encoder is not available. Default fallback is allowed. Switch to
// the next negotiated codec, VP8.
ASSERT_TRUE(channel_->GetSendCodec(&codec));
EXPECT_THAT(codec.name, Eq("VP9"));
EXPECT_THAT(codec.params, Contains(Pair(kParam, kPong)));
EXPECT_EQ("VP8", codec.name);
}
TEST_F(WebRtcVideoChannelBaseTest, RequestEncoderSwitchIncorrectParam) {
const std::string kParam = "the-param";
const std::string kPing = "ping";
const std::string kPong = "pong";
TEST_F(WebRtcVideoChannelBaseTest, RequestEncoderSwitchStrictPreference) {
VideoCodec vp9 = GetEngineCodec("VP9");
vp9.params["profile-id"] = "0";
cricket::VideoSendParameters parameters;
VideoCodec vp9 = GetEngineCodec("VP9");
vp9.params[kParam] = kPong;
parameters.codecs.push_back(GetEngineCodec("VP8"));
parameters.codecs.push_back(vp9);
VideoCodec vp8 = GetEngineCodec("VP8");
vp8.params[kParam] = kPing;
parameters.codecs.push_back(vp8);
EXPECT_TRUE(channel_->SetSendParameters(parameters));
channel_->SetVideoCodecSwitchingEnabled(true);
VideoCodec codec;
ASSERT_TRUE(channel_->GetSendCodec(&codec));
EXPECT_THAT(codec.name, Eq("VP9"));
// RequestEncoderSwitch will post a task to the worker thread (which is also
// the current thread), hence the ProcessMessages call.
webrtc::EncoderSwitchRequestCallback::Config conf1{"VP8", kParam, kPing};
channel_->RequestEncoderSwitch(conf1);
rtc::Thread::Current()->ProcessMessages(30);
ASSERT_TRUE(channel_->GetSendCodec(&codec));
EXPECT_THAT(codec.name, Eq("VP8"));
EXPECT_THAT(codec.params, Contains(Pair(kParam, kPing)));
// Incorrect conf2.value, expect no codec switch.
webrtc::EncoderSwitchRequestCallback::Config conf2{"VP9", kParam, kPing};
channel_->RequestEncoderSwitch(conf2);
rtc::Thread::Current()->ProcessMessages(30);
ASSERT_TRUE(channel_->GetSendCodec(&codec));
EXPECT_THAT(codec.name, Eq("VP8"));
EXPECT_THAT(codec.params, Contains(Pair(kParam, kPing)));
}
TEST_F(WebRtcVideoChannelBaseTest,
RequestEncoderSwitchWithConfigBeforeEnabling) {
const std::string kParam = "the-param";
const std::string kPing = "ping";
const std::string kPong = "pong";
cricket::VideoSendParameters parameters;
VideoCodec vp9 = GetEngineCodec("VP9");
vp9.params[kParam] = kPong;
parameters.codecs.push_back(vp9);
VideoCodec vp8 = GetEngineCodec("VP8");
vp8.params[kParam] = kPing;
parameters.codecs.push_back(vp8);
EXPECT_TRUE(channel_->SetSendParameters(parameters));
VideoCodec codec;
ASSERT_TRUE(channel_->GetSendCodec(&codec));
EXPECT_THAT(codec.name, Eq("VP9"));
EXPECT_EQ("VP8", codec.name);
webrtc::EncoderSwitchRequestCallback::Config conf{"VP8", kParam, kPing};
channel_->RequestEncoderSwitch(conf);
// Enable codec switching after it has been requested.
channel_->SetVideoCodecSwitchingEnabled(true);
// RequestEncoderSwitch will post a task to the worker thread (which is also
// the current thread), hence the ProcessMessages call.
channel_->RequestEncoderSwitch(
webrtc::SdpVideoFormat("VP9", {{"profile-id", "1"}}),
/*allow_default_fallback=*/false);
rtc::Thread::Current()->ProcessMessages(30);
// VP9 profile_id=1 is not available. Default fallback is not allowed. Switch
// is not performed.
ASSERT_TRUE(channel_->GetSendCodec(&codec));
EXPECT_THAT(codec.name, Eq("VP8"));
EXPECT_THAT(codec.params, Contains(Pair(kParam, kPing)));
EXPECT_EQ("VP8", codec.name);
channel_->RequestEncoderSwitch(
webrtc::SdpVideoFormat("VP9", {{"profile-id", "0"}}),
/*allow_default_fallback=*/false);
rtc::Thread::Current()->ProcessMessages(30);
// VP9 profile_id=0 is available. Switch encoder.
ASSERT_TRUE(channel_->GetSendCodec(&codec));
EXPECT_EQ("VP9", codec.name);
}
TEST_F(WebRtcVideoChannelBaseTest, SendCodecIsMovedToFrontInRtpParameters) {

View File

@ -28,6 +28,7 @@
#include "api/video/video_bitrate_allocator_factory.h"
#include "api/video/video_codec_constants.h"
#include "api/video/video_layers_allocation.h"
#include "api/video_codecs/sdp_video_format.h"
#include "api/video_codecs/video_encoder.h"
#include "call/adaptation/resource_adaptation_processor.h"
#include "call/adaptation/video_stream_adapter.h"
@ -65,6 +66,11 @@ const int64_t kPendingFrameTimeoutMs = 1000;
constexpr char kFrameDropperFieldTrial[] = "WebRTC-FrameDropper";
// TODO(bugs.webrtc.org/13572): Remove this kill switch after deploying the
// feature.
constexpr char kSwitchEncoderOnInitializationFailuresFieldTrial[] =
"WebRTC-SwitchEncoderOnInitializationFailures";
const size_t kDefaultPayloadSize = 1440;
const int64_t kParameterUpdateIntervalMs = 1000;
@ -660,6 +666,8 @@ VideoStreamEncoder::VideoStreamEncoder(
!field_trial::IsEnabled("WebRTC-DefaultBitrateLimitsKillSwitch")),
qp_parsing_allowed_(
!field_trial::IsEnabled("WebRTC-QpParsingKillSwitch")),
switch_encoder_on_init_failures_(!field_trial::IsDisabled(
kSwitchEncoderOnInitializationFailuresFieldTrial)),
encoder_queue_(std::move(encoder_queue)) {
TRACE_EVENT0("webrtc", "VideoStreamEncoder::VideoStreamEncoder");
RTC_DCHECK_RUN_ON(worker_queue_);
@ -1133,7 +1141,6 @@ void VideoStreamEncoder::ReconfigureEncoder() {
// Encoder creation block is split in two since EncoderInfo needed to start
// CPU adaptation with the correct settings should be polled after
// encoder_->InitEncode().
bool success = true;
if (encoder_reset_required) {
ReleaseEncoder();
const size_t max_data_payload_length = max_data_payload_length_ > 0
@ -1148,7 +1155,6 @@ void VideoStreamEncoder::ReconfigureEncoder() {
<< CodecTypeToPayloadString(send_codec_.codecType)
<< " (" << send_codec_.codecType << ")";
ReleaseEncoder();
success = false;
} else {
encoder_initialized_ = true;
encoder_->RegisterEncodeCompleteCallback(this);
@ -1167,7 +1173,7 @@ void VideoStreamEncoder::ReconfigureEncoder() {
// Inform dependents of updated encoder settings.
OnEncoderSettingsChanged();
if (success) {
if (encoder_initialized_) {
RTC_LOG(LS_VERBOSE) << " max bitrate " << codec.maxBitrate
<< " start bitrate " << codec.startBitrate
<< " max frame rate " << codec.maxFramerate
@ -1253,6 +1259,49 @@ void VideoStreamEncoder::ReconfigureEncoder() {
stream_resource_manager_.ConfigureQualityScaler(info);
stream_resource_manager_.ConfigureBandwidthQualityScaler(info);
if (!encoder_initialized_) {
RTC_LOG(LS_WARNING) << "Failed to initialize "
<< CodecTypeToPayloadString(codec.codecType)
<< " encoder."
<< "switch_encoder_on_init_failures: "
<< switch_encoder_on_init_failures_;
if (switch_encoder_on_init_failures_) {
RequestEncoderSwitch();
}
}
}
void VideoStreamEncoder::RequestEncoderSwitch() {
bool is_encoder_switching_supported =
settings_.encoder_switch_request_callback != nullptr;
bool is_encoder_selector_available = encoder_selector_ != nullptr;
RTC_LOG(LS_INFO) << "RequestEncoderSwitch."
<< " is_encoder_selector_available: "
<< is_encoder_selector_available
<< " is_encoder_switching_supported: "
<< is_encoder_switching_supported;
if (!is_encoder_switching_supported) {
return;
}
// If encoder selector is available, switch to the encoder it prefers.
// Otherwise try switching to VP8 (default WebRTC codec).
absl::optional<SdpVideoFormat> preferred_fallback_encoder;
if (is_encoder_selector_available) {
preferred_fallback_encoder = encoder_selector_->OnEncoderBroken();
}
if (!preferred_fallback_encoder) {
preferred_fallback_encoder =
SdpVideoFormat(CodecTypeToPayloadString(kVideoCodecVP8));
}
settings_.encoder_switch_request_callback->RequestEncoderSwitch(
*preferred_fallback_encoder, /*allow_default_fallback=*/true);
}
void VideoStreamEncoder::OnEncoderSettingsChanged() {
@ -1651,7 +1700,7 @@ void VideoStreamEncoder::EncodeVideoFrame(const VideoFrame& video_frame,
// If the encoder fail we can't continue to encode frames. When this happens
// the WebrtcVideoSender is notified and the whole VideoSendStream is
// recreated.
if (encoder_failed_)
if (encoder_failed_ || !encoder_initialized_)
return;
// It's possible that EncodeVideoFrame can be called after we've completed
@ -1789,21 +1838,7 @@ void VideoStreamEncoder::EncodeVideoFrame(const VideoFrame& video_frame,
if (encode_status == WEBRTC_VIDEO_CODEC_ENCODER_FAILURE) {
RTC_LOG(LS_ERROR) << "Encoder failed, failing encoder format: "
<< encoder_config_.video_format.ToString();
if (settings_.encoder_switch_request_callback) {
if (encoder_selector_) {
if (auto encoder = encoder_selector_->OnEncoderBroken()) {
settings_.encoder_switch_request_callback->RequestEncoderSwitch(
*encoder);
}
} else {
encoder_failed_ = true;
settings_.encoder_switch_request_callback->RequestEncoderFallback();
}
} else {
RTC_LOG(LS_ERROR)
<< "Encoder failed but no encoder fallback callback is registered";
}
RequestEncoderSwitch();
} else {
RTC_LOG(LS_ERROR) << "Failed to encode frame. Error code: "
<< encode_status;
@ -2059,7 +2094,8 @@ void VideoStreamEncoder::OnBitrateUpdated(DataRate target_bitrate,
if (!video_is_suspended && settings_.encoder_switch_request_callback &&
encoder_selector_) {
if (auto encoder = encoder_selector_->OnAvailableBitrate(link_allocation)) {
settings_.encoder_switch_request_callback->RequestEncoderSwitch(*encoder);
settings_.encoder_switch_request_callback->RequestEncoderSwitch(
*encoder, /*allow_default_fallback=*/false);
}
}

View File

@ -245,6 +245,8 @@ class VideoStreamEncoder : public VideoStreamEncoderInterface,
int64_t time_when_posted_in_ms)
RTC_RUN_ON(&encoder_queue_);
void RequestEncoderSwitch() RTC_RUN_ON(&encoder_queue_);
TaskQueueBase* const worker_queue_;
const uint32_t number_of_cores_;
@ -429,6 +431,9 @@ class VideoStreamEncoder : public VideoStreamEncoderInterface,
QpParser qp_parser_;
const bool qp_parsing_allowed_;
// Enables encoder switching on initialization failures.
bool switch_encoder_on_init_failures_;
// Public methods are proxied to the task queues. The queues must be destroyed
// first to make sure no tasks run that use other members.
rtc::TaskQueue encoder_queue_;

View File

@ -7465,10 +7465,10 @@ TEST_F(VideoStreamEncoderTest, EncoderRatesPropagatedOnReconfigure) {
struct MockEncoderSwitchRequestCallback : public EncoderSwitchRequestCallback {
MOCK_METHOD(void, RequestEncoderFallback, (), (override));
MOCK_METHOD(void, RequestEncoderSwitch, (const Config& conf), (override));
MOCK_METHOD(void,
RequestEncoderSwitch,
(const webrtc::SdpVideoFormat& format),
(const webrtc::SdpVideoFormat& format,
bool allow_default_fallback),
(override));
};
@ -7482,14 +7482,14 @@ TEST_F(VideoStreamEncoderTest, EncoderSelectorCurrentEncoderIsSignaled) {
// Reset encoder for new configuration to take effect.
ConfigureEncoder(video_encoder_config_.Copy());
EXPECT_CALL(encoder_selector, OnCurrentEncoder(_));
EXPECT_CALL(encoder_selector, OnCurrentEncoder);
video_source_.IncomingCapturedFrame(
CreateFrame(kDontCare, kDontCare, kDontCare));
AdvanceTime(TimeDelta::Zero());
video_stream_encoder_->Stop();
// The encoders produces by the VideoEncoderProxyFactory have a pointer back
// The encoders produced by the VideoEncoderProxyFactory have a pointer back
// to it's factory, so in order for the encoder instance in the
// `video_stream_encoder_` to be destroyed before the `encoder_factory` we
// reset the `video_stream_encoder_` here.
@ -7510,11 +7510,11 @@ TEST_F(VideoStreamEncoderTest, EncoderSelectorBitrateSwitch) {
// Reset encoder for new configuration to take effect.
ConfigureEncoder(video_encoder_config_.Copy());
ON_CALL(encoder_selector, OnAvailableBitrate(_))
ON_CALL(encoder_selector, OnAvailableBitrate)
.WillByDefault(Return(SdpVideoFormat("AV1")));
EXPECT_CALL(switch_callback,
RequestEncoderSwitch(Matcher<const SdpVideoFormat&>(
Field(&SdpVideoFormat::name, "AV1"))));
RequestEncoderSwitch(Field(&SdpVideoFormat::name, "AV1"),
/*allow_default_fallback=*/false));
video_stream_encoder_->OnBitrateUpdatedAndWaitForManagedResources(
/*target_bitrate=*/DataRate::KilobitsPerSec(50),
@ -7556,18 +7556,16 @@ TEST_F(VideoStreamEncoderTest, EncoderSelectorBrokenEncoderSwitch) {
/*round_trip_time_ms=*/0,
/*cwnd_reduce_ratio=*/0);
ON_CALL(video_encoder, Encode(_, _))
ON_CALL(video_encoder, Encode)
.WillByDefault(Return(WEBRTC_VIDEO_CODEC_ENCODER_FAILURE));
ON_CALL(encoder_selector, OnEncoderBroken())
ON_CALL(encoder_selector, OnEncoderBroken)
.WillByDefault(Return(SdpVideoFormat("AV2")));
rtc::Event encode_attempted;
EXPECT_CALL(switch_callback,
RequestEncoderSwitch(Matcher<const SdpVideoFormat&>(_)))
.WillOnce([&encode_attempted](const SdpVideoFormat& format) {
EXPECT_EQ(format.name, "AV2");
encode_attempted.Set();
});
RequestEncoderSwitch(Field(&SdpVideoFormat::name, "AV2"),
/*allow_default_fallback=*/true))
.WillOnce([&encode_attempted]() { encode_attempted.Set(); });
video_source_.IncomingCapturedFrame(CreateFrame(1, kDontCare, kDontCare));
encode_attempted.Wait(3000);
@ -7576,7 +7574,93 @@ TEST_F(VideoStreamEncoderTest, EncoderSelectorBrokenEncoderSwitch) {
video_stream_encoder_->Stop();
// The encoders produces by the VideoEncoderProxyFactory have a pointer back
// The encoders produced by the VideoEncoderProxyFactory have a pointer back
// to it's factory, so in order for the encoder instance in the
// `video_stream_encoder_` to be destroyed before the `encoder_factory` we
// reset the `video_stream_encoder_` here.
video_stream_encoder_.reset();
}
TEST_F(VideoStreamEncoderTest, SwitchEncoderOnInitFailureWithEncoderSelector) {
NiceMock<MockVideoEncoder> video_encoder;
NiceMock<MockEncoderSelector> encoder_selector;
StrictMock<MockEncoderSwitchRequestCallback> switch_callback;
video_send_config_.encoder_settings.encoder_switch_request_callback =
&switch_callback;
auto encoder_factory = std::make_unique<test::VideoEncoderProxyFactory>(
&video_encoder, &encoder_selector);
video_send_config_.encoder_settings.encoder_factory = encoder_factory.get();
// Reset encoder for new configuration to take effect.
ConfigureEncoder(video_encoder_config_.Copy());
video_stream_encoder_->OnBitrateUpdatedAndWaitForManagedResources(
kTargetBitrate, kTargetBitrate, kTargetBitrate, /*fraction_lost=*/0,
/*round_trip_time_ms=*/0,
/*cwnd_reduce_ratio=*/0);
ASSERT_EQ(0, sink_.number_of_reconfigurations());
ON_CALL(video_encoder, InitEncode(_, _))
.WillByDefault(Return(WEBRTC_VIDEO_CODEC_ENCODER_FAILURE));
ON_CALL(encoder_selector, OnEncoderBroken)
.WillByDefault(Return(SdpVideoFormat("AV2")));
rtc::Event encode_attempted;
EXPECT_CALL(switch_callback,
RequestEncoderSwitch(Field(&SdpVideoFormat::name, "AV2"),
/*allow_default_fallback=*/true))
.WillOnce([&encode_attempted]() { encode_attempted.Set(); });
video_source_.IncomingCapturedFrame(CreateFrame(1, nullptr));
encode_attempted.Wait(3000);
AdvanceTime(TimeDelta::Zero());
video_stream_encoder_->Stop();
// The encoders produced by the VideoEncoderProxyFactory have a pointer back
// to it's factory, so in order for the encoder instance in the
// `video_stream_encoder_` to be destroyed before the `encoder_factory` we
// reset the `video_stream_encoder_` here.
video_stream_encoder_.reset();
}
TEST_F(VideoStreamEncoderTest,
SwitchEncoderOnInitFailureWithoutEncoderSelector) {
NiceMock<MockVideoEncoder> video_encoder;
StrictMock<MockEncoderSwitchRequestCallback> switch_callback;
video_send_config_.encoder_settings.encoder_switch_request_callback =
&switch_callback;
auto encoder_factory = std::make_unique<test::VideoEncoderProxyFactory>(
&video_encoder, /*encoder_selector=*/nullptr);
video_send_config_.encoder_settings.encoder_factory = encoder_factory.get();
// Reset encoder for new configuration to take effect.
ConfigureEncoder(video_encoder_config_.Copy());
video_stream_encoder_->OnBitrateUpdatedAndWaitForManagedResources(
kTargetBitrate, kTargetBitrate, kTargetBitrate, /*fraction_lost=*/0,
/*round_trip_time_ms=*/0,
/*cwnd_reduce_ratio=*/0);
ASSERT_EQ(0, sink_.number_of_reconfigurations());
ON_CALL(video_encoder, InitEncode(_, _))
.WillByDefault(Return(WEBRTC_VIDEO_CODEC_ENCODER_FAILURE));
rtc::Event encode_attempted;
EXPECT_CALL(switch_callback,
RequestEncoderSwitch(Field(&SdpVideoFormat::name, "VP8"),
/*allow_default_fallback=*/true))
.WillOnce([&encode_attempted]() { encode_attempted.Set(); });
video_source_.IncomingCapturedFrame(CreateFrame(1, nullptr));
encode_attempted.Wait(3000);
AdvanceTime(TimeDelta::Zero());
video_stream_encoder_->Stop();
// The encoders produced by the VideoEncoderProxyFactory have a pointer back
// to it's factory, so in order for the encoder instance in the
// `video_stream_encoder_` to be destroyed before the `encoder_factory` we
// reset the `video_stream_encoder_` here.