VideoReceiveStream2: AV1 encoded sink support.

This change adds support for emitting encoded frames
for recording when the decoder can't easily read out
encoded width and height as is the case for AV1 streams,
in which case the information is buried in OBUs. Downstream
project relies on resolution information being present for key
frames. With the change, VideoReceiveStream2 infers the
resolution from decoded frames, and supplies it in the
RecordableEncodedFrames.

Bug: chromium:1191972
Change-Id: I07beda6526206c80a732976e8e19d3581489b8fe
Reviewed-on: https://webrtc-review.googlesource.com/c/src/+/214126
Reviewed-by: Harald Alvestrand <hta@webrtc.org>
Reviewed-by: Philip Eliasson <philipel@webrtc.org>
Reviewed-by: Erik Språng <sprang@webrtc.org>
Commit-Queue: Markus Handell <handellm@webrtc.org>
Cr-Commit-Position: refs/heads/master@{#33662}
This commit is contained in:
Markus Handell 2021-04-08 19:19:50 +02:00 committed by Commit Bot
parent edc946ea81
commit 588f9b3705
7 changed files with 219 additions and 21 deletions

View File

@ -26,8 +26,10 @@ class RecordableEncodedFrame {
public:
// Encoded resolution in pixels
struct EncodedResolution {
unsigned width;
unsigned height;
bool empty() const { return width == 0 && height == 0; }
unsigned width = 0;
unsigned height = 0;
};
virtual ~RecordableEncodedFrame() = default;

View File

@ -1426,6 +1426,13 @@ class WebRtcVideoChannelEncodedFrameCallbackTest : public ::testing::Test {
uint8_t* buf_ptr = packet.AllocatePayload(11);
memset(buf_ptr, 0, 11); // Pass MSAN (don't care about bytes 1-9)
buf_ptr[0] = 0x10; // Partition ID 0 + beginning of partition.
constexpr unsigned width = 1080;
constexpr unsigned height = 720;
buf_ptr[6] = width & 255;
buf_ptr[7] = width >> 8;
buf_ptr[8] = height & 255;
buf_ptr[9] = height >> 8;
call_->Receiver()->DeliverPacket(webrtc::MediaType::VIDEO, packet.Buffer(),
/*packet_time_us=*/0);
}

View File

@ -27,11 +27,6 @@
namespace webrtc {
namespace test {
namespace {
const int kDefaultWidth = 320;
const int kDefaultHeight = 180;
} // namespace
FakeDecoder::FakeDecoder() : FakeDecoder(nullptr) {}
FakeDecoder::FakeDecoder(TaskQueueFactory* task_queue_factory)

View File

@ -25,6 +25,8 @@ namespace test {
class FakeDecoder : public VideoDecoder {
public:
enum { kDefaultWidth = 320, kDefaultHeight = 180 };
FakeDecoder();
explicit FakeDecoder(TaskQueueFactory* task_queue_factory);
virtual ~FakeDecoder() {}

View File

@ -41,6 +41,7 @@
#include "rtc_base/location.h"
#include "rtc_base/logging.h"
#include "rtc_base/strings/string_builder.h"
#include "rtc_base/synchronization/mutex.h"
#include "rtc_base/system/thread_registry.h"
#include "rtc_base/time_utils.h"
#include "rtc_base/trace_event.h"
@ -70,13 +71,14 @@ constexpr int kDefaultMaximumPreStreamDecoders = 100;
// from EncodedFrame.
class WebRtcRecordableEncodedFrame : public RecordableEncodedFrame {
public:
explicit WebRtcRecordableEncodedFrame(const EncodedFrame& frame)
explicit WebRtcRecordableEncodedFrame(
const EncodedFrame& frame,
RecordableEncodedFrame::EncodedResolution resolution)
: buffer_(frame.GetEncodedData()),
render_time_ms_(frame.RenderTime()),
codec_(frame.CodecSpecific()->codecType),
is_key_frame_(frame.FrameType() == VideoFrameType::kVideoFrameKey),
resolution_{frame.EncodedImage()._encodedWidth,
frame.EncodedImage()._encodedHeight} {
resolution_(resolution) {
if (frame.ColorSpace()) {
color_space_ = *frame.ColorSpace();
}
@ -179,6 +181,12 @@ class NullVideoDecoder : public webrtc::VideoDecoder {
const char* ImplementationName() const override { return "NullVideoDecoder"; }
};
bool IsKeyFrameAndUnspecifiedResolution(const EncodedFrame& frame) {
return frame.FrameType() == VideoFrameType::kVideoFrameKey &&
frame.EncodedImage()._encodedWidth == 0 &&
frame.EncodedImage()._encodedHeight == 0;
}
// TODO(https://bugs.webrtc.org/9974): Consider removing this workaround.
// Maximum time between frames before resetting the FrameBuffer to avoid RTP
// timestamps wraparound to affect FrameBuffer.
@ -256,7 +264,6 @@ VideoReceiveStream2::VideoReceiveStream2(
RTC_DCHECK(worker_thread_);
RTC_DCHECK(config_.renderer);
RTC_DCHECK(call_stats_);
module_process_sequence_checker_.Detach();
RTC_DCHECK(!config_.decoders.empty());
@ -536,6 +543,22 @@ void VideoReceiveStream2::OnFrame(const VideoFrame& video_frame) {
source_tracker_.OnFrameDelivered(video_frame.packet_infos());
config_.renderer->OnFrame(video_frame);
webrtc::MutexLock lock(&pending_resolution_mutex_);
if (pending_resolution_.has_value()) {
if (!pending_resolution_->empty() &&
(video_frame.width() != static_cast<int>(pending_resolution_->width) ||
video_frame.height() !=
static_cast<int>(pending_resolution_->height))) {
RTC_LOG(LS_WARNING)
<< "Recordable encoded frame stream resolution was reported as "
<< pending_resolution_->width << "x" << pending_resolution_->height
<< " but the stream is now " << video_frame.width()
<< video_frame.height();
}
pending_resolution_ = RecordableEncodedFrame::EncodedResolution{
static_cast<unsigned>(video_frame.width()),
static_cast<unsigned>(video_frame.height())};
}
}
void VideoReceiveStream2::SetFrameDecryptor(
@ -699,13 +722,16 @@ void VideoReceiveStream2::HandleEncodedFrame(
}
}
int decode_result = video_receiver_.Decode(frame.get());
int64_t frame_id = frame->Id();
bool received_frame_is_keyframe =
frame->FrameType() == VideoFrameType::kVideoFrameKey;
int decode_result = DecodeAndMaybeDispatchEncodedFrame(std::move(frame));
if (decode_result == WEBRTC_VIDEO_CODEC_OK ||
decode_result == WEBRTC_VIDEO_CODEC_OK_REQUEST_KEYFRAME) {
keyframe_required_ = false;
frame_decoded_ = true;
decoded_frame_picture_id = frame->Id();
decoded_frame_picture_id = frame_id;
if (decode_result == WEBRTC_VIDEO_CODEC_OK_REQUEST_KEYFRAME)
force_request_key_frame = true;
@ -717,9 +743,6 @@ void VideoReceiveStream2::HandleEncodedFrame(
force_request_key_frame = true;
}
bool received_frame_is_keyframe =
frame->FrameType() == VideoFrameType::kVideoFrameKey;
worker_thread_->PostTask(ToQueuedTask(
task_safety_,
[this, now_ms, received_frame_is_keyframe, force_request_key_frame,
@ -733,10 +756,66 @@ void VideoReceiveStream2::HandleEncodedFrame(
force_request_key_frame,
keyframe_request_is_due);
}));
}
if (encoded_frame_buffer_function_) {
encoded_frame_buffer_function_(WebRtcRecordableEncodedFrame(*frame));
int VideoReceiveStream2::DecodeAndMaybeDispatchEncodedFrame(
std::unique_ptr<EncodedFrame> frame) {
// Running on decode_queue_.
// If |buffered_encoded_frames_| grows out of control (=60 queued frames),
// maybe due to a stuck decoder, we just halt the process here and log the
// error.
const bool encoded_frame_output_enabled =
encoded_frame_buffer_function_ != nullptr &&
buffered_encoded_frames_.size() < kBufferedEncodedFramesMaxSize;
EncodedFrame* frame_ptr = frame.get();
if (encoded_frame_output_enabled) {
// If we receive a key frame with unset resolution, hold on dispatching the
// frame and following ones until we know a resolution of the stream.
// NOTE: The code below has a race where it can report the wrong
// resolution for keyframes after an initial keyframe of other resolution.
// However, the only known consumer of this information is the W3C
// MediaRecorder and it will only use the resolution in the first encoded
// keyframe from WebRTC, so misreporting is fine.
buffered_encoded_frames_.push_back(std::move(frame));
if (buffered_encoded_frames_.size() == kBufferedEncodedFramesMaxSize)
RTC_LOG(LS_ERROR) << "About to halt recordable encoded frame output due "
"to too many buffered frames.";
webrtc::MutexLock lock(&pending_resolution_mutex_);
if (IsKeyFrameAndUnspecifiedResolution(*frame_ptr) &&
!pending_resolution_.has_value())
pending_resolution_.emplace();
}
int decode_result = video_receiver_.Decode(frame_ptr);
if (encoded_frame_output_enabled) {
absl::optional<RecordableEncodedFrame::EncodedResolution>
pending_resolution;
{
// Fish out |pending_resolution_| to avoid taking the mutex on every lap
// or dispatching under the mutex in the flush loop.
webrtc::MutexLock lock(&pending_resolution_mutex_);
if (pending_resolution_.has_value())
pending_resolution = *pending_resolution_;
}
if (!pending_resolution.has_value() || !pending_resolution->empty()) {
// Flush the buffered frames.
for (const auto& frame : buffered_encoded_frames_) {
RecordableEncodedFrame::EncodedResolution resolution{
frame->EncodedImage()._encodedWidth,
frame->EncodedImage()._encodedHeight};
if (IsKeyFrameAndUnspecifiedResolution(*frame)) {
RTC_DCHECK(!pending_resolution->empty());
resolution = *pending_resolution;
}
encoded_frame_buffer_function_(
WebRtcRecordableEncodedFrame(*frame, resolution));
}
buffered_encoded_frames_.clear();
}
}
return decode_result;
}
void VideoReceiveStream2::HandleKeyFrameGeneration(

View File

@ -28,6 +28,7 @@
#include "rtc_base/system/no_unique_address.h"
#include "rtc_base/task_queue.h"
#include "rtc_base/task_utils/pending_task_safety_flag.h"
#include "rtc_base/thread_annotations.h"
#include "system_wrappers/include/clock.h"
#include "video/receive_statistics_proxy2.h"
#include "video/rtp_streams_synchronizer2.h"
@ -84,6 +85,9 @@ class VideoReceiveStream2 : public webrtc::VideoReceiveStream,
// The default number of milliseconds to pass before re-requesting a key frame
// to be sent.
static constexpr int kMaxWaitForKeyFrameMs = 200;
// The maximum number of buffered encoded frames when encoded output is
// configured.
static constexpr size_t kBufferedEncodedFramesMaxSize = 60;
VideoReceiveStream2(TaskQueueFactory* task_queue_factory,
TaskQueueBase* current_queue,
@ -172,6 +176,8 @@ class VideoReceiveStream2 : public webrtc::VideoReceiveStream,
RTC_RUN_ON(worker_sequence_checker_);
bool IsReceivingKeyFrame(int64_t timestamp_ms) const
RTC_RUN_ON(worker_sequence_checker_);
int DecodeAndMaybeDispatchEncodedFrame(std::unique_ptr<EncodedFrame> frame)
RTC_RUN_ON(decode_queue_);
void UpdateHistograms();
@ -254,6 +260,16 @@ class VideoReceiveStream2 : public webrtc::VideoReceiveStream,
// Set to true while we're requesting keyframes but not yet received one.
bool keyframe_generation_requested_ RTC_GUARDED_BY(worker_sequence_checker_) =
false;
// Lock to avoid unnecessary per-frame idle wakeups in the code.
webrtc::Mutex pending_resolution_mutex_;
// Signal from decode queue to OnFrame callback to fill pending_resolution_.
// absl::nullopt - no resolution needed. 0x0 - next OnFrame to fill with
// received resolution. Not 0x0 - OnFrame has filled a resolution.
absl::optional<RecordableEncodedFrame::EncodedResolution> pending_resolution_
RTC_GUARDED_BY(pending_resolution_mutex_);
// Buffered encoded frames held while waiting for decoded resolution.
std::vector<std::unique_ptr<EncodedFrame>> buffered_encoded_frames_
RTC_GUARDED_BY(decode_queue_);
// Set by the field trial WebRTC-LowLatencyRenderer. The parameter |enabled|
// determines if the low-latency renderer algorithm should be used for the

View File

@ -11,12 +11,14 @@
#include "video/video_receive_stream2.h"
#include <algorithm>
#include <limits>
#include <memory>
#include <utility>
#include <vector>
#include "api/task_queue/default_task_queue_factory.h"
#include "api/test/video/function_video_decoder_factory.h"
#include "api/video/video_frame.h"
#include "api/video_codecs/video_decoder.h"
#include "call/rtp_stream_receiver_controller.h"
#include "common_video/test/utilities.h"
@ -40,9 +42,13 @@ namespace webrtc {
namespace {
using ::testing::_;
using ::testing::AllOf;
using ::testing::ElementsAreArray;
using ::testing::Field;
using ::testing::InSequence;
using ::testing::Invoke;
using ::testing::IsEmpty;
using ::testing::Property;
using ::testing::SizeIs;
constexpr int kDefaultTimeOutMs = 50;
@ -442,15 +448,25 @@ TEST_F(VideoReceiveStream2TestWithFakeDecoder, RenderedFrameUpdatesGetSources) {
}
}
std::unique_ptr<FrameObjectFake> MakeFrame(VideoFrameType frame_type,
int picture_id) {
std::unique_ptr<FrameObjectFake> MakeFrameWithResolution(
VideoFrameType frame_type,
int picture_id,
int width,
int height) {
auto frame = std::make_unique<FrameObjectFake>();
frame->SetPayloadType(99);
frame->SetId(picture_id);
frame->SetFrameType(frame_type);
frame->_encodedWidth = width;
frame->_encodedHeight = height;
return frame;
}
std::unique_ptr<FrameObjectFake> MakeFrame(VideoFrameType frame_type,
int picture_id) {
return MakeFrameWithResolution(frame_type, picture_id, 320, 240);
}
TEST_F(VideoReceiveStream2TestWithFakeDecoder,
PassesFrameWhenEncodedFramesCallbackSet) {
testing::MockFunction<void(const RecordableEncodedFrame&)> callback;
@ -485,6 +501,27 @@ TEST_F(VideoReceiveStream2TestWithFakeDecoder,
class VideoReceiveStream2TestWithSimulatedClock
: public ::testing::TestWithParam<int> {
public:
class FakeRenderer : public rtc::VideoSinkInterface<webrtc::VideoFrame> {
public:
void SignalDoneAfterFrames(int num_frames_received) {
signal_after_frame_count_ = num_frames_received;
if (frame_count_ == signal_after_frame_count_)
event_.Set();
}
void OnFrame(const webrtc::VideoFrame& frame) override {
if (++frame_count_ == signal_after_frame_count_)
event_.Set();
}
void WaitUntilDone() { event_.Wait(rtc::Event::kForever); }
private:
int signal_after_frame_count_ = std::numeric_limits<int>::max();
int frame_count_ = 0;
rtc::Event event_;
};
class FakeDecoder2 : public test::FakeDecoder {
public:
explicit FakeDecoder2(std::function<void()> decode_callback)
@ -559,7 +596,7 @@ class VideoReceiveStream2TestWithSimulatedClock
test::FunctionVideoDecoderFactory fake_decoder_factory_;
std::unique_ptr<ProcessThread> process_thread_;
MockTransport mock_transport_;
cricket::FakeVideoRenderer fake_renderer_;
FakeRenderer fake_renderer_;
VideoReceiveStream::Config config_;
internal::CallStats call_stats_;
PacketRouter packet_router_;
@ -603,6 +640,66 @@ TEST_P(VideoReceiveStream2TestWithSimulatedClock,
loop_.Run();
}
TEST_P(VideoReceiveStream2TestWithSimulatedClock,
DispatchesEncodedFrameSequenceStartingWithKeyframeWithoutResolution) {
video_receive_stream_.Start();
testing::MockFunction<void(const RecordableEncodedFrame&)> callback;
video_receive_stream_.SetAndGetRecordingState(
VideoReceiveStream::RecordingState(callback.AsStdFunction()),
/*generate_key_frame=*/false);
InSequence s;
EXPECT_CALL(
callback,
Call(AllOf(
Property(&RecordableEncodedFrame::resolution,
Field(&RecordableEncodedFrame::EncodedResolution::width,
test::FakeDecoder::kDefaultWidth)),
Property(&RecordableEncodedFrame::resolution,
Field(&RecordableEncodedFrame::EncodedResolution::height,
test::FakeDecoder::kDefaultHeight)))));
EXPECT_CALL(callback, Call);
fake_renderer_.SignalDoneAfterFrames(2);
PassEncodedFrameAndWait(
MakeFrameWithResolution(VideoFrameType::kVideoFrameKey, 0, 0, 0));
PassEncodedFrameAndWait(
MakeFrameWithResolution(VideoFrameType::kVideoFrameDelta, 1, 0, 0));
fake_renderer_.WaitUntilDone();
video_receive_stream_.Stop();
}
TEST_P(VideoReceiveStream2TestWithSimulatedClock,
DispatchesEncodedFrameSequenceStartingWithKeyframeWithResolution) {
video_receive_stream_.Start();
testing::MockFunction<void(const RecordableEncodedFrame&)> callback;
video_receive_stream_.SetAndGetRecordingState(
VideoReceiveStream::RecordingState(callback.AsStdFunction()),
/*generate_key_frame=*/false);
InSequence s;
EXPECT_CALL(
callback,
Call(AllOf(
Property(
&RecordableEncodedFrame::resolution,
Field(&RecordableEncodedFrame::EncodedResolution::width, 1080)),
Property(&RecordableEncodedFrame::resolution,
Field(&RecordableEncodedFrame::EncodedResolution::height,
720)))));
EXPECT_CALL(callback, Call);
fake_renderer_.SignalDoneAfterFrames(2);
PassEncodedFrameAndWait(
MakeFrameWithResolution(VideoFrameType::kVideoFrameKey, 0, 1080, 720));
PassEncodedFrameAndWait(
MakeFrameWithResolution(VideoFrameType::kVideoFrameDelta, 1, 0, 0));
fake_renderer_.WaitUntilDone();
video_receive_stream_.Stop();
}
INSTANTIATE_TEST_SUITE_P(
RtxTime,
VideoReceiveStream2TestWithSimulatedClock,