aboutsummaryrefslogtreecommitdiff
path: root/vda
diff options
context:
space:
mode:
authorHirokazu Honda <hiroh@google.com>2017-08-31 22:03:50 +0900
committerHirokazu Honda <hiroh@google.com>2017-09-07 17:51:17 +0900
commitf1eed46488cf0d61c9212d6db22a8c45d1160e28 (patch)
tree47d8c389c4cd3e8fb52cb59b474cf2d75ba325ee /vda
parent744cff12c41df387d950bf38d095d17f9802b105 (diff)
downloadv4l2_codec2-f1eed46488cf0d61c9212d6db22a8c45d1160e28.tar.gz
Port v4l2_video_decode_accelerator from Chromium
commit head: 726ad599611c6b8b123c9ace70b644d68671a173 file: src/media/gpu/v4l2_video_decode_accelerator.* 1) Remove GL-related components. 2) Remove Image Processor. 3) ImportBuffersForPicture: replace gfx::GpuMemoryBufferHandle with NativePixmapHandle. 4) Add IsSupportedOutputFmtFourcc() to replace device_->CanCreateEGLImageFrom(). Bug: 62360273 Test: mmm external/v4l2_codec2 Change-Id: I7ca5c269270126ec9afe565997336c4a2dd479aa
Diffstat (limited to 'vda')
-rw-r--r--vda/Android.mk1
-rw-r--r--vda/v4l2_video_decode_accelerator.cc2090
-rw-r--r--vda/v4l2_video_decode_accelerator.h510
3 files changed, 2601 insertions, 0 deletions
diff --git a/vda/Android.mk b/vda/Android.mk
index 554c5af..12f0ea1 100644
--- a/vda/Android.mk
+++ b/vda/Android.mk
@@ -16,6 +16,7 @@ LOCAL_SRC_FILES:= \
shared_memory_region.cc \
v4l2_device.cc \
v4l2_slice_video_decode_accelerator.cc \
+ v4l2_video_decode_accelerator.cc \
video_codecs.cc \
video_decode_accelerator.cc \
vp8_bool_decoder.cc \
diff --git a/vda/v4l2_video_decode_accelerator.cc b/vda/v4l2_video_decode_accelerator.cc
new file mode 100644
index 0000000..0003688
--- /dev/null
+++ b/vda/v4l2_video_decode_accelerator.cc
@@ -0,0 +1,2090 @@
+// Copyright 2014 The Chromium Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file.
+
+#include "v4l2_video_decode_accelerator.h"
+
+#include <dlfcn.h>
+#include <errno.h>
+#include <fcntl.h>
+#include <poll.h>
+#include <string.h>
+#include <sys/eventfd.h>
+#include <sys/ioctl.h>
+#include <sys/mman.h>
+
+#include "base/bind.h"
+#include "base/command_line.h"
+#include "base/message_loop/message_loop.h"
+#include "base/numerics/safe_conversions.h"
+#include "base/posix/eintr_wrapper.h"
+#include "base/single_thread_task_runner.h"
+#include "base/memory/ptr_util.h"
+#include "base/threading/thread_task_runner_handle.h"
+#include "base/trace_event/trace_event.h"
+#include "build/build_config.h"
+#include "h264_parser.h"
+#include "rect.h"
+#include "shared_memory_region.h"
+#include "videodev2.h"
+
+#define LOGF(level) LOG(level) << __func__ << "(): "
+#define DLOGF(level) DLOG(level) << __func__ << "(): "
+#define DVLOGF(level) DVLOG(level) << __func__ << "(): "
+#define VLOGF(level) VLOG(level) << __func__ << "(): "
+#define PLOGF(level) PLOG(level) << __func__ << "(): "
+
+#define NOTIFY_ERROR(x) \
+ do { \
+ LOGF(ERROR) << "Setting error state:" << x; \
+ SetErrorState(x); \
+ } while (0)
+
+#define IOCTL_OR_ERROR_RETURN_VALUE(type, arg, value, type_str) \
+ do { \
+ if (device_->Ioctl(type, arg) != 0) { \
+ PLOGF(ERROR) << "ioctl() failed: " << type_str; \
+ NOTIFY_ERROR(PLATFORM_FAILURE); \
+ return value; \
+ } \
+ } while (0)
+
+#define IOCTL_OR_ERROR_RETURN(type, arg) \
+ IOCTL_OR_ERROR_RETURN_VALUE(type, arg, ((void)0), #type)
+
+#define IOCTL_OR_ERROR_RETURN_FALSE(type, arg) \
+ IOCTL_OR_ERROR_RETURN_VALUE(type, arg, false, #type)
+
+#define IOCTL_OR_LOG_ERROR(type, arg) \
+ do { \
+ if (device_->Ioctl(type, arg) != 0) \
+ PLOGF(ERROR) << "ioctl() failed: " << #type; \
+ } while (0)
+
+namespace media {
+
+// static
+const uint32_t V4L2VideoDecodeAccelerator::supported_input_fourccs_[] = {
+ V4L2_PIX_FMT_H264, V4L2_PIX_FMT_VP8, V4L2_PIX_FMT_VP9,
+};
+
+struct V4L2VideoDecodeAccelerator::BitstreamBufferRef {
+ BitstreamBufferRef(
+ base::WeakPtr<Client>& client,
+ scoped_refptr<base::SingleThreadTaskRunner>& client_task_runner,
+ std::unique_ptr<SharedMemoryRegion> shm,
+ int32_t input_id);
+ ~BitstreamBufferRef();
+ const base::WeakPtr<Client> client;
+ const scoped_refptr<base::SingleThreadTaskRunner> client_task_runner;
+ const std::unique_ptr<SharedMemoryRegion> shm;
+ size_t bytes_used;
+ const int32_t input_id;
+};
+
+V4L2VideoDecodeAccelerator::BitstreamBufferRef::BitstreamBufferRef(
+ base::WeakPtr<Client>& client,
+ scoped_refptr<base::SingleThreadTaskRunner>& client_task_runner,
+ std::unique_ptr<SharedMemoryRegion> shm,
+ int32_t input_id)
+ : client(client),
+ client_task_runner(client_task_runner),
+ shm(std::move(shm)),
+ bytes_used(0),
+ input_id(input_id) {}
+
+V4L2VideoDecodeAccelerator::BitstreamBufferRef::~BitstreamBufferRef() {
+ if (input_id >= 0) {
+ client_task_runner->PostTask(
+ FROM_HERE,
+ base::Bind(&Client::NotifyEndOfBitstreamBuffer, client, input_id));
+ }
+}
+
+V4L2VideoDecodeAccelerator::InputRecord::InputRecord()
+ : at_device(false), address(NULL), length(0), bytes_used(0), input_id(-1) {}
+
+V4L2VideoDecodeAccelerator::InputRecord::~InputRecord() {}
+
+V4L2VideoDecodeAccelerator::OutputRecord::OutputRecord()
+ : state(kFree),
+ picture_id(-1),
+ cleared(false) {}
+
+V4L2VideoDecodeAccelerator::OutputRecord::~OutputRecord() {}
+
+V4L2VideoDecodeAccelerator::PictureRecord::PictureRecord(bool cleared,
+ const Picture& picture)
+ : cleared(cleared), picture(picture) {}
+
+V4L2VideoDecodeAccelerator::PictureRecord::~PictureRecord() {}
+
+V4L2VideoDecodeAccelerator::V4L2VideoDecodeAccelerator(
+ const scoped_refptr<V4L2Device>& device)
+ : child_task_runner_(base::ThreadTaskRunnerHandle::Get()),
+ decoder_thread_("V4L2DecoderThread"),
+ decoder_state_(kUninitialized),
+ output_mode_(Config::OutputMode::ALLOCATE),
+ device_(device),
+ decoder_delay_bitstream_buffer_id_(-1),
+ decoder_current_input_buffer_(-1),
+ decoder_decode_buffer_tasks_scheduled_(0),
+ decoder_frames_at_client_(0),
+ decoder_flushing_(false),
+ decoder_cmd_supported_(false),
+ flush_awaiting_last_output_buffer_(false),
+ reset_pending_(false),
+ decoder_partial_frame_pending_(false),
+ input_streamon_(false),
+ input_buffer_queued_count_(0),
+ output_streamon_(false),
+ output_buffer_queued_count_(0),
+ output_dpb_size_(0),
+ output_planes_count_(0),
+ picture_clearing_count_(0),
+ device_poll_thread_("V4L2DevicePollThread"),
+ video_profile_(VIDEO_CODEC_PROFILE_UNKNOWN),
+ input_format_fourcc_(0),
+ output_format_fourcc_(0),
+ weak_this_factory_(this) {
+ weak_this_ = weak_this_factory_.GetWeakPtr();
+}
+
+V4L2VideoDecodeAccelerator::~V4L2VideoDecodeAccelerator() {
+ DCHECK(!decoder_thread_.IsRunning());
+ DCHECK(!device_poll_thread_.IsRunning());
+
+ // These maps have members that should be manually destroyed, e.g. file
+ // descriptors, mmap() segments, etc.
+ DCHECK(input_buffer_map_.empty());
+ DCHECK(output_buffer_map_.empty());
+}
+
+bool V4L2VideoDecodeAccelerator::Initialize(const Config& config,
+ Client* client) {
+ VLOGF(2) << "profile: " << config.profile
+ << ", output_mode=" << static_cast<int>(config.output_mode);
+ DCHECK(child_task_runner_->BelongsToCurrentThread());
+ DCHECK_EQ(decoder_state_, kUninitialized);
+
+ if (config.output_mode != Config::OutputMode::IMPORT) {
+ NOTREACHED() << "Only IMPORT OutputModes are supported";
+ return false;
+ }
+
+ client_ptr_factory_.reset(new base::WeakPtrFactory<Client>(client));
+ client_ = client_ptr_factory_->GetWeakPtr();
+ // If we haven't been set up to decode on separate thread via
+ // TryToSetupDecodeOnSeparateThread(), use the main thread/client for
+ // decode tasks.
+ if (!decode_task_runner_) {
+ decode_task_runner_ = child_task_runner_;
+ DCHECK(!decode_client_);
+ decode_client_ = client_;
+ }
+
+ video_profile_ = config.profile;
+
+ input_format_fourcc_ =
+ V4L2Device::VideoCodecProfileToV4L2PixFmt(video_profile_, false);
+
+ if (!device_->Open(V4L2Device::Type::kDecoder, input_format_fourcc_)) {
+ VLOGF(1) << "Failed to open device for profile: " << config.profile
+ << " fourcc: " << std::hex << "0x" << input_format_fourcc_;
+ return false;
+ }
+
+ // Capabilities check.
+ struct v4l2_capability caps;
+ const __u32 kCapsRequired = V4L2_CAP_VIDEO_M2M_MPLANE | V4L2_CAP_STREAMING;
+ IOCTL_OR_ERROR_RETURN_FALSE(VIDIOC_QUERYCAP, &caps);
+ if ((caps.capabilities & kCapsRequired) != kCapsRequired) {
+ LOGF(ERROR) << "ioctl() failed: VIDIOC_QUERYCAP"
+ << ", caps check failed: 0x" << std::hex << caps.capabilities;
+ return false;
+ }
+
+ if (!SetupFormats())
+ return false;
+
+ if (video_profile_ >= H264PROFILE_MIN && video_profile_ <= H264PROFILE_MAX) {
+ decoder_h264_parser_.reset(new H264Parser());
+ }
+
+ if (!decoder_thread_.Start()) {
+ LOGF(ERROR) << "decoder thread failed to start";
+ return false;
+ }
+
+ decoder_state_ = kInitialized;
+ output_mode_ = config.output_mode;
+
+ // InitializeTask will NOTIFY_ERROR on failure.
+ decoder_thread_.task_runner()->PostTask(
+ FROM_HERE, base::Bind(&V4L2VideoDecodeAccelerator::InitializeTask,
+ base::Unretained(this)));
+
+ return true;
+}
+
+void V4L2VideoDecodeAccelerator::InitializeTask() {
+ VLOGF(2);
+ DCHECK(decoder_thread_.task_runner()->BelongsToCurrentThread());
+ DCHECK_EQ(decoder_state_, kInitialized);
+
+ // Subscribe to the resolution change event.
+ struct v4l2_event_subscription sub;
+ memset(&sub, 0, sizeof(sub));
+ sub.type = V4L2_EVENT_SOURCE_CHANGE;
+ IOCTL_OR_ERROR_RETURN(VIDIOC_SUBSCRIBE_EVENT, &sub);
+
+ if (!CreateInputBuffers()) {
+ NOTIFY_ERROR(PLATFORM_FAILURE);
+ return;
+ }
+
+ decoder_cmd_supported_ = IsDecoderCmdSupported();
+
+ if (!StartDevicePoll())
+ return;
+}
+
+void V4L2VideoDecodeAccelerator::Decode(
+ const BitstreamBuffer& bitstream_buffer) {
+ DVLOGF(4) << "input_id=" << bitstream_buffer.id()
+ << ", size=" << bitstream_buffer.size();
+ DCHECK(decode_task_runner_->BelongsToCurrentThread());
+
+ if (bitstream_buffer.id() < 0) {
+ LOGF(ERROR) << "Invalid bitstream_buffer, id: " << bitstream_buffer.id();
+ if (base::SharedMemory::IsHandleValid(bitstream_buffer.handle()))
+ base::SharedMemory::CloseHandle(bitstream_buffer.handle());
+ NOTIFY_ERROR(INVALID_ARGUMENT);
+ return;
+ }
+
+ // DecodeTask() will take care of running a DecodeBufferTask().
+ decoder_thread_.task_runner()->PostTask(
+ FROM_HERE, base::Bind(&V4L2VideoDecodeAccelerator::DecodeTask,
+ base::Unretained(this), bitstream_buffer));
+}
+
+void V4L2VideoDecodeAccelerator::AssignPictureBuffers(
+ const std::vector<PictureBuffer>& buffers) {
+ VLOGF(2) << "buffer_count=" << buffers.size();
+ DCHECK(child_task_runner_->BelongsToCurrentThread());
+
+ decoder_thread_.task_runner()->PostTask(
+ FROM_HERE,
+ base::Bind(&V4L2VideoDecodeAccelerator::AssignPictureBuffersTask,
+ base::Unretained(this), buffers));
+}
+
+void V4L2VideoDecodeAccelerator::AssignPictureBuffersTask(
+ const std::vector<PictureBuffer>& buffers) {
+ VLOGF(2);
+ DCHECK(decoder_thread_.task_runner()->BelongsToCurrentThread());
+ DCHECK_EQ(decoder_state_, kAwaitingPictureBuffers);
+
+ uint32_t req_buffer_count = output_dpb_size_ + kDpbOutputBufferExtraCount;
+
+ if (buffers.size() < req_buffer_count) {
+ LOGF(ERROR) << "Failed to provide requested picture buffers. (Got "
+ << buffers.size() << ", requested " << req_buffer_count << ")";
+ NOTIFY_ERROR(INVALID_ARGUMENT);
+ return;
+ }
+
+ // Allocate the output buffers.
+ struct v4l2_requestbuffers reqbufs;
+ memset(&reqbufs, 0, sizeof(reqbufs));
+ reqbufs.count = buffers.size();
+ reqbufs.type = V4L2_BUF_TYPE_VIDEO_CAPTURE_MPLANE;
+ reqbufs.memory = V4L2_MEMORY_MMAP;
+ IOCTL_OR_ERROR_RETURN(VIDIOC_REQBUFS, &reqbufs);
+
+ if (reqbufs.count != buffers.size()) {
+ DLOGF(ERROR) << "Could not allocate enough output buffers";
+ NOTIFY_ERROR(PLATFORM_FAILURE);
+ return;
+ }
+
+ DCHECK(free_output_buffers_.empty());
+ DCHECK(output_buffer_map_.empty());
+ output_buffer_map_.resize(buffers.size());
+
+ // Always use IMPORT output mode for Android solution.
+ DCHECK_EQ(output_mode_, Config::OutputMode::IMPORT);
+
+ for (size_t i = 0; i < output_buffer_map_.size(); ++i) {
+ OutputRecord& output_record = output_buffer_map_[i];
+ DCHECK_EQ(output_record.state, kFree);
+ DCHECK_EQ(output_record.picture_id, -1);
+ DCHECK_EQ(output_record.cleared, false);
+
+ output_record.picture_id = buffers[i].id();
+
+ // This will remain kAtClient until ImportBufferForPicture is called, either
+ // by the client, or by ourselves, if we are allocating.
+ output_record.state = kAtClient;
+
+ DVLOGF(3) << "buffer[" << i << "]: picture_id=" << output_record.picture_id;
+ }
+}
+
+void V4L2VideoDecodeAccelerator::ImportBufferForPicture(
+ int32_t picture_buffer_id,
+ const NativePixmapHandle& native_pixmap_handle) {
+ DVLOGF(3) << "picture_buffer_id=" << picture_buffer_id;
+ DCHECK(child_task_runner_->BelongsToCurrentThread());
+
+ if (output_mode_ != Config::OutputMode::IMPORT) {
+ LOGF(ERROR) << "Cannot import in non-import mode";
+ NOTIFY_ERROR(INVALID_ARGUMENT);
+ return;
+ }
+ std::vector<base::ScopedFD> dmabuf_fds;
+ for (const auto& fd : native_pixmap_handle.fds) {
+ DCHECK_NE(fd.fd, -1);
+ dmabuf_fds.push_back(base::ScopedFD(fd.fd));
+ }
+
+ decode_task_runner_->PostTask(
+ FROM_HERE,
+ base::Bind(&V4L2VideoDecodeAccelerator::ImportBufferForPictureTask,
+ base::Unretained(this), picture_buffer_id,
+ base::Passed(&dmabuf_fds)));
+}
+
+void V4L2VideoDecodeAccelerator::ImportBufferForPictureTask(
+ int32_t picture_buffer_id,
+ std::vector<base::ScopedFD> dmabuf_fds) {
+ DVLOGF(3) << "picture_buffer_id=" << picture_buffer_id
+ << ", dmabuf_fds.size()=" << dmabuf_fds.size();
+ DCHECK(decoder_thread_.task_runner()->BelongsToCurrentThread());
+
+ const auto iter =
+ std::find_if(output_buffer_map_.begin(), output_buffer_map_.end(),
+ [picture_buffer_id](const OutputRecord& output_record) {
+ return output_record.picture_id == picture_buffer_id;
+ });
+ if (iter == output_buffer_map_.end()) {
+ // It's possible that we've already posted a DismissPictureBuffer for this
+ // picture, but it has not yet executed when this ImportBufferForPicture was
+ // posted to us by the client. In that case just ignore this (we've already
+ // dismissed it and accounted for that).
+ DVLOGF(3) << "got picture id=" << picture_buffer_id
+ << " not in use (anymore?).";
+ return;
+ }
+
+ if (iter->state != kAtClient) {
+ LOGF(ERROR) << "Cannot import buffer not owned by client";
+ NOTIFY_ERROR(INVALID_ARGUMENT);
+ return;
+ }
+
+ size_t index = iter - output_buffer_map_.begin();
+ DCHECK_EQ(std::count(free_output_buffers_.begin(), free_output_buffers_.end(),
+ index),
+ 0);
+
+ iter->state = kFree;
+
+ DCHECK_EQ(output_planes_count_, dmabuf_fds.size());
+
+ iter->processor_output_fds.swap(dmabuf_fds);
+ free_output_buffers_.push_back(index);
+ if (decoder_state_ != kChangingResolution) {
+ Enqueue();
+ ScheduleDecodeBufferTaskIfNeeded();
+ }
+}
+
+void V4L2VideoDecodeAccelerator::ReusePictureBuffer(int32_t picture_buffer_id) {
+ DVLOGF(4) << "picture_buffer_id=" << picture_buffer_id;
+ // Must be run on child thread, as we'll insert a sync in the EGL context.
+ DCHECK(child_task_runner_->BelongsToCurrentThread());
+
+ decoder_thread_.task_runner()->PostTask(
+ FROM_HERE, base::Bind(&V4L2VideoDecodeAccelerator::ReusePictureBufferTask,
+ base::Unretained(this), picture_buffer_id));
+}
+
+void V4L2VideoDecodeAccelerator::Flush() {
+ VLOGF(2);
+ DCHECK(child_task_runner_->BelongsToCurrentThread());
+ decoder_thread_.task_runner()->PostTask(
+ FROM_HERE, base::Bind(&V4L2VideoDecodeAccelerator::FlushTask,
+ base::Unretained(this)));
+}
+
+void V4L2VideoDecodeAccelerator::Reset() {
+ VLOGF(2);
+ DCHECK(child_task_runner_->BelongsToCurrentThread());
+ decoder_thread_.task_runner()->PostTask(
+ FROM_HERE, base::Bind(&V4L2VideoDecodeAccelerator::ResetTask,
+ base::Unretained(this)));
+}
+
+void V4L2VideoDecodeAccelerator::Destroy() {
+ VLOGF(2);
+ DCHECK(child_task_runner_->BelongsToCurrentThread());
+
+ // We're destroying; cancel all callbacks.
+ client_ptr_factory_.reset();
+ weak_this_factory_.InvalidateWeakPtrs();
+
+ // If the decoder thread is running, destroy using posted task.
+ if (decoder_thread_.IsRunning()) {
+ decoder_thread_.task_runner()->PostTask(
+ FROM_HERE, base::Bind(&V4L2VideoDecodeAccelerator::DestroyTask,
+ base::Unretained(this)));
+ // DestroyTask() will cause the decoder_thread_ to flush all tasks.
+ decoder_thread_.Stop();
+ } else {
+ // Otherwise, call the destroy task directly.
+ DestroyTask();
+ }
+
+ delete this;
+}
+
+bool V4L2VideoDecodeAccelerator::TryToSetupDecodeOnSeparateThread(
+ const base::WeakPtr<Client>& decode_client,
+ const scoped_refptr<base::SingleThreadTaskRunner>& decode_task_runner) {
+ VLOGF(2);
+ decode_client_ = decode_client;
+ decode_task_runner_ = decode_task_runner;
+ return true;
+}
+
+// static
+VideoDecodeAccelerator::SupportedProfiles
+V4L2VideoDecodeAccelerator::GetSupportedProfiles() {
+ scoped_refptr<V4L2Device> device(new V4L2Device());
+ if (!device)
+ return SupportedProfiles();
+
+ return device->GetSupportedDecodeProfiles(arraysize(supported_input_fourccs_),
+ supported_input_fourccs_);
+}
+
+void V4L2VideoDecodeAccelerator::DecodeTask(
+ const BitstreamBuffer& bitstream_buffer) {
+ DVLOGF(4) << "input_id=" << bitstream_buffer.id();
+ DCHECK(decoder_thread_.task_runner()->BelongsToCurrentThread());
+ DCHECK_NE(decoder_state_, kUninitialized);
+ TRACE_EVENT1("Video Decoder", "V4L2VDA::DecodeTask", "input_id",
+ bitstream_buffer.id());
+
+ std::unique_ptr<BitstreamBufferRef> bitstream_record(new BitstreamBufferRef(
+ decode_client_, decode_task_runner_,
+ std::unique_ptr<SharedMemoryRegion>(
+ new SharedMemoryRegion(bitstream_buffer, true)),
+ bitstream_buffer.id()));
+
+ // Skip empty buffer.
+ if (bitstream_buffer.size() == 0)
+ return;
+
+ if (!bitstream_record->shm->Map()) {
+ LOGF(ERROR) << "could not map bitstream_buffer";
+ NOTIFY_ERROR(UNREADABLE_INPUT);
+ return;
+ }
+ DVLOGF(4) << "mapped at=" << bitstream_record->shm->memory();
+
+ if (decoder_state_ == kResetting || decoder_flushing_) {
+ // In the case that we're resetting or flushing, we need to delay decoding
+ // the BitstreamBuffers that come after the Reset() or Flush() call. When
+ // we're here, we know that this DecodeTask() was scheduled by a Decode()
+ // call that came after (in the client thread) the Reset() or Flush() call;
+ // thus set up the delay if necessary.
+ if (decoder_delay_bitstream_buffer_id_ == -1)
+ decoder_delay_bitstream_buffer_id_ = bitstream_record->input_id;
+ } else if (decoder_state_ == kError) {
+ VLOGF(2) << "early out: kError state";
+ return;
+ }
+
+ decoder_input_queue_.push(
+ linked_ptr<BitstreamBufferRef>(bitstream_record.release()));
+ decoder_decode_buffer_tasks_scheduled_++;
+ DecodeBufferTask();
+}
+
+void V4L2VideoDecodeAccelerator::DecodeBufferTask() {
+ DVLOGF(4);
+ DCHECK(decoder_thread_.task_runner()->BelongsToCurrentThread());
+ DCHECK_NE(decoder_state_, kUninitialized);
+ TRACE_EVENT0("Video Decoder", "V4L2VDA::DecodeBufferTask");
+
+ decoder_decode_buffer_tasks_scheduled_--;
+
+ if (decoder_state_ != kInitialized && decoder_state_ != kDecoding) {
+ VLOGF(2) << "early out: state=" << decoder_state_;
+ return;
+ }
+
+ if (decoder_current_bitstream_buffer_ == NULL) {
+ if (decoder_input_queue_.empty()) {
+ // We're waiting for a new buffer -- exit without scheduling a new task.
+ return;
+ }
+ linked_ptr<BitstreamBufferRef>& buffer_ref = decoder_input_queue_.front();
+ if (decoder_delay_bitstream_buffer_id_ == buffer_ref->input_id) {
+ // We're asked to delay decoding on this and subsequent buffers.
+ return;
+ }
+
+ // Setup to use the next buffer.
+ decoder_current_bitstream_buffer_.reset(buffer_ref.release());
+ decoder_input_queue_.pop();
+ const auto& shm = decoder_current_bitstream_buffer_->shm;
+ if (shm) {
+ DVLOGF(4) << "reading input_id="
+ << decoder_current_bitstream_buffer_->input_id
+ << ", addr=" << shm->memory() << ", size=" << shm->size();
+ } else {
+ DCHECK_EQ(decoder_current_bitstream_buffer_->input_id, kFlushBufferId);
+ DVLOGF(4) << "reading input_id=kFlushBufferId";
+ }
+ }
+ bool schedule_task = false;
+ size_t decoded_size = 0;
+ const auto& shm = decoder_current_bitstream_buffer_->shm;
+ if (!shm) {
+ // This is a dummy buffer, queued to flush the pipe. Flush.
+ DCHECK_EQ(decoder_current_bitstream_buffer_->input_id, kFlushBufferId);
+ // Enqueue a buffer guaranteed to be empty. To do that, we flush the
+ // current input, enqueue no data to the next frame, then flush that down.
+ schedule_task = true;
+ if (decoder_current_input_buffer_ != -1 &&
+ input_buffer_map_[decoder_current_input_buffer_].input_id !=
+ kFlushBufferId)
+ schedule_task = FlushInputFrame();
+
+ if (schedule_task && AppendToInputFrame(NULL, 0) && FlushInputFrame()) {
+ VLOGF(2) << "enqueued flush buffer";
+ decoder_partial_frame_pending_ = false;
+ schedule_task = true;
+ } else {
+ // If we failed to enqueue the empty buffer (due to pipeline
+ // backpressure), don't advance the bitstream buffer queue, and don't
+ // schedule the next task. This bitstream buffer queue entry will get
+ // reprocessed when the pipeline frees up.
+ schedule_task = false;
+ }
+ } else if (shm->size() == 0) {
+ // This is a buffer queued from the client that has zero size. Skip.
+ schedule_task = true;
+ } else {
+ // This is a buffer queued from the client, with actual contents. Decode.
+ const uint8_t* const data =
+ reinterpret_cast<const uint8_t*>(shm->memory()) +
+ decoder_current_bitstream_buffer_->bytes_used;
+ const size_t data_size =
+ shm->size() - decoder_current_bitstream_buffer_->bytes_used;
+ if (!AdvanceFrameFragment(data, data_size, &decoded_size)) {
+ NOTIFY_ERROR(UNREADABLE_INPUT);
+ return;
+ }
+ // AdvanceFrameFragment should not return a size larger than the buffer
+ // size, even on invalid data.
+ CHECK_LE(decoded_size, data_size);
+
+ switch (decoder_state_) {
+ case kInitialized:
+ schedule_task = DecodeBufferInitial(data, decoded_size, &decoded_size);
+ break;
+ case kDecoding:
+ schedule_task = DecodeBufferContinue(data, decoded_size);
+ break;
+ default:
+ NOTIFY_ERROR(ILLEGAL_STATE);
+ return;
+ }
+ }
+ if (decoder_state_ == kError) {
+ // Failed during decode.
+ return;
+ }
+
+ if (schedule_task) {
+ decoder_current_bitstream_buffer_->bytes_used += decoded_size;
+ if ((shm ? shm->size() : 0) ==
+ decoder_current_bitstream_buffer_->bytes_used) {
+ // Our current bitstream buffer is done; return it.
+ int32_t input_id = decoder_current_bitstream_buffer_->input_id;
+ DVLOGF(4) << "finished input_id=" << input_id;
+ // BitstreamBufferRef destructor calls NotifyEndOfBitstreamBuffer().
+ decoder_current_bitstream_buffer_.reset();
+ }
+ ScheduleDecodeBufferTaskIfNeeded();
+ }
+}
+
+bool V4L2VideoDecodeAccelerator::AdvanceFrameFragment(const uint8_t* data,
+ size_t size,
+ size_t* endpos) {
+ if (video_profile_ >= H264PROFILE_MIN && video_profile_ <= H264PROFILE_MAX) {
+ // For H264, we need to feed HW one frame at a time. This is going to take
+ // some parsing of our input stream.
+ decoder_h264_parser_->SetStream(data, size);
+ H264NALU nalu;
+ H264Parser::Result result;
+ *endpos = 0;
+
+ // Keep on peeking the next NALs while they don't indicate a frame
+ // boundary.
+ for (;;) {
+ bool end_of_frame = false;
+ result = decoder_h264_parser_->AdvanceToNextNALU(&nalu);
+ if (result == H264Parser::kInvalidStream ||
+ result == H264Parser::kUnsupportedStream)
+ return false;
+ if (result == H264Parser::kEOStream) {
+ // We've reached the end of the buffer before finding a frame boundary.
+ decoder_partial_frame_pending_ = true;
+ *endpos = size;
+ return true;
+ }
+ switch (nalu.nal_unit_type) {
+ case H264NALU::kNonIDRSlice:
+ case H264NALU::kIDRSlice:
+ if (nalu.size < 1)
+ return false;
+ // For these two, if the "first_mb_in_slice" field is zero, start a
+ // new frame and return. This field is Exp-Golomb coded starting on
+ // the eighth data bit of the NAL; a zero value is encoded with a
+ // leading '1' bit in the byte, which we can detect as the byte being
+ // (unsigned) greater than or equal to 0x80.
+ if (nalu.data[1] >= 0x80) {
+ end_of_frame = true;
+ break;
+ }
+ break;
+ case H264NALU::kSEIMessage:
+ case H264NALU::kSPS:
+ case H264NALU::kPPS:
+ case H264NALU::kAUD:
+ case H264NALU::kEOSeq:
+ case H264NALU::kEOStream:
+ case H264NALU::kReserved14:
+ case H264NALU::kReserved15:
+ case H264NALU::kReserved16:
+ case H264NALU::kReserved17:
+ case H264NALU::kReserved18:
+ // These unconditionally signal a frame boundary.
+ end_of_frame = true;
+ break;
+ default:
+ // For all others, keep going.
+ break;
+ }
+ if (end_of_frame) {
+ if (!decoder_partial_frame_pending_ && *endpos == 0) {
+ // The frame was previously restarted, and we haven't filled the
+ // current frame with any contents yet. Start the new frame here and
+ // continue parsing NALs.
+ } else {
+ // The frame wasn't previously restarted and/or we have contents for
+ // the current frame; signal the start of a new frame here: we don't
+ // have a partial frame anymore.
+ decoder_partial_frame_pending_ = false;
+ return true;
+ }
+ }
+ *endpos = (nalu.data + nalu.size) - data;
+ }
+ NOTREACHED();
+ return false;
+ } else {
+ DCHECK_GE(video_profile_, VP8PROFILE_MIN);
+ DCHECK_LE(video_profile_, VP9PROFILE_MAX);
+ // For VP8/9, we can just dump the entire buffer. No fragmentation needed,
+ // and we never return a partial frame.
+ *endpos = size;
+ decoder_partial_frame_pending_ = false;
+ return true;
+ }
+}
+
+void V4L2VideoDecodeAccelerator::ScheduleDecodeBufferTaskIfNeeded() {
+ DCHECK(decoder_thread_.task_runner()->BelongsToCurrentThread());
+
+ // If we're behind on tasks, schedule another one.
+ int buffers_to_decode = decoder_input_queue_.size();
+ if (decoder_current_bitstream_buffer_ != NULL)
+ buffers_to_decode++;
+ if (decoder_decode_buffer_tasks_scheduled_ < buffers_to_decode) {
+ decoder_decode_buffer_tasks_scheduled_++;
+ decoder_thread_.task_runner()->PostTask(
+ FROM_HERE, base::Bind(&V4L2VideoDecodeAccelerator::DecodeBufferTask,
+ base::Unretained(this)));
+ }
+}
+
+bool V4L2VideoDecodeAccelerator::DecodeBufferInitial(const void* data,
+ size_t size,
+ size_t* endpos) {
+ DVLOGF(4) << "data=" << data << ", size=" << size;
+ DCHECK(decoder_thread_.task_runner()->BelongsToCurrentThread());
+ DCHECK_EQ(decoder_state_, kInitialized);
+ // Initial decode. We haven't been able to get output stream format info yet.
+ // Get it, and start decoding.
+
+ // Copy in and send to HW.
+ if (!AppendToInputFrame(data, size))
+ return false;
+
+ // If we only have a partial frame, don't flush and process yet.
+ if (decoder_partial_frame_pending_)
+ return true;
+
+ if (!FlushInputFrame())
+ return false;
+
+ // Recycle buffers.
+ Dequeue();
+
+ // Check and see if we have format info yet.
+ struct v4l2_format format;
+ Size visible_size;
+ bool again = false;
+ if (!GetFormatInfo(&format, &visible_size, &again))
+ return false;
+
+ *endpos = size;
+
+ if (again) {
+ // Need more stream to decode format, return true and schedule next buffer.
+ return true;
+ }
+
+ // Run this initialization only on first startup.
+ if (output_buffer_map_.empty()) {
+ DVLOGF(4) << "running initialization";
+ // Success! Setup our parameters.
+ if (!CreateBuffersForFormat(format, visible_size))
+ return false;
+ // We are waiting for AssignPictureBuffers. Do not set the state to
+ // kDecoding.
+ } else {
+ decoder_state_ = kDecoding;
+ ScheduleDecodeBufferTaskIfNeeded();
+ }
+ return true;
+}
+
+bool V4L2VideoDecodeAccelerator::DecodeBufferContinue(const void* data,
+ size_t size) {
+ DVLOGF(4) << "data=" << data << ", size=" << size;
+ DCHECK(decoder_thread_.task_runner()->BelongsToCurrentThread());
+ DCHECK_EQ(decoder_state_, kDecoding);
+
+ // Both of these calls will set kError state if they fail.
+ // Only flush the frame if it's complete.
+ return (AppendToInputFrame(data, size) &&
+ (decoder_partial_frame_pending_ || FlushInputFrame()));
+}
+
+bool V4L2VideoDecodeAccelerator::AppendToInputFrame(const void* data,
+ size_t size) {
+ DVLOGF(4);
+ DCHECK(decoder_thread_.task_runner()->BelongsToCurrentThread());
+ DCHECK_NE(decoder_state_, kUninitialized);
+ DCHECK_NE(decoder_state_, kResetting);
+ DCHECK_NE(decoder_state_, kError);
+ // This routine can handle data == NULL and size == 0, which occurs when
+ // we queue an empty buffer for the purposes of flushing the pipe.
+
+ // Flush if we're too big
+ if (decoder_current_input_buffer_ != -1) {
+ InputRecord& input_record =
+ input_buffer_map_[decoder_current_input_buffer_];
+ if (input_record.bytes_used + size > input_record.length) {
+ if (!FlushInputFrame())
+ return false;
+ decoder_current_input_buffer_ = -1;
+ }
+ }
+
+ // Try to get an available input buffer
+ if (decoder_current_input_buffer_ == -1) {
+ if (free_input_buffers_.empty()) {
+ // See if we can get more free buffers from HW
+ Dequeue();
+ if (free_input_buffers_.empty()) {
+ // Nope!
+ DVLOGF(3) << "stalled for input buffers";
+ return false;
+ }
+ }
+ decoder_current_input_buffer_ = free_input_buffers_.back();
+ free_input_buffers_.pop_back();
+ InputRecord& input_record =
+ input_buffer_map_[decoder_current_input_buffer_];
+ DCHECK_EQ(input_record.bytes_used, 0);
+ DCHECK_EQ(input_record.input_id, -1);
+ DCHECK(decoder_current_bitstream_buffer_ != NULL);
+ input_record.input_id = decoder_current_bitstream_buffer_->input_id;
+ }
+
+ DCHECK(data != NULL || size == 0);
+ if (size == 0) {
+ // If we asked for an empty buffer, return now. We return only after
+ // getting the next input buffer, since we might actually want an empty
+ // input buffer for flushing purposes.
+ return true;
+ }
+
+ // Copy in to the buffer.
+ InputRecord& input_record = input_buffer_map_[decoder_current_input_buffer_];
+ if (size > input_record.length - input_record.bytes_used) {
+ LOGF(ERROR) << "over-size frame, erroring";
+ NOTIFY_ERROR(UNREADABLE_INPUT);
+ return false;
+ }
+ memcpy(reinterpret_cast<uint8_t*>(input_record.address) +
+ input_record.bytes_used,
+ data, size);
+ input_record.bytes_used += size;
+
+ return true;
+}
+
+bool V4L2VideoDecodeAccelerator::FlushInputFrame() {
+ DVLOGF(4);
+ DCHECK(decoder_thread_.task_runner()->BelongsToCurrentThread());
+ DCHECK_NE(decoder_state_, kUninitialized);
+ DCHECK_NE(decoder_state_, kResetting);
+ DCHECK_NE(decoder_state_, kError);
+
+ if (decoder_current_input_buffer_ == -1)
+ return true;
+
+ InputRecord& input_record = input_buffer_map_[decoder_current_input_buffer_];
+ DCHECK_NE(input_record.input_id, -1);
+ DCHECK(input_record.input_id != kFlushBufferId ||
+ input_record.bytes_used == 0);
+ // * if input_id >= 0, this input buffer was prompted by a bitstream buffer we
+ // got from the client. We can skip it if it is empty.
+ // * if input_id < 0 (should be kFlushBufferId in this case), this input
+ // buffer was prompted by a flush buffer, and should be queued even when
+ // empty.
+ if (input_record.input_id >= 0 && input_record.bytes_used == 0) {
+ input_record.input_id = -1;
+ free_input_buffers_.push_back(decoder_current_input_buffer_);
+ decoder_current_input_buffer_ = -1;
+ return true;
+ }
+
+ // Queue it.
+ input_ready_queue_.push(decoder_current_input_buffer_);
+ decoder_current_input_buffer_ = -1;
+ DVLOGF(4) << "submitting input_id=" << input_record.input_id;
+ // Enqueue once since there's new available input for it.
+ Enqueue();
+
+ return (decoder_state_ != kError);
+}
+
+void V4L2VideoDecodeAccelerator::ServiceDeviceTask(bool event_pending) {
+ DVLOGF(3);
+ DCHECK(decoder_thread_.task_runner()->BelongsToCurrentThread());
+ DCHECK_NE(decoder_state_, kUninitialized);
+ TRACE_EVENT0("Video Decoder", "V4L2VDA::ServiceDeviceTask");
+
+ if (decoder_state_ == kResetting) {
+ DVLOGF(3) << "early out: kResetting state";
+ return;
+ } else if (decoder_state_ == kError) {
+ DVLOGF(3) << "early out: kError state";
+ return;
+ } else if (decoder_state_ == kChangingResolution) {
+ DVLOGF(3) << "early out: kChangingResolution state";
+ return;
+ }
+
+ bool resolution_change_pending = false;
+ if (event_pending)
+ resolution_change_pending = DequeueResolutionChangeEvent();
+ Dequeue();
+ Enqueue();
+
+ // Clear the interrupt fd.
+ if (!device_->ClearDevicePollInterrupt()) {
+ NOTIFY_ERROR(PLATFORM_FAILURE);
+ return;
+ }
+
+ bool poll_device = false;
+ // Add fd, if we should poll on it.
+ // Can be polled as soon as either input or output buffers are queued.
+ if (input_buffer_queued_count_ + output_buffer_queued_count_ > 0)
+ poll_device = true;
+
+ // ServiceDeviceTask() should only ever be scheduled from DevicePollTask(),
+ // so either:
+ // * device_poll_thread_ is running normally
+ // * device_poll_thread_ scheduled us, but then a ResetTask() or DestroyTask()
+ // shut it down, in which case we're either in kResetting or kError states
+ // respectively, and we should have early-outed already.
+ DCHECK(device_poll_thread_.message_loop());
+ // Queue the DevicePollTask() now.
+ device_poll_thread_.task_runner()->PostTask(
+ FROM_HERE, base::Bind(&V4L2VideoDecodeAccelerator::DevicePollTask,
+ base::Unretained(this), poll_device));
+
+ DVLOG(3) << "ServiceDeviceTask(): buffer counts: DEC["
+ << decoder_input_queue_.size() << "->"
+ << input_ready_queue_.size() << "] => DEVICE["
+ << free_input_buffers_.size() << "+"
+ << input_buffer_queued_count_ << "/"
+ << input_buffer_map_.size() << "->"
+ << free_output_buffers_.size() << "+"
+ << output_buffer_queued_count_ << "/"
+ << output_buffer_map_.size() << "] => CLIENT["
+ << decoder_frames_at_client_ << "]";
+
+ ScheduleDecodeBufferTaskIfNeeded();
+ if (resolution_change_pending)
+ StartResolutionChange();
+}
+
+void V4L2VideoDecodeAccelerator::Enqueue() {
+ DVLOGF(4);
+ DCHECK(decoder_thread_.task_runner()->BelongsToCurrentThread());
+ DCHECK_NE(decoder_state_, kUninitialized);
+ TRACE_EVENT0("Video Decoder", "V4L2VDA::Enqueue");
+
+ // Drain the pipe of completed decode buffers.
+ const int old_inputs_queued = input_buffer_queued_count_;
+ while (!input_ready_queue_.empty()) {
+ const int buffer = input_ready_queue_.front();
+ InputRecord& input_record = input_buffer_map_[buffer];
+ if (input_record.input_id == kFlushBufferId && decoder_cmd_supported_) {
+ // Send the flush command after all input buffers are dequeued. This makes
+ // sure all previous resolution changes have been handled because the
+ // driver must hold the input buffer that triggers resolution change. The
+ // driver cannot decode data in it without new output buffers. If we send
+ // the flush now and a queued input buffer triggers resolution change
+ // later, the driver will send an output buffer that has
+ // V4L2_BUF_FLAG_LAST. But some queued input buffer have not been decoded
+ // yet. Also, V4L2VDA calls STREAMOFF and STREAMON after resolution
+ // change. They implicitly send a V4L2_DEC_CMD_STOP and V4L2_DEC_CMD_START
+ // to the decoder.
+ if (input_buffer_queued_count_ == 0) {
+ if (!SendDecoderCmdStop())
+ return;
+ input_ready_queue_.pop();
+ free_input_buffers_.push_back(buffer);
+ input_record.input_id = -1;
+ } else {
+ break;
+ }
+ } else if (!EnqueueInputRecord())
+ return;
+ }
+ if (old_inputs_queued == 0 && input_buffer_queued_count_ != 0) {
+ // We just started up a previously empty queue.
+ // Queue state changed; signal interrupt.
+ if (!device_->SetDevicePollInterrupt()) {
+ PLOGF(ERROR) << "SetDevicePollInterrupt failed";
+ NOTIFY_ERROR(PLATFORM_FAILURE);
+ return;
+ }
+ // Start VIDIOC_STREAMON if we haven't yet.
+ if (!input_streamon_) {
+ __u32 type = V4L2_BUF_TYPE_VIDEO_OUTPUT_MPLANE;
+ IOCTL_OR_ERROR_RETURN(VIDIOC_STREAMON, &type);
+ input_streamon_ = true;
+ }
+ }
+
+ // Enqueue all the outputs we can.
+ const int old_outputs_queued = output_buffer_queued_count_;
+ while (!free_output_buffers_.empty()) {
+ if (!EnqueueOutputRecord())
+ return;
+ }
+ if (old_outputs_queued == 0 && output_buffer_queued_count_ != 0) {
+ // We just started up a previously empty queue.
+ // Queue state changed; signal interrupt.
+ if (!device_->SetDevicePollInterrupt()) {
+ PLOGF(ERROR) << "SetDevicePollInterrupt(): failed";
+ NOTIFY_ERROR(PLATFORM_FAILURE);
+ return;
+ }
+ // Start VIDIOC_STREAMON if we haven't yet.
+ if (!output_streamon_) {
+ __u32 type = V4L2_BUF_TYPE_VIDEO_CAPTURE_MPLANE;
+ IOCTL_OR_ERROR_RETURN(VIDIOC_STREAMON, &type);
+ output_streamon_ = true;
+ }
+ }
+}
+
+bool V4L2VideoDecodeAccelerator::DequeueResolutionChangeEvent() {
+ DCHECK(decoder_thread_.task_runner()->BelongsToCurrentThread());
+ DCHECK_NE(decoder_state_, kUninitialized);
+ DVLOGF(3);
+
+ struct v4l2_event ev;
+ memset(&ev, 0, sizeof(ev));
+
+ while (device_->Ioctl(VIDIOC_DQEVENT, &ev) == 0) {
+ if (ev.type == V4L2_EVENT_SOURCE_CHANGE) {
+ if (ev.u.src_change.changes & V4L2_EVENT_SRC_CH_RESOLUTION) {
+ VLOGF(2) << "got resolution change event.";
+ return true;
+ }
+ } else {
+ LOGF(ERROR) << "got an event (" << ev.type
+ << ") we haven't subscribed to.";
+ }
+ }
+ return false;
+}
+
+void V4L2VideoDecodeAccelerator::Dequeue() {
+ DVLOGF(4);
+ DCHECK(decoder_thread_.task_runner()->BelongsToCurrentThread());
+ DCHECK_NE(decoder_state_, kUninitialized);
+ TRACE_EVENT0("Video Decoder", "V4L2VDA::Dequeue");
+
+ while (input_buffer_queued_count_ > 0) {
+ if (!DequeueInputBuffer())
+ break;
+ }
+ while (output_buffer_queued_count_ > 0) {
+ if (!DequeueOutputBuffer())
+ break;
+ }
+ NotifyFlushDoneIfNeeded();
+}
+
+bool V4L2VideoDecodeAccelerator::DequeueInputBuffer() {
+ DCHECK(decoder_thread_.task_runner()->BelongsToCurrentThread());
+ DCHECK_GT(input_buffer_queued_count_, 0);
+ DCHECK(input_streamon_);
+
+ // Dequeue a completed input (VIDEO_OUTPUT) buffer, and recycle to the free
+ // list.
+ struct v4l2_buffer dqbuf;
+ struct v4l2_plane planes[1];
+ memset(&dqbuf, 0, sizeof(dqbuf));
+ memset(planes, 0, sizeof(planes));
+ dqbuf.type = V4L2_BUF_TYPE_VIDEO_OUTPUT_MPLANE;
+ dqbuf.memory = V4L2_MEMORY_MMAP;
+ dqbuf.m.planes = planes;
+ dqbuf.length = 1;
+ if (device_->Ioctl(VIDIOC_DQBUF, &dqbuf) != 0) {
+ if (errno == EAGAIN) {
+ // EAGAIN if we're just out of buffers to dequeue.
+ return false;
+ }
+ PLOGF(ERROR) << "ioctl() failed: VIDIOC_DQBUF";
+ NOTIFY_ERROR(PLATFORM_FAILURE);
+ return false;
+ }
+ InputRecord& input_record = input_buffer_map_[dqbuf.index];
+ DCHECK(input_record.at_device);
+ free_input_buffers_.push_back(dqbuf.index);
+ input_record.at_device = false;
+ input_record.bytes_used = 0;
+ input_record.input_id = -1;
+ input_buffer_queued_count_--;
+
+ return true;
+}
+
+bool V4L2VideoDecodeAccelerator::DequeueOutputBuffer() {
+ DCHECK(decoder_thread_.task_runner()->BelongsToCurrentThread());
+ DCHECK_GT(output_buffer_queued_count_, 0);
+ DCHECK(output_streamon_);
+
+ // Dequeue a completed output (VIDEO_CAPTURE) buffer, and queue to the
+ // completed queue.
+ struct v4l2_buffer dqbuf;
+ std::unique_ptr<struct v4l2_plane[]> planes(
+ new v4l2_plane[output_planes_count_]);
+ memset(&dqbuf, 0, sizeof(dqbuf));
+ memset(planes.get(), 0, sizeof(struct v4l2_plane) * output_planes_count_);
+ dqbuf.type = V4L2_BUF_TYPE_VIDEO_CAPTURE_MPLANE;
+ dqbuf.memory = V4L2_MEMORY_MMAP;
+ dqbuf.m.planes = planes.get();
+ dqbuf.length = output_planes_count_;
+ if (device_->Ioctl(VIDIOC_DQBUF, &dqbuf) != 0) {
+ if (errno == EAGAIN) {
+ // EAGAIN if we're just out of buffers to dequeue.
+ return false;
+ } else if (errno == EPIPE) {
+ DVLOGF(3) << "Got EPIPE. Last output buffer was already dequeued.";
+ return false;
+ }
+ PLOGF(ERROR) << "ioctl() failed: VIDIOC_DQBUF";
+ NOTIFY_ERROR(PLATFORM_FAILURE);
+ return false;
+ }
+ OutputRecord& output_record = output_buffer_map_[dqbuf.index];
+ DCHECK_EQ(output_record.state, kAtDevice);
+ DCHECK_NE(output_record.picture_id, -1);
+ output_buffer_queued_count_--;
+ if (dqbuf.m.planes[0].bytesused == 0) {
+ // This is an empty output buffer returned as part of a flush.
+ output_record.state = kFree;
+ free_output_buffers_.push_back(dqbuf.index);
+ } else {
+ int32_t bitstream_buffer_id = dqbuf.timestamp.tv_sec;
+ DCHECK_GE(bitstream_buffer_id, 0);
+ DVLOGF(4) << "Dequeue output buffer: dqbuf index=" << dqbuf.index
+ << " bitstream input_id=" << bitstream_buffer_id;
+ output_record.state = kAtClient;
+ decoder_frames_at_client_++;
+
+ const Picture picture(output_record.picture_id, bitstream_buffer_id,
+ Rect(visible_size_), false);
+ pending_picture_ready_.push(PictureRecord(output_record.cleared, picture));
+ SendPictureReady();
+ output_record.cleared = true;
+ }
+ if (dqbuf.flags & V4L2_BUF_FLAG_LAST) {
+ DVLOGF(3) << "Got last output buffer. Waiting last buffer="
+ << flush_awaiting_last_output_buffer_;
+ if (flush_awaiting_last_output_buffer_) {
+ flush_awaiting_last_output_buffer_ = false;
+ struct v4l2_decoder_cmd cmd;
+ memset(&cmd, 0, sizeof(cmd));
+ cmd.cmd = V4L2_DEC_CMD_START;
+ IOCTL_OR_ERROR_RETURN_FALSE(VIDIOC_DECODER_CMD, &cmd);
+ }
+ }
+ return true;
+}
+
+bool V4L2VideoDecodeAccelerator::EnqueueInputRecord() {
+ DVLOGF(4);
+ DCHECK(!input_ready_queue_.empty());
+
+ // Enqueue an input (VIDEO_OUTPUT) buffer.
+ const int buffer = input_ready_queue_.front();
+ InputRecord& input_record = input_buffer_map_[buffer];
+ DCHECK(!input_record.at_device);
+ struct v4l2_buffer qbuf;
+ struct v4l2_plane qbuf_plane;
+ memset(&qbuf, 0, sizeof(qbuf));
+ memset(&qbuf_plane, 0, sizeof(qbuf_plane));
+ qbuf.index = buffer;
+ qbuf.type = V4L2_BUF_TYPE_VIDEO_OUTPUT_MPLANE;
+ qbuf.timestamp.tv_sec = input_record.input_id;
+ qbuf.memory = V4L2_MEMORY_MMAP;
+ qbuf.m.planes = &qbuf_plane;
+ qbuf.m.planes[0].bytesused = input_record.bytes_used;
+ qbuf.length = 1;
+ IOCTL_OR_ERROR_RETURN_FALSE(VIDIOC_QBUF, &qbuf);
+ input_ready_queue_.pop();
+ input_record.at_device = true;
+ input_buffer_queued_count_++;
+ DVLOGF(4) << "enqueued input_id=" << input_record.input_id
+ << " size=" << input_record.bytes_used;
+ return true;
+}
+
+bool V4L2VideoDecodeAccelerator::EnqueueOutputRecord() {
+ DCHECK(!free_output_buffers_.empty());
+
+ // Enqueue an output (VIDEO_CAPTURE) buffer.
+ const int buffer = free_output_buffers_.front();
+ DVLOGF(4) << "buffer " << buffer;
+ OutputRecord& output_record = output_buffer_map_[buffer];
+ DCHECK_EQ(output_record.state, kFree);
+ DCHECK_NE(output_record.picture_id, -1);
+ struct v4l2_buffer qbuf;
+ std::unique_ptr<struct v4l2_plane[]> qbuf_planes(
+ new v4l2_plane[output_planes_count_]);
+ memset(&qbuf, 0, sizeof(qbuf));
+ memset(qbuf_planes.get(), 0,
+ sizeof(struct v4l2_plane) * output_planes_count_);
+ qbuf.index = buffer;
+ qbuf.type = V4L2_BUF_TYPE_VIDEO_CAPTURE_MPLANE;
+ qbuf.memory = V4L2_MEMORY_MMAP;
+ qbuf.m.planes = qbuf_planes.get();
+ qbuf.length = output_planes_count_;
+ DVLOGF(4) << "qbuf.index=" << qbuf.index;
+ IOCTL_OR_ERROR_RETURN_FALSE(VIDIOC_QBUF, &qbuf);
+ free_output_buffers_.pop_front();
+ output_record.state = kAtDevice;
+ output_buffer_queued_count_++;
+ return true;
+}
+
+void V4L2VideoDecodeAccelerator::ReusePictureBufferTask(int32_t picture_buffer_id) {
+ DVLOGF(4) << "picture_buffer_id=" << picture_buffer_id;
+ DCHECK(decoder_thread_.task_runner()->BelongsToCurrentThread());
+ TRACE_EVENT0("Video Decoder", "V4L2VDA::ReusePictureBufferTask");
+
+ // We run ReusePictureBufferTask even if we're in kResetting.
+ if (decoder_state_ == kError) {
+ DVLOGF(4) << "early out: kError state";
+ return;
+ }
+
+ if (decoder_state_ == kChangingResolution) {
+ DVLOGF(4) << "early out: kChangingResolution";
+ return;
+ }
+
+ size_t index;
+ for (index = 0; index < output_buffer_map_.size(); ++index)
+ if (output_buffer_map_[index].picture_id == picture_buffer_id)
+ break;
+
+ if (index >= output_buffer_map_.size()) {
+ // It's possible that we've already posted a DismissPictureBuffer for this
+ // picture, but it has not yet executed when this ReusePictureBuffer was
+ // posted to us by the client. In that case just ignore this (we've already
+ // dismissed it and accounted for that) and let the sync object get
+ // destroyed.
+ DVLOGF(4) << "got picture id= " << picture_buffer_id
+ << " not in use (anymore?).";
+ return;
+ }
+
+ OutputRecord& output_record = output_buffer_map_[index];
+ if (output_record.state != kAtClient) {
+ LOGF(ERROR) << "picture_buffer_id not reusable";
+ NOTIFY_ERROR(INVALID_ARGUMENT);
+ return;
+ }
+
+ output_record.state = kFree;
+ free_output_buffers_.push_back(index);
+ decoder_frames_at_client_--;
+ // We got a buffer back, so enqueue it back.
+ Enqueue();
+}
+
+void V4L2VideoDecodeAccelerator::FlushTask() {
+ VLOGF(2);
+ DCHECK(decoder_thread_.task_runner()->BelongsToCurrentThread());
+ TRACE_EVENT0("Video Decoder", "V4L2VDA::FlushTask");
+
+ // Flush outstanding buffers.
+ if (decoder_state_ == kInitialized) {
+ // There's nothing in the pipe, so return done immediately.
+ VLOGF(2) << "returning flush";
+ child_task_runner_->PostTask(FROM_HERE,
+ base::Bind(&Client::NotifyFlushDone, client_));
+ return;
+ } else if (decoder_state_ == kError) {
+ VLOGF(2) << "early out: kError state";
+ return;
+ }
+
+ // We don't support stacked flushing.
+ DCHECK(!decoder_flushing_);
+
+ // Queue up an empty buffer -- this triggers the flush.
+ decoder_input_queue_.push(
+ linked_ptr<BitstreamBufferRef>(new BitstreamBufferRef(
+ decode_client_, decode_task_runner_, nullptr, kFlushBufferId)));
+ decoder_flushing_ = true;
+ SendPictureReady(); // Send all pending PictureReady.
+
+ ScheduleDecodeBufferTaskIfNeeded();
+}
+
+void V4L2VideoDecodeAccelerator::NotifyFlushDoneIfNeeded() {
+ DCHECK(decoder_thread_.task_runner()->BelongsToCurrentThread());
+ if (!decoder_flushing_)
+ return;
+
+ // Pipeline is empty when:
+ // * Decoder input queue is empty of non-delayed buffers.
+ // * There is no currently filling input buffer.
+ // * Input holding queue is empty.
+ // * All input (VIDEO_OUTPUT) buffers are returned.
+ // * All image processor buffers are returned.
+ if (!decoder_input_queue_.empty()) {
+ if (decoder_input_queue_.front()->input_id !=
+ decoder_delay_bitstream_buffer_id_) {
+ DVLOGF(3) << "Some input bitstream buffers are not queued.";
+ return;
+ }
+ }
+ if (decoder_current_input_buffer_ != -1) {
+ DVLOGF(3) << "Current input buffer != -1";
+ return;
+ }
+ if ((input_ready_queue_.size() + input_buffer_queued_count_) != 0) {
+ DVLOGF(3) << "Some input buffers are not dequeued.";
+ return;
+ }
+ if (flush_awaiting_last_output_buffer_) {
+ DVLOGF(3) << "Waiting for last output buffer.";
+ return;
+ }
+
+ // TODO(posciak): crbug.com/270039. Exynos requires a streamoff-streamon
+ // sequence after flush to continue, even if we are not resetting. This would
+ // make sense, because we don't really want to resume from a non-resume point
+ // (e.g. not from an IDR) if we are flushed.
+ // MSE player however triggers a Flush() on chunk end, but never Reset(). One
+ // could argue either way, or even say that Flush() is not needed/harmful when
+ // transitioning to next chunk.
+ // For now, do the streamoff-streamon cycle to satisfy Exynos and not freeze
+ // when doing MSE. This should be harmless otherwise.
+ if (!(StopDevicePoll() && StopOutputStream() && StopInputStream()))
+ return;
+
+ if (!StartDevicePoll())
+ return;
+
+ decoder_delay_bitstream_buffer_id_ = -1;
+ decoder_flushing_ = false;
+ VLOGF(2) << "returning flush";
+ child_task_runner_->PostTask(FROM_HERE,
+ base::Bind(&Client::NotifyFlushDone, client_));
+
+ // While we were flushing, we early-outed DecodeBufferTask()s.
+ ScheduleDecodeBufferTaskIfNeeded();
+}
+
+bool V4L2VideoDecodeAccelerator::IsDecoderCmdSupported() {
+ // CMD_STOP should always succeed. If the decoder is started, the command can
+ // flush it. If the decoder is stopped, the command does nothing. We use this
+ // to know if a driver supports V4L2_DEC_CMD_STOP to flush.
+ struct v4l2_decoder_cmd cmd;
+ memset(&cmd, 0, sizeof(cmd));
+ cmd.cmd = V4L2_DEC_CMD_STOP;
+ if (device_->Ioctl(VIDIOC_TRY_DECODER_CMD, &cmd) != 0) {
+ DVLOGF(3) "V4L2_DEC_CMD_STOP is not supported.";
+ return false;
+ }
+
+ return true;
+}
+
+bool V4L2VideoDecodeAccelerator::SendDecoderCmdStop() {
+ VLOGF(2);
+ DCHECK(decoder_thread_.task_runner()->BelongsToCurrentThread());
+ DCHECK(!flush_awaiting_last_output_buffer_);
+
+ struct v4l2_decoder_cmd cmd;
+ memset(&cmd, 0, sizeof(cmd));
+ cmd.cmd = V4L2_DEC_CMD_STOP;
+ IOCTL_OR_ERROR_RETURN_FALSE(VIDIOC_DECODER_CMD, &cmd);
+ flush_awaiting_last_output_buffer_ = true;
+
+ return true;
+}
+
+void V4L2VideoDecodeAccelerator::ResetTask() {
+ VLOGF(2);
+ DCHECK(decoder_thread_.task_runner()->BelongsToCurrentThread());
+ TRACE_EVENT0("Video Decoder", "V4L2VDA::ResetTask");
+
+ if (decoder_state_ == kError) {
+ VLOGF(2) << "early out: kError state";
+ return;
+ }
+ decoder_current_bitstream_buffer_.reset();
+ while (!decoder_input_queue_.empty())
+ decoder_input_queue_.pop();
+
+ decoder_current_input_buffer_ = -1;
+
+ // If we are in the middle of switching resolutions or awaiting picture
+ // buffers, postpone reset until it's done. We don't have to worry about
+ // timing of this wrt to decoding, because output pipe is already
+ // stopped if we are changing resolution. We will come back here after
+ // we are done.
+ DCHECK(!reset_pending_);
+ if (decoder_state_ == kChangingResolution ||
+ decoder_state_ == kAwaitingPictureBuffers) {
+ reset_pending_ = true;
+ return;
+ }
+ FinishReset();
+}
+
+void V4L2VideoDecodeAccelerator::FinishReset() {
+ VLOGF(2);
+ DCHECK(decoder_thread_.task_runner()->BelongsToCurrentThread());
+
+ reset_pending_ = false;
+ // After the output stream is stopped, the codec should not post any
+ // resolution change events. So we dequeue the resolution change event
+ // afterwards. The event could be posted before or while stopping the output
+ // stream. The codec will expect the buffer of new size after the seek, so
+ // we need to handle the resolution change event first.
+ if (!(StopDevicePoll() && StopOutputStream()))
+ return;
+
+ if (DequeueResolutionChangeEvent()) {
+ reset_pending_ = true;
+ StartResolutionChange();
+ return;
+ }
+
+ if (!StopInputStream())
+ return;
+
+ // If we were flushing, we'll never return any more BitstreamBuffers or
+ // PictureBuffers; they have all been dropped and returned by now.
+ NotifyFlushDoneIfNeeded();
+
+ // Mark that we're resetting, then enqueue a ResetDoneTask(). All intervening
+ // jobs will early-out in the kResetting state.
+ decoder_state_ = kResetting;
+ SendPictureReady(); // Send all pending PictureReady.
+ decoder_thread_.task_runner()->PostTask(
+ FROM_HERE, base::Bind(&V4L2VideoDecodeAccelerator::ResetDoneTask,
+ base::Unretained(this)));
+}
+
+void V4L2VideoDecodeAccelerator::ResetDoneTask() {
+ VLOGF(2);
+ DCHECK(decoder_thread_.task_runner()->BelongsToCurrentThread());
+ TRACE_EVENT0("Video Decoder", "V4L2VDA::ResetDoneTask");
+
+ if (decoder_state_ == kError) {
+ VLOGF(2) << "early out: kError state";
+ return;
+ }
+
+ // Start poll thread if NotifyFlushDoneIfNeeded has not already.
+ if (!device_poll_thread_.IsRunning()) {
+ if (!StartDevicePoll())
+ return;
+ }
+
+ // Reset format-specific bits.
+ if (video_profile_ >= H264PROFILE_MIN && video_profile_ <= H264PROFILE_MAX) {
+ decoder_h264_parser_.reset(new H264Parser());
+ }
+
+ // Jobs drained, we're finished resetting.
+ DCHECK_EQ(decoder_state_, kResetting);
+ decoder_state_ = kInitialized;
+
+ decoder_partial_frame_pending_ = false;
+ decoder_delay_bitstream_buffer_id_ = -1;
+ child_task_runner_->PostTask(FROM_HERE,
+ base::Bind(&Client::NotifyResetDone, client_));
+
+ // While we were resetting, we early-outed DecodeBufferTask()s.
+ ScheduleDecodeBufferTaskIfNeeded();
+}
+
+void V4L2VideoDecodeAccelerator::DestroyTask() {
+ VLOGF(2);
+ TRACE_EVENT0("Video Decoder", "V4L2VDA::DestroyTask");
+
+ // DestroyTask() should run regardless of decoder_state_.
+
+ StopDevicePoll();
+ StopOutputStream();
+ StopInputStream();
+
+ decoder_current_bitstream_buffer_.reset();
+ decoder_current_input_buffer_ = -1;
+ decoder_decode_buffer_tasks_scheduled_ = 0;
+ decoder_frames_at_client_ = 0;
+ while (!decoder_input_queue_.empty())
+ decoder_input_queue_.pop();
+ decoder_flushing_ = false;
+
+ // Set our state to kError. Just in case.
+ decoder_state_ = kError;
+
+ DestroyInputBuffers();
+ DestroyOutputBuffers();
+}
+
+bool V4L2VideoDecodeAccelerator::StartDevicePoll() {
+ DVLOGF(3);
+ DCHECK(!device_poll_thread_.IsRunning());
+ DCHECK(decoder_thread_.task_runner()->BelongsToCurrentThread());
+
+ // Start up the device poll thread and schedule its first DevicePollTask().
+ if (!device_poll_thread_.Start()) {
+ LOGF(ERROR) << "Device thread failed to start";
+ NOTIFY_ERROR(PLATFORM_FAILURE);
+ return false;
+ }
+ device_poll_thread_.task_runner()->PostTask(
+ FROM_HERE, base::Bind(&V4L2VideoDecodeAccelerator::DevicePollTask,
+ base::Unretained(this), 0));
+
+ return true;
+}
+
+bool V4L2VideoDecodeAccelerator::StopDevicePoll() {
+ DVLOGF(3);
+
+ if (!device_poll_thread_.IsRunning())
+ return true;
+
+ if (decoder_thread_.IsRunning())
+ DCHECK(decoder_thread_.task_runner()->BelongsToCurrentThread());
+
+ // Signal the DevicePollTask() to stop, and stop the device poll thread.
+ if (!device_->SetDevicePollInterrupt()) {
+ PLOGF(ERROR) << "SetDevicePollInterrupt(): failed";
+ NOTIFY_ERROR(PLATFORM_FAILURE);
+ return false;
+ }
+ device_poll_thread_.Stop();
+ // Clear the interrupt now, to be sure.
+ if (!device_->ClearDevicePollInterrupt()) {
+ NOTIFY_ERROR(PLATFORM_FAILURE);
+ return false;
+ }
+ DVLOGF(3) << "device poll stopped";
+ return true;
+}
+
+bool V4L2VideoDecodeAccelerator::StopOutputStream() {
+ VLOGF(2);
+ if (!output_streamon_)
+ return true;
+
+ __u32 type = V4L2_BUF_TYPE_VIDEO_CAPTURE_MPLANE;
+ IOCTL_OR_ERROR_RETURN_FALSE(VIDIOC_STREAMOFF, &type);
+ output_streamon_ = false;
+
+ // Output stream is stopped. No need to wait for the buffer anymore.
+ flush_awaiting_last_output_buffer_ = false;
+
+ for (size_t i = 0; i < output_buffer_map_.size(); ++i) {
+ // After streamoff, the device drops ownership of all buffers, even if we
+ // don't dequeue them explicitly. Some of them may still be owned by the
+ // client however. Reuse only those that aren't.
+ OutputRecord& output_record = output_buffer_map_[i];
+ if (output_record.state == kAtDevice) {
+ output_record.state = kFree;
+ free_output_buffers_.push_back(i);
+ }
+ }
+ output_buffer_queued_count_ = 0;
+ return true;
+}
+
+bool V4L2VideoDecodeAccelerator::StopInputStream() {
+ VLOGF(2);
+ if (!input_streamon_)
+ return true;
+
+ __u32 type = V4L2_BUF_TYPE_VIDEO_OUTPUT_MPLANE;
+ IOCTL_OR_ERROR_RETURN_FALSE(VIDIOC_STREAMOFF, &type);
+ input_streamon_ = false;
+
+ // Reset accounting info for input.
+ while (!input_ready_queue_.empty())
+ input_ready_queue_.pop();
+ free_input_buffers_.clear();
+ for (size_t i = 0; i < input_buffer_map_.size(); ++i) {
+ free_input_buffers_.push_back(i);
+ input_buffer_map_[i].at_device = false;
+ input_buffer_map_[i].bytes_used = 0;
+ input_buffer_map_[i].input_id = -1;
+ }
+ input_buffer_queued_count_ = 0;
+
+ return true;
+}
+
+void V4L2VideoDecodeAccelerator::StartResolutionChange() {
+ DCHECK(decoder_thread_.task_runner()->BelongsToCurrentThread());
+ DCHECK_NE(decoder_state_, kUninitialized);
+ DCHECK_NE(decoder_state_, kResetting);
+
+ VLOGF(2) << "Initiate resolution change";
+
+ if (!(StopDevicePoll() && StopOutputStream()))
+ return;
+
+ decoder_state_ = kChangingResolution;
+ SendPictureReady(); // Send all pending PictureReady.
+
+ if (!DestroyOutputBuffers()) {
+ LOGF(ERROR) << "Failed destroying output buffers.";
+ NOTIFY_ERROR(PLATFORM_FAILURE);
+ return;
+ }
+
+ FinishResolutionChange();
+}
+
+void V4L2VideoDecodeAccelerator::FinishResolutionChange() {
+ DCHECK(decoder_thread_.task_runner()->BelongsToCurrentThread());
+ DCHECK_EQ(decoder_state_, kChangingResolution);
+ VLOGF(2);
+
+ if (decoder_state_ == kError) {
+ VLOGF(2) << "early out: kError state";
+ return;
+ }
+
+ struct v4l2_format format;
+ bool again;
+ Size visible_size;
+ bool ret = GetFormatInfo(&format, &visible_size, &again);
+ if (!ret || again) {
+ LOGF(ERROR) << "Couldn't get format information after resolution change";
+ NOTIFY_ERROR(PLATFORM_FAILURE);
+ return;
+ }
+
+ if (!CreateBuffersForFormat(format, visible_size)) {
+ LOGF(ERROR) << "Couldn't reallocate buffers after resolution change";
+ NOTIFY_ERROR(PLATFORM_FAILURE);
+ return;
+ }
+
+ if (!StartDevicePoll())
+ return;
+}
+
+void V4L2VideoDecodeAccelerator::DevicePollTask(bool poll_device) {
+ DVLOGF(4);
+ DCHECK(device_poll_thread_.task_runner()->BelongsToCurrentThread());
+ TRACE_EVENT0("Video Decoder", "V4L2VDA::DevicePollTask");
+
+ bool event_pending = false;
+
+ if (!device_->Poll(poll_device, &event_pending)) {
+ NOTIFY_ERROR(PLATFORM_FAILURE);
+ return;
+ }
+
+ // All processing should happen on ServiceDeviceTask(), since we shouldn't
+ // touch decoder state from this thread.
+ decoder_thread_.task_runner()->PostTask(
+ FROM_HERE, base::Bind(&V4L2VideoDecodeAccelerator::ServiceDeviceTask,
+ base::Unretained(this), event_pending));
+}
+
+void V4L2VideoDecodeAccelerator::NotifyError(Error error) {
+ VLOGF(2);
+
+ if (!child_task_runner_->BelongsToCurrentThread()) {
+ child_task_runner_->PostTask(
+ FROM_HERE, base::Bind(&V4L2VideoDecodeAccelerator::NotifyError,
+ weak_this_, error));
+ return;
+ }
+
+ if (client_) {
+ client_->NotifyError(error);
+ client_ptr_factory_.reset();
+ }
+}
+
+void V4L2VideoDecodeAccelerator::SetErrorState(Error error) {
+ // We can touch decoder_state_ only if this is the decoder thread or the
+ // decoder thread isn't running.
+ if (decoder_thread_.task_runner() &&
+ !decoder_thread_.task_runner()->BelongsToCurrentThread()) {
+ decoder_thread_.task_runner()->PostTask(
+ FROM_HERE, base::Bind(&V4L2VideoDecodeAccelerator::SetErrorState,
+ base::Unretained(this), error));
+ return;
+ }
+
+ // Post NotifyError only if we are already initialized, as the API does
+ // not allow doing so before that.
+ if (decoder_state_ != kError && decoder_state_ != kUninitialized)
+ NotifyError(error);
+
+ decoder_state_ = kError;
+}
+
+bool V4L2VideoDecodeAccelerator::GetFormatInfo(struct v4l2_format* format,
+ Size* visible_size,
+ bool* again) {
+ DCHECK(decoder_thread_.task_runner()->BelongsToCurrentThread());
+
+ *again = false;
+ memset(format, 0, sizeof(*format));
+ format->type = V4L2_BUF_TYPE_VIDEO_CAPTURE_MPLANE;
+ if (device_->Ioctl(VIDIOC_G_FMT, format) != 0) {
+ if (errno == EINVAL) {
+ // EINVAL means we haven't seen sufficient stream to decode the format.
+ *again = true;
+ return true;
+ } else {
+ PLOGF(ERROR) << "ioctl() failed: VIDIOC_G_FMT";
+ NOTIFY_ERROR(PLATFORM_FAILURE);
+ return false;
+ }
+ }
+
+ // Make sure we are still getting the format we set on initialization.
+ if (format->fmt.pix_mp.pixelformat != output_format_fourcc_) {
+ LOGF(ERROR) << "Unexpected format from G_FMT on output";
+ return false;
+ }
+
+ Size coded_size(format->fmt.pix_mp.width, format->fmt.pix_mp.height);
+ if (visible_size != nullptr)
+ *visible_size = GetVisibleSize(coded_size);
+
+ return true;
+}
+
+bool V4L2VideoDecodeAccelerator::CreateBuffersForFormat(
+ const struct v4l2_format& format,
+ const Size& visible_size) {
+ DCHECK(decoder_thread_.task_runner()->BelongsToCurrentThread());
+ output_planes_count_ = format.fmt.pix_mp.num_planes;
+ coded_size_.SetSize(format.fmt.pix_mp.width, format.fmt.pix_mp.height);
+ visible_size_ = visible_size;
+
+ VLOGF(2) << "new resolution: " << coded_size_.ToString()
+ << ", visible size: " << visible_size_.ToString()
+ << ", decoder output planes count: " << output_planes_count_;
+
+ return CreateOutputBuffers();
+}
+
+Size V4L2VideoDecodeAccelerator::GetVisibleSize(
+ const Size& coded_size) {
+ DCHECK(decoder_thread_.task_runner()->BelongsToCurrentThread());
+
+ struct v4l2_rect* visible_rect;
+ struct v4l2_selection selection_arg;
+ memset(&selection_arg, 0, sizeof(selection_arg));
+ selection_arg.type = V4L2_BUF_TYPE_VIDEO_CAPTURE;
+ selection_arg.target = V4L2_SEL_TGT_COMPOSE;
+
+ if (device_->Ioctl(VIDIOC_G_SELECTION, &selection_arg) == 0) {
+ DVLOGF(2) << "VIDIOC_G_SELECTION is supported";
+ visible_rect = &selection_arg.r;
+ } else {
+ DVLOGF(2) << "Fallback to VIDIOC_G_CROP";
+ struct v4l2_crop crop_arg;
+ memset(&crop_arg, 0, sizeof(crop_arg));
+ crop_arg.type = V4L2_BUF_TYPE_VIDEO_CAPTURE_MPLANE;
+
+ if (device_->Ioctl(VIDIOC_G_CROP, &crop_arg) != 0) {
+ PLOGF(ERROR) << "ioctl() VIDIOC_G_CROP failed";
+ return coded_size;
+ }
+ visible_rect = &crop_arg.c;
+ }
+
+ Rect rect(visible_rect->left, visible_rect->top, visible_rect->width,
+ visible_rect->height);
+ VLOGF(2) << "visible rectangle is " << rect.ToString();
+ if (!Rect(coded_size).Contains(rect)) {
+ DLOGF(ERROR) << "visible rectangle " << rect.ToString()
+ << " is not inside coded size " << coded_size.ToString();
+ return coded_size;
+ }
+ if (rect.IsEmpty()) {
+ DLOGF(ERROR) << "visible size is empty";
+ return coded_size;
+ }
+
+ // Chrome assume picture frame is coded at (0, 0).
+ if (rect.x() != 0 || rect.y() != 0) {
+ DLOGF(ERROR) << "Unexpected visible rectangle " << rect.ToString()
+ << ", top-left is not origin";
+ return coded_size;
+ }
+
+ return rect.size();
+}
+
+bool V4L2VideoDecodeAccelerator::CreateInputBuffers() {
+ VLOGF(2);
+ DCHECK(decoder_thread_.task_runner()->BelongsToCurrentThread());
+ // We always run this as we prepare to initialize.
+ DCHECK_EQ(decoder_state_, kInitialized);
+ DCHECK(!input_streamon_);
+ DCHECK(input_buffer_map_.empty());
+
+ struct v4l2_requestbuffers reqbufs;
+ memset(&reqbufs, 0, sizeof(reqbufs));
+ reqbufs.count = kInputBufferCount;
+ reqbufs.type = V4L2_BUF_TYPE_VIDEO_OUTPUT_MPLANE;
+ reqbufs.memory = V4L2_MEMORY_MMAP;
+ IOCTL_OR_ERROR_RETURN_FALSE(VIDIOC_REQBUFS, &reqbufs);
+ input_buffer_map_.resize(reqbufs.count);
+ for (size_t i = 0; i < input_buffer_map_.size(); ++i) {
+ free_input_buffers_.push_back(i);
+
+ // Query for the MEMORY_MMAP pointer.
+ struct v4l2_plane planes[1];
+ struct v4l2_buffer buffer;
+ memset(&buffer, 0, sizeof(buffer));
+ memset(planes, 0, sizeof(planes));
+ buffer.index = i;
+ buffer.type = V4L2_BUF_TYPE_VIDEO_OUTPUT_MPLANE;
+ buffer.memory = V4L2_MEMORY_MMAP;
+ buffer.m.planes = planes;
+ buffer.length = 1;
+ IOCTL_OR_ERROR_RETURN_FALSE(VIDIOC_QUERYBUF, &buffer);
+ void* address = device_->Mmap(NULL,
+ buffer.m.planes[0].length,
+ PROT_READ | PROT_WRITE,
+ MAP_SHARED,
+ buffer.m.planes[0].m.mem_offset);
+ if (address == MAP_FAILED) {
+ PLOGF(ERROR) << "mmap() failed";
+ return false;
+ }
+ input_buffer_map_[i].address = address;
+ input_buffer_map_[i].length = buffer.m.planes[0].length;
+ }
+
+ return true;
+}
+
+static bool IsSupportedOutputFormat(uint32_t v4l2_format) {
+ // Only support V4L2_PIX_FMT_NV12 output format for now.
+ // TODO(johnylin): add more supported format if necessary.
+ uint32_t kSupportedOutputFmtFourcc[] = { V4L2_PIX_FMT_NV12 };
+ return std::find(
+ kSupportedOutputFmtFourcc,
+ kSupportedOutputFmtFourcc + arraysize(kSupportedOutputFmtFourcc),
+ v4l2_format) !=
+ kSupportedOutputFmtFourcc + arraysize(kSupportedOutputFmtFourcc);
+}
+
+bool V4L2VideoDecodeAccelerator::SetupFormats() {
+ // We always run this as we prepare to initialize.
+ DCHECK(child_task_runner_->BelongsToCurrentThread());
+ DCHECK_EQ(decoder_state_, kUninitialized);
+ DCHECK(!input_streamon_);
+ DCHECK(!output_streamon_);
+
+ size_t input_size;
+ Size max_resolution, min_resolution;
+ device_->GetSupportedResolution(input_format_fourcc_, &min_resolution,
+ &max_resolution);
+ if (max_resolution.width() > 1920 && max_resolution.height() > 1088)
+ input_size = kInputBufferMaxSizeFor4k;
+ else
+ input_size = kInputBufferMaxSizeFor1080p;
+
+ struct v4l2_fmtdesc fmtdesc;
+ memset(&fmtdesc, 0, sizeof(fmtdesc));
+ fmtdesc.type = V4L2_BUF_TYPE_VIDEO_OUTPUT_MPLANE;
+ bool is_format_supported = false;
+ while (device_->Ioctl(VIDIOC_ENUM_FMT, &fmtdesc) == 0) {
+ if (fmtdesc.pixelformat == input_format_fourcc_) {
+ is_format_supported = true;
+ break;
+ }
+ ++fmtdesc.index;
+ }
+
+ if (!is_format_supported) {
+ VLOGF(1) << "Input fourcc " << input_format_fourcc_
+ << " not supported by device.";
+ return false;
+ }
+
+ struct v4l2_format format;
+ memset(&format, 0, sizeof(format));
+ format.type = V4L2_BUF_TYPE_VIDEO_OUTPUT_MPLANE;
+ format.fmt.pix_mp.pixelformat = input_format_fourcc_;
+ format.fmt.pix_mp.plane_fmt[0].sizeimage = input_size;
+ format.fmt.pix_mp.num_planes = 1;
+ IOCTL_OR_ERROR_RETURN_FALSE(VIDIOC_S_FMT, &format);
+
+ // We have to set up the format for output, because the driver may not allow
+ // changing it once we start streaming; whether it can support our chosen
+ // output format or not may depend on the input format.
+ memset(&fmtdesc, 0, sizeof(fmtdesc));
+ fmtdesc.type = V4L2_BUF_TYPE_VIDEO_CAPTURE_MPLANE;
+ while (device_->Ioctl(VIDIOC_ENUM_FMT, &fmtdesc) == 0) {
+ if (IsSupportedOutputFormat(fmtdesc.pixelformat)) {
+ output_format_fourcc_ = fmtdesc.pixelformat;
+ break;
+ }
+ ++fmtdesc.index;
+ }
+
+ if (output_format_fourcc_ == 0) {
+ VLOGF(1) << "Image processor not available";
+ return false;
+ }
+ VLOGF(2) << "Output format=" << output_format_fourcc_;
+
+ // Just set the fourcc for output; resolution, etc., will come from the
+ // driver once it extracts it from the stream.
+ memset(&format, 0, sizeof(format));
+ format.type = V4L2_BUF_TYPE_VIDEO_CAPTURE_MPLANE;
+ format.fmt.pix_mp.pixelformat = output_format_fourcc_;
+ IOCTL_OR_ERROR_RETURN_FALSE(VIDIOC_S_FMT, &format);
+
+ return true;
+}
+
+bool V4L2VideoDecodeAccelerator::CreateOutputBuffers() {
+ VLOGF(2);
+ DCHECK(decoder_state_ == kInitialized ||
+ decoder_state_ == kChangingResolution);
+ DCHECK(!output_streamon_);
+ DCHECK(output_buffer_map_.empty());
+ DCHECK_EQ(output_mode_, Config::OutputMode::IMPORT);
+
+ // Number of output buffers we need.
+ struct v4l2_control ctrl;
+ memset(&ctrl, 0, sizeof(ctrl));
+ ctrl.id = V4L2_CID_MIN_BUFFERS_FOR_CAPTURE;
+ IOCTL_OR_ERROR_RETURN_FALSE(VIDIOC_G_CTRL, &ctrl);
+ output_dpb_size_ = ctrl.value;
+
+ // Output format setup in Initialize().
+
+ uint32_t buffer_count = output_dpb_size_ + kDpbOutputBufferExtraCount;
+
+ VideoPixelFormat pixel_format =
+ V4L2Device::V4L2PixFmtToVideoPixelFormat(output_format_fourcc_);
+
+ child_task_runner_->PostTask(
+ FROM_HERE, base::Bind(&Client::ProvidePictureBuffers, client_,
+ buffer_count, pixel_format, coded_size_));
+
+
+ // Go into kAwaitingPictureBuffers to prevent us from doing any more decoding
+ // or event handling while we are waiting for AssignPictureBuffers(). Not
+ // having Pictures available would not have prevented us from making decoding
+ // progress entirely e.g. in the case of H.264 where we could further decode
+ // non-slice NALUs and could even get another resolution change before we were
+ // done with this one. After we get the buffers, we'll go back into kIdle and
+ // kick off further event processing, and eventually go back into kDecoding
+ // once no more events are pending (if any).
+ decoder_state_ = kAwaitingPictureBuffers;
+
+ return true;
+}
+
+void V4L2VideoDecodeAccelerator::DestroyInputBuffers() {
+ VLOGF(2);
+ DCHECK(!decoder_thread_.IsRunning() ||
+ decoder_thread_.task_runner()->BelongsToCurrentThread());
+ DCHECK(!input_streamon_);
+
+ if (input_buffer_map_.empty())
+ return;
+
+ for (size_t i = 0; i < input_buffer_map_.size(); ++i) {
+ if (input_buffer_map_[i].address != NULL) {
+ device_->Munmap(input_buffer_map_[i].address,
+ input_buffer_map_[i].length);
+ }
+ }
+
+ struct v4l2_requestbuffers reqbufs;
+ memset(&reqbufs, 0, sizeof(reqbufs));
+ reqbufs.count = 0;
+ reqbufs.type = V4L2_BUF_TYPE_VIDEO_OUTPUT_MPLANE;
+ reqbufs.memory = V4L2_MEMORY_MMAP;
+ IOCTL_OR_LOG_ERROR(VIDIOC_REQBUFS, &reqbufs);
+
+ input_buffer_map_.clear();
+ free_input_buffers_.clear();
+}
+
+bool V4L2VideoDecodeAccelerator::DestroyOutputBuffers() {
+ VLOGF(2);
+ DCHECK(!decoder_thread_.IsRunning() ||
+ decoder_thread_.task_runner()->BelongsToCurrentThread());
+ DCHECK(!output_streamon_);
+ bool success = true;
+
+ if (output_buffer_map_.empty())
+ return true;
+
+ for (size_t i = 0; i < output_buffer_map_.size(); ++i) {
+ OutputRecord& output_record = output_buffer_map_[i];
+
+ DVLOGF(3) << "dismissing PictureBuffer id=" << output_record.picture_id;
+ child_task_runner_->PostTask(
+ FROM_HERE, base::Bind(&Client::DismissPictureBuffer, client_,
+ output_record.picture_id));
+ }
+
+ struct v4l2_requestbuffers reqbufs;
+ memset(&reqbufs, 0, sizeof(reqbufs));
+ reqbufs.count = 0;
+ reqbufs.type = V4L2_BUF_TYPE_VIDEO_CAPTURE_MPLANE;
+ reqbufs.memory = V4L2_MEMORY_MMAP;
+ if (device_->Ioctl(VIDIOC_REQBUFS, &reqbufs) != 0) {
+ PLOGF(ERROR) << "ioctl() failed: VIDIOC_REQBUFS";
+ NOTIFY_ERROR(PLATFORM_FAILURE);
+ success = false;
+ }
+
+ output_buffer_map_.clear();
+ while (!free_output_buffers_.empty())
+ free_output_buffers_.pop_front();
+ output_buffer_queued_count_ = 0;
+ // The client may still hold some buffers. The texture holds a reference to
+ // the buffer. It is OK to free the buffer and destroy EGLImage here.
+ decoder_frames_at_client_ = 0;
+
+ return success;
+}
+
+void V4L2VideoDecodeAccelerator::SendPictureReady() {
+ DVLOGF(4);
+ DCHECK(decoder_thread_.task_runner()->BelongsToCurrentThread());
+ bool send_now = (decoder_state_ == kChangingResolution ||
+ decoder_state_ == kResetting || decoder_flushing_);
+ while (pending_picture_ready_.size() > 0) {
+ bool cleared = pending_picture_ready_.front().cleared;
+ const Picture& picture = pending_picture_ready_.front().picture;
+ if (cleared && picture_clearing_count_ == 0) {
+ // This picture is cleared. It can be posted to a thread different than
+ // the main GPU thread to reduce latency. This should be the case after
+ // all pictures are cleared at the beginning.
+ decode_task_runner_->PostTask(
+ FROM_HERE,
+ base::Bind(&Client::PictureReady, decode_client_, picture));
+ pending_picture_ready_.pop();
+ } else if (!cleared || send_now) {
+ DVLOGF(4) << "cleared=" << pending_picture_ready_.front().cleared
+ << ", decoder_state_=" << decoder_state_
+ << ", decoder_flushing_=" << decoder_flushing_
+ << ", picture_clearing_count_=" << picture_clearing_count_;
+ // If the picture is not cleared, post it to the child thread because it
+ // has to be cleared in the child thread. A picture only needs to be
+ // cleared once. If the decoder is changing resolution, resetting or
+ // flushing, send all pictures to ensure PictureReady arrive before
+ // ProvidePictureBuffers, NotifyResetDone, or NotifyFlushDone.
+ child_task_runner_->PostTaskAndReply(
+ FROM_HERE, base::Bind(&Client::PictureReady, client_, picture),
+ // Unretained is safe. If Client::PictureReady gets to run, |this| is
+ // alive. Destroy() will wait the decode thread to finish.
+ base::Bind(&V4L2VideoDecodeAccelerator::PictureCleared,
+ base::Unretained(this)));
+ picture_clearing_count_++;
+ pending_picture_ready_.pop();
+ } else {
+ // This picture is cleared. But some pictures are about to be cleared on
+ // the child thread. To preserve the order, do not send this until those
+ // pictures are cleared.
+ break;
+ }
+ }
+}
+
+void V4L2VideoDecodeAccelerator::PictureCleared() {
+ DVLOGF(4) << "clearing count=" << picture_clearing_count_;
+ DCHECK(decoder_thread_.task_runner()->BelongsToCurrentThread());
+ DCHECK_GT(picture_clearing_count_, 0);
+ picture_clearing_count_--;
+ SendPictureReady();
+}
+
+} // namespace media
diff --git a/vda/v4l2_video_decode_accelerator.h b/vda/v4l2_video_decode_accelerator.h
new file mode 100644
index 0000000..9377043
--- /dev/null
+++ b/vda/v4l2_video_decode_accelerator.h
@@ -0,0 +1,510 @@
+// Copyright 2014 The Chromium Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file.
+//
+// This file contains an implementation of VideoDecodeAccelerator
+// that utilizes hardware video decoders, which expose Video4Linux 2 API
+// (http://linuxtv.org/downloads/v4l-dvb-apis/).
+
+#ifndef MEDIA_GPU_V4L2_VIDEO_DECODE_ACCELERATOR_H_
+#define MEDIA_GPU_V4L2_VIDEO_DECODE_ACCELERATOR_H_
+
+#include <stddef.h>
+#include <stdint.h>
+
+#include <list>
+#include <memory>
+#include <queue>
+#include <vector>
+
+#include "base/callback_forward.h"
+#include "base/macros.h"
+#include "base/memory/linked_ptr.h"
+#include "base/memory/ref_counted.h"
+#include "base/synchronization/waitable_event.h"
+#include "base/threading/thread.h"
+#include "picture.h"
+#include "size.h"
+#include "v4l2_device.h"
+#include "video_decode_accelerator.h"
+
+namespace media {
+
+class H264Parser;
+
+// This class handles video accelerators directly through a V4L2 device exported
+// by the hardware blocks.
+//
+// The threading model of this class is driven by the fact that it needs to
+// interface two fundamentally different event queues -- the one Chromium
+// provides through MessageLoop, and the one driven by the V4L2 devices which
+// is waited on with epoll(). There are three threads involved in this class:
+//
+// * The child thread, which is the main GPU process thread which calls the
+// VideoDecodeAccelerator entry points. Calls from this thread
+// generally do not block (with the exception of Initialize() and Destroy()).
+// They post tasks to the decoder_thread_, which actually services the task
+// and calls back when complete through the
+// VideoDecodeAccelerator::Client interface.
+// * The decoder_thread_, owned by this class. It services API tasks, through
+// the *Task() routines, as well as V4L2 device events, through
+// ServiceDeviceTask(). Almost all state modification is done on this thread
+// (this doesn't include buffer (re)allocation sequence, see below).
+// * The device_poll_thread_, owned by this class. All it does is epoll() on
+// the V4L2 in DevicePollTask() and schedule a ServiceDeviceTask() on the
+// decoder_thread_ when something interesting happens.
+// TODO(sheu): replace this thread with an TYPE_IO decoder_thread_.
+//
+// Note that this class has (almost) no locks, apart from the pictures_assigned_
+// WaitableEvent. Everything (apart from buffer (re)allocation) is serviced on
+// the decoder_thread_, so there are no synchronization issues.
+// ... well, there are, but it's a matter of getting messages posted in the
+// right order, not fiddling with locks.
+// Buffer creation is a two-step process that is serviced partially on the
+// Child thread, because we need to wait for the client to provide textures
+// for the buffers we allocate. We cannot keep the decoder thread running while
+// the client allocates Pictures for us, because we need to REQBUFS first to get
+// the required number of output buffers from the device and that cannot be done
+// unless we free the previous set of buffers, leaving the decoding in a
+// inoperable state for the duration of the wait for Pictures. So to prevent
+// subtle races (esp. if we get Reset() in the meantime), we block the decoder
+// thread while we wait for AssignPictureBuffers from the client.
+//
+// V4L2VideoDecodeAccelerator may use image processor to convert the output.
+// There are three cases:
+// Flush: V4L2VDA should wait until image processor returns all processed
+// frames.
+// Reset: V4L2VDA doesn't need to wait for image processor. When image processor
+// returns an old frame, drop it.
+// Resolution change: V4L2VDA destroy image processor when destroying output
+// buffrers. We cannot drop any frame during resolution change. So V4L2VDA
+// should destroy output buffers after image processor returns all the frames.
+class V4L2VideoDecodeAccelerator
+ : public VideoDecodeAccelerator {
+ public:
+ V4L2VideoDecodeAccelerator(
+ const scoped_refptr<V4L2Device>& device);
+ ~V4L2VideoDecodeAccelerator() override;
+
+ // VideoDecodeAccelerator implementation.
+ // Note: Initialize() and Destroy() are synchronous.
+ bool Initialize(const Config& config, Client* client) override;
+ void Decode(const BitstreamBuffer& bitstream_buffer) override;
+ void AssignPictureBuffers(const std::vector<PictureBuffer>& buffers) override;
+ void ImportBufferForPicture(
+ int32_t picture_buffer_id,
+ const NativePixmapHandle& native_pixmap_handle) override;
+ void ReusePictureBuffer(int32_t picture_buffer_id) override;
+ void Flush() override;
+ void Reset() override;
+ void Destroy() override;
+ bool TryToSetupDecodeOnSeparateThread(
+ const base::WeakPtr<Client>& decode_client,
+ const scoped_refptr<base::SingleThreadTaskRunner>& decode_task_runner)
+ override;
+
+ static VideoDecodeAccelerator::SupportedProfiles GetSupportedProfiles();
+
+ private:
+ // These are rather subjectively tuned.
+ enum {
+ kInputBufferCount = 8,
+ // TODO(posciak): determine input buffer size based on level limits.
+ // See http://crbug.com/255116.
+ // Input bitstream buffer size for up to 1080p streams.
+ kInputBufferMaxSizeFor1080p = 1024 * 1024,
+ // Input bitstream buffer size for up to 4k streams.
+ kInputBufferMaxSizeFor4k = 4 * kInputBufferMaxSizeFor1080p,
+ // This is originally from media/base/limits.h in Chromium.
+ kMaxVideoFrames = 4,
+ // Number of output buffers to use for each VDA stage above what's required
+ // by the decoder (e.g. DPB size, in H264). We need
+ // limits::kMaxVideoFrames to fill up the GpuVideoDecode pipeline,
+ // and +1 for a frame in transit.
+ kDpbOutputBufferExtraCount = kMaxVideoFrames + 1,
+ // Number of extra output buffers if image processor is used.
+ kDpbOutputBufferExtraCountForImageProcessor = 1,
+ };
+
+ // Internal state of the decoder.
+ enum State {
+ kUninitialized, // Initialize() not yet called.
+ kInitialized, // Initialize() returned true; ready to start decoding.
+ kDecoding, // DecodeBufferInitial() successful; decoding frames.
+ kResetting, // Presently resetting.
+ // Performing resolution change and waiting for image processor to return
+ // all frames.
+ kChangingResolution,
+ // Requested new PictureBuffers via ProvidePictureBuffers(), awaiting
+ // AssignPictureBuffers().
+ kAwaitingPictureBuffers,
+ kError, // Error in kDecoding state.
+ };
+
+ enum OutputRecordState {
+ kFree, // Ready to be queued to the device.
+ kAtDevice, // Held by device.
+ kAtProcessor, // Held by image processor.
+ kAtClient, // Held by client of V4L2VideoDecodeAccelerator.
+ };
+
+ enum BufferId {
+ kFlushBufferId = -2 // Buffer id for flush buffer, queued by FlushTask().
+ };
+
+ // Auto-destruction reference for BitstreamBuffer, for message-passing from
+ // Decode() to DecodeTask().
+ struct BitstreamBufferRef;
+
+ // Record for decoded pictures that can be sent to PictureReady.
+ struct PictureRecord {
+ PictureRecord(bool cleared, const Picture& picture);
+ ~PictureRecord();
+ bool cleared; // Whether the texture is cleared and safe to render from.
+ Picture picture; // The decoded picture.
+ };
+
+ // Record for input buffers.
+ struct InputRecord {
+ InputRecord();
+ ~InputRecord();
+ bool at_device; // held by device.
+ void* address; // mmap() address.
+ size_t length; // mmap() length.
+ off_t bytes_used; // bytes filled in the mmap() segment.
+ int32_t input_id; // triggering input_id as given to Decode().
+ };
+
+ // Record for output buffers.
+ struct OutputRecord {
+ OutputRecord();
+ OutputRecord(OutputRecord&&) = default;
+ ~OutputRecord();
+ OutputRecordState state;
+ int32_t picture_id; // picture buffer id as returned to PictureReady().
+ bool cleared; // Whether the texture is cleared and safe to render
+ // from. See TextureManager for details.
+ // Output fds of the processor. Used only when OutputMode is IMPORT.
+ std::vector<base::ScopedFD> processor_output_fds;
+ };
+
+ //
+ // Decoding tasks, to be run on decode_thread_.
+ //
+
+ // Task to finish initialization on decoder_thread_.
+ void InitializeTask();
+
+ // Enqueue a BitstreamBuffer to decode. This will enqueue a buffer to the
+ // decoder_input_queue_, then queue a DecodeBufferTask() to actually decode
+ // the buffer.
+ void DecodeTask(const BitstreamBuffer& bitstream_buffer);
+
+ // Decode from the buffers queued in decoder_input_queue_. Calls
+ // DecodeBufferInitial() or DecodeBufferContinue() as appropriate.
+ void DecodeBufferTask();
+ // Advance to the next fragment that begins a frame.
+ bool AdvanceFrameFragment(const uint8_t* data, size_t size, size_t* endpos);
+ // Schedule another DecodeBufferTask() if we're behind.
+ void ScheduleDecodeBufferTaskIfNeeded();
+
+ // Return true if we should continue to schedule DecodeBufferTask()s after
+ // completion. Store the amount of input actually consumed in |endpos|.
+ bool DecodeBufferInitial(const void* data, size_t size, size_t* endpos);
+ bool DecodeBufferContinue(const void* data, size_t size);
+
+ // Accumulate data for the next frame to decode. May return false in
+ // non-error conditions; for example when pipeline is full and should be
+ // retried later.
+ bool AppendToInputFrame(const void* data, size_t size);
+ // Flush data for one decoded frame.
+ bool FlushInputFrame();
+
+ // Allocate V4L2 buffers and assign them to |buffers| provided by the client
+ // via AssignPictureBuffers() on decoder thread.
+ void AssignPictureBuffersTask(const std::vector<PictureBuffer>& buffers);
+
+ // Use buffer backed by dmabuf file descriptors in |dmabuf_fds| for the
+ // OutputRecord associated with |picture_buffer_id|, taking ownership of the
+ // file descriptors.
+ void ImportBufferForPictureTask(int32_t picture_buffer_id,
+ std::vector<base::ScopedFD> dmabuf_fds);
+
+ // Service I/O on the V4L2 devices. This task should only be scheduled from
+ // DevicePollTask(). If |event_pending| is true, one or more events
+ // on file descriptor are pending.
+ void ServiceDeviceTask(bool event_pending);
+ // Handle the various device queues.
+ void Enqueue();
+ void Dequeue();
+ // Dequeue one input buffer. Return true if success.
+ bool DequeueInputBuffer();
+ // Dequeue one output buffer. Return true if success.
+ bool DequeueOutputBuffer();
+
+ // Return true if there is a resolution change event pending.
+ bool DequeueResolutionChangeEvent();
+
+ // Enqueue a buffer on the corresponding queue.
+ bool EnqueueInputRecord();
+ bool EnqueueOutputRecord();
+
+ // Process a ReusePictureBuffer() API call. The API call create an EGLSync
+ // object on the main (GPU process) thread; we will record this object so we
+ // can wait on it before reusing the buffer.
+ void ReusePictureBufferTask(int32_t picture_buffer_id);
+
+ // Flush() task. Child thread should not submit any more buffers until it
+ // receives the NotifyFlushDone callback. This task will schedule an empty
+ // BitstreamBufferRef (with input_id == kFlushBufferId) to perform the flush.
+ void FlushTask();
+ // Notify the client of a flush completion, if required. This should be
+ // called any time a relevant queue could potentially be emptied: see
+ // function definition.
+ void NotifyFlushDoneIfNeeded();
+ // Returns true if VIDIOC_DECODER_CMD is supported.
+ bool IsDecoderCmdSupported();
+ // Send V4L2_DEC_CMD_START to the driver. Return true if success.
+ bool SendDecoderCmdStop();
+
+ // Reset() task. Drop all input buffers. If V4L2VDA is not doing resolution
+ // change or waiting picture buffers, call FinishReset.
+ void ResetTask();
+ // This will schedule a ResetDoneTask() that will send the NotifyResetDone
+ // callback, then set the decoder state to kResetting so that all intervening
+ // tasks will drain.
+ void FinishReset();
+ void ResetDoneTask();
+
+ // Device destruction task.
+ void DestroyTask();
+
+ // Start |device_poll_thread_|.
+ bool StartDevicePoll();
+
+ // Stop |device_poll_thread_|.
+ bool StopDevicePoll();
+
+ bool StopInputStream();
+ bool StopOutputStream();
+
+ void StartResolutionChange();
+ void FinishResolutionChange();
+
+ // Try to get output format and visible size, detected after parsing the
+ // beginning of the stream. Sets |again| to true if more parsing is needed.
+ // |visible_size| could be nullptr and ignored.
+ bool GetFormatInfo(struct v4l2_format* format,
+ Size* visible_size,
+ bool* again);
+ // Create output buffers for the given |format| and |visible_size|.
+ bool CreateBuffersForFormat(const struct v4l2_format& format,
+ const Size& visible_size);
+
+ // Try to get |visible_size|. Return visible size, or, if querying it is not
+ // supported or produces invalid size, return |coded_size| instead.
+ Size GetVisibleSize(const Size& coded_size);
+
+ //
+ // Device tasks, to be run on device_poll_thread_.
+ //
+
+ // The device task.
+ void DevicePollTask(bool poll_device);
+
+ //
+ // Safe from any thread.
+ //
+
+ // Error notification (using PostTask() to child thread, if necessary).
+ void NotifyError(Error error);
+
+ // Set the decoder_state_ to kError and notify the client (if necessary).
+ void SetErrorState(Error error);
+
+ //
+ // Other utility functions. Called on decoder_thread_, unless
+ // decoder_thread_ is not yet started, in which case the child thread can call
+ // these (e.g. in Initialize() or Destroy()).
+ //
+
+ // Create the buffers we need.
+ bool CreateInputBuffers();
+ bool CreateOutputBuffers();
+
+ // Destroy buffers.
+ void DestroyInputBuffers();
+ // In contrast to DestroyInputBuffers, which is called only on destruction,
+ // we call DestroyOutputBuffers also during playback, on resolution change.
+ // Even if anything fails along the way, we still want to go on and clean
+ // up as much as possible, so return false if this happens, so that the
+ // caller can error out on resolution change.
+ bool DestroyOutputBuffers();
+
+ // Set input and output formats before starting decode.
+ bool SetupFormats();
+
+ //
+ // Methods run on child thread.
+ //
+
+ // Send decoded pictures to PictureReady.
+ void SendPictureReady();
+
+ // Callback that indicates a picture has been cleared.
+ void PictureCleared();
+
+ // Our original calling task runner for the child thread.
+ scoped_refptr<base::SingleThreadTaskRunner> child_task_runner_;
+
+ // Task runner Decode() and PictureReady() run on.
+ scoped_refptr<base::SingleThreadTaskRunner> decode_task_runner_;
+
+ // WeakPtr<> pointing to |this| for use in posting tasks from the decoder or
+ // device worker threads back to the child thread. Because the worker threads
+ // are members of this class, any task running on those threads is guaranteed
+ // that this object is still alive. As a result, tasks posted from the child
+ // thread to the decoder or device thread should use base::Unretained(this),
+ // and tasks posted the other way should use |weak_this_|.
+ base::WeakPtr<V4L2VideoDecodeAccelerator> weak_this_;
+
+ // To expose client callbacks from VideoDecodeAccelerator.
+ // NOTE: all calls to these objects *MUST* be executed on
+ // child_task_runner_.
+ std::unique_ptr<base::WeakPtrFactory<Client>> client_ptr_factory_;
+ base::WeakPtr<Client> client_;
+ // Callbacks to |decode_client_| must be executed on |decode_task_runner_|.
+ base::WeakPtr<Client> decode_client_;
+
+ //
+ // Decoder state, owned and operated by decoder_thread_.
+ // Before decoder_thread_ has started, the decoder state is managed by
+ // the child (main) thread. After decoder_thread_ has started, the decoder
+ // thread should be the only one managing these.
+ //
+
+ // This thread services tasks posted from the VDA API entry points by the
+ // child thread and device service callbacks posted from the device thread.
+ base::Thread decoder_thread_;
+ // Decoder state machine state.
+ State decoder_state_;
+
+ Config::OutputMode output_mode_;
+
+ // BitstreamBuffer we're presently reading.
+ std::unique_ptr<BitstreamBufferRef> decoder_current_bitstream_buffer_;
+ // The V4L2Device this class is operating upon.
+ scoped_refptr<V4L2Device> device_;
+ // FlushTask() and ResetTask() should not affect buffers that have been
+ // queued afterwards. For flushing or resetting the pipeline then, we will
+ // delay these buffers until after the flush or reset completes.
+ int decoder_delay_bitstream_buffer_id_;
+ // Input buffer we're presently filling.
+ int decoder_current_input_buffer_;
+ // We track the number of buffer decode tasks we have scheduled, since each
+ // task execution should complete one buffer. If we fall behind (due to
+ // resource backpressure, etc.), we'll have to schedule more to catch up.
+ int decoder_decode_buffer_tasks_scheduled_;
+ // Picture buffers held by the client.
+ int decoder_frames_at_client_;
+
+ // Are we flushing?
+ bool decoder_flushing_;
+ // True if VIDIOC_DECODER_CMD is supported.
+ bool decoder_cmd_supported_;
+ // True if flushing is waiting for last output buffer. After
+ // VIDIOC_DECODER_CMD is sent to the driver, this flag will be set to true to
+ // wait for the last output buffer. When this flag is true, flush done will
+ // not be sent. After an output buffer that has the flag V4L2_BUF_FLAG_LAST is
+ // received, this is set to false.
+ bool flush_awaiting_last_output_buffer_;
+
+ // Got a reset request while we were performing resolution change or waiting
+ // picture buffers.
+ bool reset_pending_;
+ // Input queue for decoder_thread_: BitstreamBuffers in.
+ std::queue<linked_ptr<BitstreamBufferRef>> decoder_input_queue_;
+ // For H264 decode, hardware requires that we send it frame-sized chunks.
+ // We'll need to parse the stream.
+ std::unique_ptr<H264Parser> decoder_h264_parser_;
+ // Set if the decoder has a pending incomplete frame in an input buffer.
+ bool decoder_partial_frame_pending_;
+
+ //
+ // Hardware state and associated queues. Since decoder_thread_ services
+ // the hardware, decoder_thread_ owns these too.
+ // output_buffer_map_, free_output_buffers_ and output_planes_count_ are an
+ // exception during the buffer (re)allocation sequence, when the
+ // decoder_thread_ is blocked briefly while the Child thread manipulates
+ // them.
+ //
+
+ // Completed decode buffers.
+ std::queue<int> input_ready_queue_;
+
+ // Input buffer state.
+ bool input_streamon_;
+ // Input buffers enqueued to device.
+ int input_buffer_queued_count_;
+ // Input buffers ready to use, as a LIFO since we don't care about ordering.
+ std::vector<int> free_input_buffers_;
+ // Mapping of int index to input buffer record.
+ std::vector<InputRecord> input_buffer_map_;
+
+ // Output buffer state.
+ bool output_streamon_;
+ // Output buffers enqueued to device.
+ int output_buffer_queued_count_;
+ // Output buffers ready to use, as a FIFO since we want oldest-first to hide
+ // synchronization latency with GL.
+ std::list<int> free_output_buffers_;
+ // Mapping of int index to output buffer record.
+ std::vector<OutputRecord> output_buffer_map_;
+ // Required size of DPB for decoding.
+ int output_dpb_size_;
+
+ // Number of planes (i.e. separate memory buffers) for output.
+ size_t output_planes_count_;
+
+ // Pictures that are ready but not sent to PictureReady yet.
+ std::queue<PictureRecord> pending_picture_ready_;
+
+ // The number of pictures that are sent to PictureReady and will be cleared.
+ int picture_clearing_count_;
+
+ // Output picture coded size.
+ Size coded_size_;
+
+ // Output picture visible size.
+ Size visible_size_;
+
+ //
+ // The device polling thread handles notifications of V4L2 device changes.
+ //
+
+ // The thread.
+ base::Thread device_poll_thread_;
+
+ //
+ // Other state, held by the child (main) thread.
+ //
+
+ // The codec we'll be decoding for.
+ VideoCodecProfile video_profile_;
+ // Chosen input format for video_profile_.
+ uint32_t input_format_fourcc_;
+ // Chosen output format.
+ uint32_t output_format_fourcc_;
+
+ // Input format V4L2 fourccs this class supports.
+ static const uint32_t supported_input_fourccs_[];
+
+ // The WeakPtrFactory for |weak_this_|.
+ base::WeakPtrFactory<V4L2VideoDecodeAccelerator> weak_this_factory_;
+
+ DISALLOW_COPY_AND_ASSIGN(V4L2VideoDecodeAccelerator);
+};
+
+} // namespace media
+
+#endif // MEDIA_GPU_V4L2_VIDEO_DECODE_ACCELERATOR_H_