Do not use PostDelayedTask in video codec tester's pacer

PostDelayedTask doesn't guarantee task execution order. For example,
if you post two tasks, A and B, back-to-back using the same delay
there is no guarantee that A will be executed before B.

Re-implemented pacing using sleep(). Changed pacer to compute task
scheduled time instead of delay. Sleep time is calculated right before
task start. This provides better accuracy by accounting for any delays
that may happen after pacing time is computed and before task queue is
ready to run the task.

It is tricky to implement pacer tests using simulated clocks. The test
use system time which make them flacky on low performance bots. Keep
the test disabled by default.

Bug: b/261160916, webrtc:14852
Change-Id: I88e1a2001e6d33cf3bb7fe16730ec28abf90acc8
Reviewed-on: https://webrtc-review.googlesource.com/c/src/+/291804
Reviewed-by: Rasmus Brandt <brandtr@webrtc.org>
Commit-Queue: Sergey Silkin <ssilkin@webrtc.org>
Cr-Commit-Position: refs/heads/main@{#39302}
This commit is contained in:
Sergey Silkin 2023-02-03 16:23:41 +01:00 committed by WebRTC LUCI CQ
parent 5f246cb2bf
commit 5dd493b3da
6 changed files with 129 additions and 213 deletions

View File

@ -56,11 +56,8 @@ Psnr CalcPsnr(const I420BufferInterface& ref_buffer,
} // namespace
VideoCodecAnalyzer::VideoCodecAnalyzer(
rtc::TaskQueue& task_queue,
ReferenceVideoSource* reference_video_source)
: task_queue_(task_queue),
reference_video_source_(reference_video_source),
num_frames_(0) {
: reference_video_source_(reference_video_source), num_frames_(0) {
sequence_checker_.Detach();
}

View File

@ -41,8 +41,8 @@ class VideoCodecAnalyzer {
Resolution resolution) = 0;
};
VideoCodecAnalyzer(rtc::TaskQueue& task_queue,
ReferenceVideoSource* reference_video_source = nullptr);
explicit VideoCodecAnalyzer(
ReferenceVideoSource* reference_video_source = nullptr);
void StartEncode(const VideoFrame& frame);
@ -55,7 +55,7 @@ class VideoCodecAnalyzer {
std::unique_ptr<VideoCodecStats> GetStats();
protected:
rtc::TaskQueue& task_queue_;
TaskQueueForTest task_queue_;
ReferenceVideoSource* const reference_video_source_;

View File

@ -59,8 +59,7 @@ EncodedImage CreateEncodedImage(uint32_t timestamp_rtp, int spatial_idx = 0) {
} // namespace
TEST(VideoCodecAnalyzerTest, StartEncode) {
TaskQueueForTest task_queue;
VideoCodecAnalyzer analyzer(task_queue);
VideoCodecAnalyzer analyzer;
analyzer.StartEncode(CreateVideoFrame(kTimestamp));
auto fs = analyzer.GetStats()->Slice();
@ -69,8 +68,7 @@ TEST(VideoCodecAnalyzerTest, StartEncode) {
}
TEST(VideoCodecAnalyzerTest, FinishEncode) {
TaskQueueForTest task_queue;
VideoCodecAnalyzer analyzer(task_queue);
VideoCodecAnalyzer analyzer;
analyzer.StartEncode(CreateVideoFrame(kTimestamp));
EncodedImage encoded_frame = CreateEncodedImage(kTimestamp, kSpatialIdx);
@ -82,8 +80,7 @@ TEST(VideoCodecAnalyzerTest, FinishEncode) {
}
TEST(VideoCodecAnalyzerTest, StartDecode) {
TaskQueueForTest task_queue;
VideoCodecAnalyzer analyzer(task_queue);
VideoCodecAnalyzer analyzer;
analyzer.StartDecode(CreateEncodedImage(kTimestamp, kSpatialIdx));
auto fs = analyzer.GetStats()->Slice();
@ -92,8 +89,7 @@ TEST(VideoCodecAnalyzerTest, StartDecode) {
}
TEST(VideoCodecAnalyzerTest, FinishDecode) {
TaskQueueForTest task_queue;
VideoCodecAnalyzer analyzer(task_queue);
VideoCodecAnalyzer analyzer;
analyzer.StartDecode(CreateEncodedImage(kTimestamp, kSpatialIdx));
VideoFrame decoded_frame = CreateVideoFrame(kTimestamp);
analyzer.FinishDecode(decoded_frame, kSpatialIdx);
@ -105,9 +101,8 @@ TEST(VideoCodecAnalyzerTest, FinishDecode) {
}
TEST(VideoCodecAnalyzerTest, ReferenceVideoSource) {
TaskQueueForTest task_queue;
MockReferenceVideoSource reference_video_source;
VideoCodecAnalyzer analyzer(task_queue, &reference_video_source);
VideoCodecAnalyzer analyzer(&reference_video_source);
analyzer.StartDecode(CreateEncodedImage(kTimestamp, kSpatialIdx));
EXPECT_CALL(reference_video_source, GetFrame)

View File

@ -71,37 +71,37 @@ class Pacer {
public:
explicit Pacer(PacingSettings settings)
: settings_(settings), delay_(TimeDelta::Zero()) {}
TimeDelta Delay(Timestamp beat) {
Timestamp Schedule(Timestamp timestamp) {
Timestamp now = Timestamp::Micros(rtc::TimeMicros());
if (settings_.mode == PacingMode::kNoPacing) {
return TimeDelta::Zero();
return now;
}
Timestamp now = Timestamp::Micros(rtc::TimeMicros());
if (prev_time_.has_value()) {
delay_ += PacingTime(beat);
delay_ -= (now - *prev_time_);
if (delay_.ns() < 0) {
delay_ = TimeDelta::Zero();
Timestamp scheduled = now;
if (prev_scheduled_) {
scheduled = *prev_scheduled_ + PacingTime(timestamp);
if (scheduled < now) {
scheduled = now;
}
}
prev_beat_ = beat;
prev_time_ = now;
return delay_;
prev_timestamp_ = timestamp;
prev_scheduled_ = scheduled;
return scheduled;
}
private:
TimeDelta PacingTime(Timestamp beat) {
TimeDelta PacingTime(Timestamp timestamp) {
if (settings_.mode == PacingMode::kRealTime) {
return beat - *prev_beat_;
return timestamp - *prev_timestamp_;
}
RTC_CHECK_EQ(PacingMode::kConstantRate, settings_.mode);
return 1 / settings_.constant_rate;
}
PacingSettings settings_;
absl::optional<Timestamp> prev_beat_;
absl::optional<Timestamp> prev_time_;
absl::optional<Timestamp> prev_timestamp_;
absl::optional<Timestamp> prev_scheduled_;
TimeDelta delay_;
};
@ -119,18 +119,20 @@ class LimitedTaskQueue {
// task starts.
static constexpr int kMaxTaskQueueSize = 3;
explicit LimitedTaskQueue(rtc::TaskQueue& task_queue)
: task_queue_(task_queue), queue_size_(0) {}
LimitedTaskQueue() : queue_size_(0) {}
void PostDelayedTask(absl::AnyInvocable<void() &&> task, TimeDelta delay) {
void PostScheduledTask(absl::AnyInvocable<void() &&> task, Timestamp start) {
++queue_size_;
task_queue_.PostDelayedTask(
[this, task = std::move(task)]() mutable {
std::move(task)();
--queue_size_;
task_executed_.Set();
},
delay);
task_queue_.PostTask([this, task = std::move(task), start]() mutable {
int wait_ms = static_cast<int>(start.ms() - rtc::TimeMillis());
if (wait_ms > 0) {
SleepMs(wait_ms);
}
std::move(task)();
--queue_size_;
task_executed_.Set();
});
task_executed_.Reset();
if (queue_size_ > kMaxTaskQueueSize) {
@ -146,7 +148,7 @@ class LimitedTaskQueue {
}
}
rtc::TaskQueue& task_queue_;
TaskQueueForTest task_queue_;
std::atomic_int queue_size_;
rtc::Event task_executed_;
};
@ -155,20 +157,18 @@ class TesterDecoder {
public:
TesterDecoder(Decoder* decoder,
VideoCodecAnalyzer* analyzer,
const DecoderSettings& settings,
rtc::TaskQueue& task_queue)
const DecoderSettings& settings)
: decoder_(decoder),
analyzer_(analyzer),
settings_(settings),
pacer_(settings.pacing),
task_queue_(task_queue) {
pacer_(settings.pacing) {
RTC_CHECK(analyzer_) << "Analyzer must be provided";
}
void Decode(const EncodedImage& frame) {
Timestamp timestamp = Timestamp::Micros((frame.Timestamp() / k90kHz).us());
task_queue_.PostDelayedTask(
task_queue_.PostScheduledTask(
[this, frame] {
analyzer_->StartDecode(frame);
decoder_->Decode(
@ -177,7 +177,7 @@ class TesterDecoder {
this->analyzer_->FinishDecode(decoded_frame, spatial_idx);
});
},
pacer_.Delay(timestamp));
pacer_.Schedule(timestamp));
}
void Flush() { task_queue_.WaitForPreviouslyPostedTasks(); }
@ -195,21 +195,19 @@ class TesterEncoder {
TesterEncoder(Encoder* encoder,
TesterDecoder* decoder,
VideoCodecAnalyzer* analyzer,
const EncoderSettings& settings,
rtc::TaskQueue& task_queue)
const EncoderSettings& settings)
: encoder_(encoder),
decoder_(decoder),
analyzer_(analyzer),
settings_(settings),
pacer_(settings.pacing),
task_queue_(task_queue) {
pacer_(settings.pacing) {
RTC_CHECK(analyzer_) << "Analyzer must be provided";
}
void Encode(const VideoFrame& frame) {
Timestamp timestamp = Timestamp::Micros((frame.timestamp() / k90kHz).us());
task_queue_.PostDelayedTask(
task_queue_.PostScheduledTask(
[this, frame] {
analyzer_->StartEncode(frame);
encoder_->Encode(frame, [this](const EncodedImage& encoded_frame) {
@ -219,7 +217,7 @@ class TesterEncoder {
}
});
},
pacer_.Delay(timestamp));
pacer_.Schedule(timestamp));
}
void Flush() { task_queue_.WaitForPreviouslyPostedTasks(); }
@ -235,29 +233,12 @@ class TesterEncoder {
} // namespace
VideoCodecTesterImpl::VideoCodecTesterImpl()
: VideoCodecTesterImpl(/*task_queue_factory=*/nullptr) {}
VideoCodecTesterImpl::VideoCodecTesterImpl(TaskQueueFactory* task_queue_factory)
: task_queue_factory_(task_queue_factory) {
if (task_queue_factory_ == nullptr) {
owned_task_queue_factory_ = CreateDefaultTaskQueueFactory();
task_queue_factory_ = owned_task_queue_factory_.get();
}
}
std::unique_ptr<VideoCodecStats> VideoCodecTesterImpl::RunDecodeTest(
CodedVideoSource* video_source,
Decoder* decoder,
const DecoderSettings& decoder_settings) {
rtc::TaskQueue analyser_task_queue(task_queue_factory_->CreateTaskQueue(
"Analyzer", TaskQueueFactory::Priority::NORMAL));
rtc::TaskQueue decoder_task_queue(task_queue_factory_->CreateTaskQueue(
"Decoder", TaskQueueFactory::Priority::NORMAL));
VideoCodecAnalyzer perf_analyzer(analyser_task_queue);
TesterDecoder tester_decoder(decoder, &perf_analyzer, decoder_settings,
decoder_task_queue);
VideoCodecAnalyzer perf_analyzer;
TesterDecoder tester_decoder(decoder, &perf_analyzer, decoder_settings);
while (auto frame = video_source->PullFrame()) {
tester_decoder.Decode(*frame);
@ -272,15 +253,10 @@ std::unique_ptr<VideoCodecStats> VideoCodecTesterImpl::RunEncodeTest(
RawVideoSource* video_source,
Encoder* encoder,
const EncoderSettings& encoder_settings) {
rtc::TaskQueue analyser_task_queue(task_queue_factory_->CreateTaskQueue(
"Analyzer", TaskQueueFactory::Priority::NORMAL));
rtc::TaskQueue encoder_task_queue(task_queue_factory_->CreateTaskQueue(
"Encoder", TaskQueueFactory::Priority::NORMAL));
SyncRawVideoSource sync_source(video_source);
VideoCodecAnalyzer perf_analyzer(analyser_task_queue);
VideoCodecAnalyzer perf_analyzer;
TesterEncoder tester_encoder(encoder, /*decoder=*/nullptr, &perf_analyzer,
encoder_settings, encoder_task_queue);
encoder_settings);
while (auto frame = sync_source.PullFrame()) {
tester_encoder.Encode(*frame);
@ -297,19 +273,11 @@ std::unique_ptr<VideoCodecStats> VideoCodecTesterImpl::RunEncodeDecodeTest(
Decoder* decoder,
const EncoderSettings& encoder_settings,
const DecoderSettings& decoder_settings) {
rtc::TaskQueue analyser_task_queue(task_queue_factory_->CreateTaskQueue(
"Analyzer", TaskQueueFactory::Priority::NORMAL));
rtc::TaskQueue decoder_task_queue(task_queue_factory_->CreateTaskQueue(
"Decoder", TaskQueueFactory::Priority::NORMAL));
rtc::TaskQueue encoder_task_queue(task_queue_factory_->CreateTaskQueue(
"Encoder", TaskQueueFactory::Priority::NORMAL));
SyncRawVideoSource sync_source(video_source);
VideoCodecAnalyzer perf_analyzer(analyser_task_queue, &sync_source);
TesterDecoder tester_decoder(decoder, &perf_analyzer, decoder_settings,
decoder_task_queue);
VideoCodecAnalyzer perf_analyzer(&sync_source);
TesterDecoder tester_decoder(decoder, &perf_analyzer, decoder_settings);
TesterEncoder tester_encoder(encoder, &tester_decoder, &perf_analyzer,
encoder_settings, encoder_task_queue);
encoder_settings);
while (auto frame = sync_source.PullFrame()) {
tester_encoder.Encode(*frame);

View File

@ -13,7 +13,6 @@
#include <memory>
#include "api/task_queue/task_queue_factory.h"
#include "api/test/video_codec_tester.h"
namespace webrtc {
@ -22,9 +21,6 @@ namespace test {
// A stateless implementation of `VideoCodecTester`. This class is thread safe.
class VideoCodecTesterImpl : public VideoCodecTester {
public:
VideoCodecTesterImpl();
explicit VideoCodecTesterImpl(TaskQueueFactory* task_queue_factory);
std::unique_ptr<VideoCodecStats> RunDecodeTest(
CodedVideoSource* video_source,
Decoder* decoder,
@ -41,10 +37,6 @@ class VideoCodecTesterImpl : public VideoCodecTester {
Decoder* decoder,
const EncoderSettings& encoder_settings,
const DecoderSettings& decoder_settings) override;
protected:
std::unique_ptr<TaskQueueFactory> owned_task_queue_factory_;
TaskQueueFactory* task_queue_factory_;
};
} // namespace test

View File

@ -49,6 +49,13 @@ using PacingMode = PacingSettings::PacingMode;
constexpr Frequency k90kHz = Frequency::Hertz(90000);
struct PacingTestParams {
PacingSettings pacing_settings;
Frequency framerate;
int num_frames;
std::vector<int> expected_delta_ms;
};
VideoFrame CreateVideoFrame(uint32_t timestamp_rtp) {
rtc::scoped_refptr<I420Buffer> buffer(I420Buffer::Create(2, 2));
return VideoFrame::Builder()
@ -65,16 +72,47 @@ EncodedImage CreateEncodedImage(uint32_t timestamp_rtp) {
class MockRawVideoSource : public RawVideoSource {
public:
MOCK_METHOD(absl::optional<VideoFrame>, PullFrame, (), (override));
MockRawVideoSource(int num_frames, Frequency framerate)
: num_frames_(num_frames), frame_num_(0), framerate_(framerate) {}
absl::optional<VideoFrame> PullFrame() override {
if (frame_num_ >= num_frames_) {
return absl::nullopt;
}
uint32_t timestamp_rtp = frame_num_ * k90kHz / framerate_;
++frame_num_;
return CreateVideoFrame(timestamp_rtp);
}
MOCK_METHOD(VideoFrame,
GetFrame,
(uint32_t timestamp_rtp, Resolution),
(override));
private:
int num_frames_;
int frame_num_;
Frequency framerate_;
};
class MockCodedVideoSource : public CodedVideoSource {
public:
MOCK_METHOD(absl::optional<EncodedImage>, PullFrame, (), (override));
MockCodedVideoSource(int num_frames, Frequency framerate)
: num_frames_(num_frames), frame_num_(0), framerate_(framerate) {}
absl::optional<EncodedImage> PullFrame() override {
if (frame_num_ >= num_frames_) {
return absl::nullopt;
}
uint32_t timestamp_rtp = frame_num_ * k90kHz / framerate_;
++frame_num_;
return CreateEncodedImage(timestamp_rtp);
}
private:
int num_frames_;
int frame_num_;
Frequency framerate_;
};
class MockDecoder : public Decoder {
@ -110,144 +148,70 @@ class MockTaskQueueFactory : public TaskQueueFactory {
} // namespace
class VideoCodecTesterImplPacingTest
: public ::testing::TestWithParam<std::tuple<PacingSettings,
std::vector<int>,
std::vector<int>,
std::vector<int>>> {
: public ::testing::TestWithParam<PacingTestParams> {
public:
VideoCodecTesterImplPacingTest()
: pacing_settings_(std::get<0>(GetParam())),
frame_timestamp_ms_(std::get<1>(GetParam())),
frame_capture_delay_ms_(std::get<2>(GetParam())),
expected_frame_start_ms_(std::get<3>(GetParam())),
num_frames_(frame_timestamp_ms_.size()),
task_queue_factory_(task_queue_) {}
void SetUp() override {
ON_CALL(task_queue_, PostTask)
.WillByDefault(Invoke(
[](absl::AnyInvocable<void() &&> task) { std::move(task)(); }));
ON_CALL(task_queue_, PostDelayedTask)
.WillByDefault(
Invoke([&](absl::AnyInvocable<void() &&> task, TimeDelta delay) {
clock_.AdvanceTime(delay);
std::move(task)();
}));
}
VideoCodecTesterImplPacingTest() : test_params_(GetParam()) {}
protected:
PacingSettings pacing_settings_;
std::vector<int> frame_timestamp_ms_;
std::vector<int> frame_capture_delay_ms_;
std::vector<int> expected_frame_start_ms_;
size_t num_frames_;
rtc::ScopedFakeClock clock_;
MockTaskQueueBase task_queue_;
MockTaskQueueFactory task_queue_factory_;
PacingTestParams test_params_;
};
TEST_P(VideoCodecTesterImplPacingTest, PaceEncode) {
MockRawVideoSource video_source;
size_t frame_num = 0;
EXPECT_CALL(video_source, PullFrame).WillRepeatedly(Invoke([&]() mutable {
if (frame_num >= num_frames_) {
return absl::optional<VideoFrame>();
}
clock_.AdvanceTime(TimeDelta::Millis(frame_capture_delay_ms_[frame_num]));
uint32_t timestamp_rtp = frame_timestamp_ms_[frame_num] * k90kHz.hertz() /
rtc::kNumMillisecsPerSec;
++frame_num;
return absl::optional<VideoFrame>(CreateVideoFrame(timestamp_rtp));
}));
MockRawVideoSource video_source(test_params_.num_frames,
test_params_.framerate);
MockEncoder encoder;
EncoderSettings encoder_settings;
encoder_settings.pacing = pacing_settings_;
encoder_settings.pacing = test_params_.pacing_settings;
VideoCodecTesterImpl tester(&task_queue_factory_);
VideoCodecTesterImpl tester;
auto fs =
tester.RunEncodeTest(&video_source, &encoder, encoder_settings)->Slice();
ASSERT_EQ(fs.size(), num_frames_);
ASSERT_EQ(static_cast<int>(fs.size()), test_params_.num_frames);
for (size_t i = 0; i < fs.size(); ++i) {
int encode_start_ms = (fs[i].encode_start - fs[0].encode_start).ms();
EXPECT_NEAR(encode_start_ms, expected_frame_start_ms_[i], 10);
for (size_t i = 1; i < fs.size(); ++i) {
int delta_ms = (fs[i].encode_start - fs[i - 1].encode_start).ms();
EXPECT_NEAR(delta_ms, test_params_.expected_delta_ms[i - 1], 10);
}
}
TEST_P(VideoCodecTesterImplPacingTest, PaceDecode) {
MockCodedVideoSource video_source;
size_t frame_num = 0;
EXPECT_CALL(video_source, PullFrame).WillRepeatedly(Invoke([&]() mutable {
if (frame_num >= num_frames_) {
return absl::optional<EncodedImage>();
}
clock_.AdvanceTime(TimeDelta::Millis(frame_capture_delay_ms_[frame_num]));
uint32_t timestamp_rtp = frame_timestamp_ms_[frame_num] * k90kHz.hertz() /
rtc::kNumMillisecsPerSec;
++frame_num;
return absl::optional<EncodedImage>(CreateEncodedImage(timestamp_rtp));
}));
MockCodedVideoSource video_source(test_params_.num_frames,
test_params_.framerate);
MockDecoder decoder;
DecoderSettings decoder_settings;
decoder_settings.pacing = pacing_settings_;
decoder_settings.pacing = test_params_.pacing_settings;
VideoCodecTesterImpl tester(&task_queue_factory_);
VideoCodecTesterImpl tester;
auto fs =
tester.RunDecodeTest(&video_source, &decoder, decoder_settings)->Slice();
ASSERT_EQ(fs.size(), num_frames_);
ASSERT_EQ(static_cast<int>(fs.size()), test_params_.num_frames);
for (size_t i = 0; i < fs.size(); ++i) {
int decode_start_ms = (fs[i].decode_start - fs[0].decode_start).ms();
EXPECT_NEAR(decode_start_ms, expected_frame_start_ms_[i], 10);
for (size_t i = 1; i < fs.size(); ++i) {
int delta_ms = (fs[i].decode_start - fs[i - 1].decode_start).ms();
EXPECT_NEAR(delta_ms, test_params_.expected_delta_ms[i - 1], 20);
}
}
INSTANTIATE_TEST_SUITE_P(
All,
DISABLED_All,
VideoCodecTesterImplPacingTest,
::testing::ValuesIn(
{std::make_tuple(PacingSettings({.mode = PacingMode::kNoPacing}),
/*frame_timestamp_ms=*/std::vector<int>{0, 100},
/*frame_capture_delay_ms=*/std::vector<int>{0, 0},
/*expected_frame_start_ms=*/std::vector<int>{0, 0}),
// Pace with rate equal to the source frame rate. Frames are captured
// instantly. Verify that frames are paced with the source frame rate.
std::make_tuple(PacingSettings({.mode = PacingMode::kRealTime}),
/*frame_timestamp_ms=*/std::vector<int>{0, 100},
/*frame_capture_delay_ms=*/std::vector<int>{0, 0},
/*expected_frame_start_ms=*/std::vector<int>{0, 100}),
// Pace with rate equal to the source frame rate. Frame capture is
// delayed by more than pacing time. Verify that no extra delay is
// added.
std::make_tuple(PacingSettings({.mode = PacingMode::kRealTime}),
/*frame_timestamp_ms=*/std::vector<int>{0, 100},
/*frame_capture_delay_ms=*/std::vector<int>{0, 200},
/*expected_frame_start_ms=*/std::vector<int>{0, 200}),
// Pace with constant rate less then source frame rate. Frames are
// captured instantly. Verify that frames are paced with the requested
// constant rate.
std::make_tuple(
PacingSettings({.mode = PacingMode::kConstantRate,
.constant_rate = Frequency::Hertz(20)}),
/*frame_timestamp_ms=*/std::vector<int>{0, 100},
/*frame_capture_delay_ms=*/std::vector<int>{0, 0},
/*expected_frame_start_ms=*/std::vector<int>{0, 50}),
// Pace with constant rate less then source frame rate. Frame capture
// is delayed by more than the pacing time. Verify that no extra delay
// is added.
std::make_tuple(
PacingSettings({.mode = PacingMode::kConstantRate,
.constant_rate = Frequency::Hertz(20)}),
/*frame_timestamp_ms=*/std::vector<int>{0, 100},
/*frame_capture_delay_ms=*/std::vector<int>{0, 200},
/*expected_frame_start_ms=*/std::vector<int>{0, 200})}));
{// No pacing.
PacingTestParams({.pacing_settings = {.mode = PacingMode::kNoPacing},
.framerate = Frequency::Hertz(10),
.num_frames = 3,
.expected_delta_ms = {0, 0}}),
// Real-time pacing.
PacingTestParams({.pacing_settings = {.mode = PacingMode::kRealTime},
.framerate = Frequency::Hertz(10),
.num_frames = 3,
.expected_delta_ms = {100, 100}}),
// Pace with specified constant rate.
PacingTestParams(
{.pacing_settings = {.mode = PacingMode::kConstantRate,
.constant_rate = Frequency::Hertz(20)},
.framerate = Frequency::Hertz(10),
.num_frames = 3,
.expected_delta_ms = {50, 50}})}));
} // namespace test
} // namespace webrtc