Compare commits
2 Commits
sandbox/wa
...
sandbox/hk
Author | SHA1 | Date | |
---|---|---|---|
![]() |
21e9f79d25 | ||
![]() |
ffd20828d0 |
@@ -62,6 +62,18 @@ typedef struct {
|
||||
int ref_count;
|
||||
vpx_codec_frame_buffer_t raw_frame_buffer;
|
||||
YV12_BUFFER_CONFIG buf;
|
||||
|
||||
// The Following variables will only be used in frame parallel decode.
|
||||
|
||||
// owner_thread_id indicates which FrameWorker owns this buffer. -1 means
|
||||
// that no FrameWorker owns, or is decoding, this buffer.
|
||||
int owner_worker_id;
|
||||
|
||||
// Buffer has been decoded to (row, col) position. When first start decoding,
|
||||
// they are reset to -1. If a frame has been fully decoded, row and col will
|
||||
// be set to INT_MAX.
|
||||
int row;
|
||||
int col;
|
||||
} RefCntBuffer;
|
||||
|
||||
typedef struct {
|
||||
@@ -80,7 +92,7 @@ typedef struct {
|
||||
|
||||
RefCntBuffer frame_bufs[FRAME_BUFFERS];
|
||||
|
||||
// Frame buffers allocated internally by the codec.
|
||||
// Handles memory for the codec.
|
||||
InternalFrameBufferList int_frame_buffers;
|
||||
} BufferPool;
|
||||
|
||||
@@ -112,6 +124,9 @@ typedef struct VP9Common {
|
||||
|
||||
int ref_frame_map[REF_FRAMES]; /* maps fb_idx to reference slot */
|
||||
|
||||
// Prepare ref_frame_map for next frame. Only used in frame parallel decode.
|
||||
int next_ref_frame_map[REF_FRAMES];
|
||||
|
||||
// TODO(jkoleszar): could expand active_ref_idx to 4, with 0 as intra, and
|
||||
// roll new_fb_idx into it.
|
||||
|
||||
@@ -235,12 +250,19 @@ static INLINE YV12_BUFFER_CONFIG *get_frame_new_buffer(VP9_COMMON *cm) {
|
||||
static INLINE int get_free_fb(VP9_COMMON *cm) {
|
||||
RefCntBuffer *const frame_bufs = cm->buffer_pool->frame_bufs;
|
||||
int i;
|
||||
for (i = 0; i < FRAME_BUFFERS; ++i)
|
||||
|
||||
#if CONFIG_MULTITHREAD
|
||||
pthread_mutex_lock(&cm->buffer_pool->pool_mutex);
|
||||
#endif
|
||||
for (i = 0; i < FRAME_BUFFERS; i++)
|
||||
if (frame_bufs[i].ref_count == 0)
|
||||
break;
|
||||
|
||||
assert(i < FRAME_BUFFERS);
|
||||
frame_bufs[i].ref_count = 1;
|
||||
#if CONFIG_MULTITHREAD
|
||||
pthread_mutex_unlock(&cm->buffer_pool->pool_mutex);
|
||||
#endif
|
||||
return i;
|
||||
}
|
||||
|
||||
@@ -325,6 +347,7 @@ static INLINE void update_partition_context(MACROBLOCKD *xd,
|
||||
PARTITION_CONTEXT *const above_ctx = xd->above_seg_context + mi_col;
|
||||
PARTITION_CONTEXT *const left_ctx = xd->left_seg_context + (mi_row & MI_MASK);
|
||||
|
||||
// num_4x4_blocks_wide_lookup[bsize] / 2
|
||||
const int bs = num_8x8_blocks_wide_lookup[bsize];
|
||||
|
||||
// update the partition context at the end notes. set partition bits
|
||||
|
@@ -1078,8 +1078,8 @@ static size_t read_uncompressed_header(VP9Decoder *pbi,
|
||||
struct vp9_read_bit_buffer *rb) {
|
||||
VP9_COMMON *const cm = &pbi->common;
|
||||
RefCntBuffer *const frame_bufs = cm->buffer_pool->frame_bufs;
|
||||
int mask, i, ref_index = 0;
|
||||
size_t sz;
|
||||
int i;
|
||||
|
||||
cm->last_frame_type = cm->frame_type;
|
||||
|
||||
@@ -1097,12 +1097,18 @@ static size_t read_uncompressed_header(VP9Decoder *pbi,
|
||||
// Show an existing frame directly.
|
||||
const int frame_to_show = cm->ref_frame_map[vp9_rb_read_literal(rb, 3)];
|
||||
|
||||
#if CONFIG_MULTITHREAD
|
||||
pthread_mutex_lock(&cm->buffer_pool->pool_mutex);
|
||||
#endif
|
||||
if (frame_to_show < 0 || frame_bufs[frame_to_show].ref_count < 1)
|
||||
vpx_internal_error(&cm->error, VPX_CODEC_UNSUP_BITSTREAM,
|
||||
"Buffer %d does not contain a decoded frame",
|
||||
frame_to_show);
|
||||
|
||||
ref_cnt_fb(frame_bufs, &cm->new_fb_idx, frame_to_show);
|
||||
#if CONFIG_MULTITHREAD
|
||||
pthread_mutex_unlock(&cm->buffer_pool->pool_mutex);
|
||||
#endif
|
||||
pbi->refresh_frame_flags = 0;
|
||||
cm->lf.filter_level = 0;
|
||||
cm->show_frame = 1;
|
||||
@@ -1198,6 +1204,28 @@ static size_t read_uncompressed_header(VP9Decoder *pbi,
|
||||
// below, forcing the use of context 0 for those frame types.
|
||||
cm->frame_context_idx = vp9_rb_read_literal(rb, FRAME_CONTEXTS_LOG2);
|
||||
|
||||
// Update next_ref_frame_map in frame parallel decode.
|
||||
if (pbi->frame_parallel_decode) {
|
||||
for (mask = pbi->refresh_frame_flags; mask; mask >>= 1) {
|
||||
if (mask & 1) {
|
||||
cm->next_ref_frame_map[ref_index] = cm->new_fb_idx;
|
||||
#if CONFIG_MULTITHREAD
|
||||
pthread_mutex_lock(&cm->buffer_pool->pool_mutex);
|
||||
#endif
|
||||
++cm->buffer_pool->frame_bufs[cm->new_fb_idx].ref_count;
|
||||
#if CONFIG_MULTITHREAD
|
||||
pthread_mutex_unlock(&cm->buffer_pool->pool_mutex);
|
||||
#endif
|
||||
} else {
|
||||
cm->next_ref_frame_map[ref_index] = cm->ref_frame_map[ref_index];
|
||||
}
|
||||
++ref_index;
|
||||
}
|
||||
|
||||
for (; ref_index < REF_FRAMES; ++ref_index)
|
||||
cm->next_ref_frame_map[ref_index] = cm->ref_frame_map[ref_index];
|
||||
}
|
||||
|
||||
if (frame_is_intra_only(cm) || cm->error_resilient_mode)
|
||||
vp9_setup_past_independence(cm);
|
||||
|
||||
@@ -1394,6 +1422,17 @@ void vp9_decode_frame(VP9Decoder *pbi,
|
||||
|
||||
new_fb->corrupted |= xd->corrupted;
|
||||
|
||||
// Update progress in frame parallel decode.
|
||||
if (pbi->frame_parallel_decode) {
|
||||
VP9Worker *worker = pbi->owner_frame_worker;
|
||||
FrameWorkerData *const worker_data = worker->data1;
|
||||
pthread_mutex_lock(&worker_data->stats_mutex);
|
||||
pbi->cur_buf->row = INT_MAX;
|
||||
pbi->cur_buf->col = INT_MAX;
|
||||
pthread_cond_signal(&worker_data->stats_cond);
|
||||
pthread_mutex_unlock(&worker_data->stats_mutex);
|
||||
}
|
||||
|
||||
if (!new_fb->corrupted) {
|
||||
if (!cm->error_resilient_mode && !cm->frame_parallel_decoding_mode) {
|
||||
vp9_adapt_coef_probs(cm);
|
||||
|
@@ -63,6 +63,7 @@ VP9Decoder *vp9_decoder_create(BufferPool *const pool) {
|
||||
|
||||
// Initialize the references to not point to any frame buffers.
|
||||
vpx_memset(&cm->ref_frame_map, -1, sizeof(cm->ref_frame_map));
|
||||
vpx_memset(&cm->next_ref_frame_map, -1, sizeof(cm->next_ref_frame_map));
|
||||
|
||||
cm->current_video_frame = 0;
|
||||
pbi->ready_for_new_data = 1;
|
||||
@@ -205,12 +206,18 @@ static void swap_frame_buffers(VP9Decoder *pbi) {
|
||||
for (mask = pbi->refresh_frame_flags; mask; mask >>= 1) {
|
||||
if (mask & 1) {
|
||||
const int old_idx = cm->ref_frame_map[ref_index];
|
||||
#if CONFIG_MULTITHREAD
|
||||
pthread_mutex_lock(&cm->buffer_pool->pool_mutex);
|
||||
#endif
|
||||
ref_cnt_fb(frame_bufs, &cm->ref_frame_map[ref_index],
|
||||
cm->new_fb_idx);
|
||||
if (old_idx >= 0 && frame_bufs[old_idx].ref_count == 0)
|
||||
pool->release_fb_cb(pool->cb_priv,
|
||||
&frame_bufs[old_idx].raw_frame_buffer);
|
||||
}
|
||||
#if CONFIG_MULTITHREAD
|
||||
pthread_mutex_unlock(&cm->buffer_pool->pool_mutex);
|
||||
#endif
|
||||
++ref_index;
|
||||
}
|
||||
|
||||
@@ -256,6 +263,17 @@ int vp9_receive_compressed_data(VP9Decoder *pbi,
|
||||
&frame_bufs[cm->new_fb_idx].raw_frame_buffer);
|
||||
cm->new_fb_idx = get_free_fb(cm);
|
||||
|
||||
if (pbi->frame_parallel_decode) {
|
||||
VP9Worker *worker = pbi->owner_frame_worker;
|
||||
FrameWorkerData *const worker_data = worker->data1;
|
||||
pbi->cur_buf = &pool->frame_bufs[cm->new_fb_idx];
|
||||
pool->frame_bufs[cm->new_fb_idx].owner_worker_id = worker_data->worker_id;
|
||||
|
||||
// Reset the decoding progress.
|
||||
pbi->cur_buf->row = -1;
|
||||
pbi->cur_buf->col = -1;
|
||||
}
|
||||
|
||||
if (setjmp(cm->error.jmp)) {
|
||||
cm->error.setjmp = 0;
|
||||
|
||||
|
@@ -45,8 +45,14 @@ typedef struct VP9Decoder {
|
||||
|
||||
int frame_parallel_decode; // frame-based threading.
|
||||
|
||||
// TODO(hkuang): Combine this with cur_buf in macroblockd as they are
|
||||
// the same.
|
||||
RefCntBuffer *cur_buf; // current decoding reference buffer.
|
||||
|
||||
VP9Worker lf_worker;
|
||||
VP9Worker *tile_workers;
|
||||
VP9Worker *owner_frame_worker; // frame_worker that owns this pbi;
|
||||
|
||||
int num_tile_workers;
|
||||
|
||||
TileData *tile_data;
|
||||
|
@@ -279,3 +279,78 @@ void vp9_loop_filter_dealloc(VP9LfSync *lf_sync, int rows) {
|
||||
vp9_zero(*lf_sync);
|
||||
}
|
||||
}
|
||||
|
||||
void vp9_frameworker_wait(VP9Worker* const worker, int row, int col,
|
||||
RefCntBuffer *ref_buf) {
|
||||
FrameWorkerData *const worker_data = (FrameWorkerData *)worker->data1;
|
||||
const VP9Decoder *const pbi = worker_data->pbi;
|
||||
const RefCntBuffer *const cur_buf = pbi->cur_buf;
|
||||
|
||||
// Check if worker already release the ref_buf.
|
||||
if (!worker || ref_buf->owner_worker_id == -1) return;
|
||||
|
||||
pthread_mutex_lock(&worker_data->stats_mutex);
|
||||
while (!(cur_buf->row >= row && cur_buf->col >= col)
|
||||
&& pbi->cur_buf == ref_buf && ref_buf->owner_worker_id != -1) {
|
||||
pthread_cond_wait(&worker_data->stats_cond, &worker_data->stats_mutex);
|
||||
}
|
||||
pthread_mutex_unlock(&worker_data->stats_mutex);
|
||||
}
|
||||
|
||||
void vp9_frameworker_broadcast(VP9Worker* const worker, int row, int col) {
|
||||
FrameWorkerData *const worker_data = (FrameWorkerData *)worker->data1;
|
||||
const VP9Decoder *const pbi = worker_data->pbi;
|
||||
RefCntBuffer *const cur_buf = pbi->cur_buf;
|
||||
|
||||
pthread_mutex_lock(&worker_data->stats_mutex);
|
||||
cur_buf->row = row;
|
||||
cur_buf->col = col;
|
||||
pthread_cond_signal(&worker_data->stats_cond);
|
||||
pthread_mutex_unlock(&worker_data->stats_mutex);
|
||||
}
|
||||
|
||||
void vp9_frameworker_copy_context(VP9Worker *const dst_worker,
|
||||
const VP9Worker *const src_worker) {
|
||||
FrameWorkerData *const src_worker_data =
|
||||
(FrameWorkerData *)dst_worker->data1;
|
||||
FrameWorkerData *const dst_worker_data =
|
||||
(FrameWorkerData *)src_worker->data1;
|
||||
const VP9_COMMON *const src_cm = &src_worker_data->pbi->common;
|
||||
VP9_COMMON *const dst_cm = &dst_worker_data->pbi->common;
|
||||
int i;
|
||||
|
||||
// Wait until source frame's context is ready.
|
||||
pthread_mutex_lock(&src_worker_data->stats_mutex);
|
||||
while (!src_worker_data->frame_context_ready) {
|
||||
pthread_cond_wait(&src_worker_data->stats_cond,
|
||||
&src_worker_data->stats_mutex);
|
||||
}
|
||||
pthread_mutex_unlock(&src_worker_data->stats_mutex);
|
||||
|
||||
dst_cm->last_width = src_cm->width;
|
||||
dst_cm->last_height = src_cm->height;
|
||||
dst_cm->subsampling_x = src_cm->subsampling_x;
|
||||
dst_cm->subsampling_y = src_cm->subsampling_y;
|
||||
|
||||
for (i = 0; i < REF_FRAMES; ++i)
|
||||
dst_cm->ref_frame_map[i] = src_cm->next_ref_frame_map[i];
|
||||
|
||||
dst_cm->last_show_frame = src_cm->show_frame;
|
||||
|
||||
dst_cm->prev_mip = src_cm->mip;
|
||||
dst_cm->prev_mi = src_cm->mi;
|
||||
dst_cm->prev_mi_grid_base = src_cm->mi_grid_base;
|
||||
dst_cm->prev_mi_grid_visible = src_cm->mi_grid_visible;
|
||||
dst_cm->lf.last_sharpness_level = src_cm->lf.sharpness_level;
|
||||
|
||||
for (i = 0; i < MAX_REF_LF_DELTAS; ++i) {
|
||||
dst_cm->lf.last_ref_deltas[i] = src_cm->lf.ref_deltas[i];
|
||||
dst_cm->lf.ref_deltas[i] = src_cm->lf.ref_deltas[i];
|
||||
}
|
||||
|
||||
for (i = 0; i < MAX_MODE_LF_DELTAS; ++i)
|
||||
dst_cm-> lf.last_mode_deltas[i] = src_cm->lf.mode_deltas[i];
|
||||
|
||||
for (i = 0; i < FRAME_CONTEXTS; ++i)
|
||||
dst_cm-> frame_contexts[i] = src_cm->frame_contexts[i];
|
||||
}
|
||||
|
@@ -55,6 +55,11 @@ typedef struct FrameWorkerData {
|
||||
// It is used to make a copy of the compressed data.
|
||||
uint8_t *scratch_buffer;
|
||||
size_t scratch_buffer_size;
|
||||
|
||||
pthread_mutex_t stats_mutex;
|
||||
pthread_cond_t stats_cond;
|
||||
|
||||
int frame_context_ready; // Current frame's context is ready to read.
|
||||
} FrameWorkerData;
|
||||
|
||||
// Allocate memory for loopfilter row synchronization.
|
||||
@@ -71,4 +76,19 @@ void vp9_loop_filter_frame_mt(YV12_BUFFER_CONFIG *frame,
|
||||
int frame_filter_level,
|
||||
int y_only);
|
||||
|
||||
// Wait for FrameWorker to finish decoding ref_buf to (r,c) position.
|
||||
// Note: worker may already finish decoding ref_buf and release it in order to
|
||||
// start decoding next frame. So need to check whether worker is still decoding
|
||||
// ref_buf.
|
||||
void vp9_frameworker_wait(VP9Worker* const worker, int row, int col,
|
||||
RefCntBuffer *ref_buf);
|
||||
|
||||
// FrameWorker broadcasts its decoding progress so other workers that are
|
||||
// waiting it could resume decoding.
|
||||
void vp9_frameworker_broadcast(VP9Worker* const worker, int row, int col);
|
||||
|
||||
// Copy necessary decoding context from src worker to dst worker.
|
||||
void vp9_frameworker_copy_context(VP9Worker *const dst_worker,
|
||||
const VP9Worker *const src_worker);
|
||||
|
||||
#endif // VP9_DECODER_VP9_DTHREAD_H_
|
||||
|
@@ -303,7 +303,9 @@ static vpx_codec_err_t init_decoder(vpx_codec_alg_priv_t *ctx) {
|
||||
set_error_detail(ctx, "Failed to allocate worker_data");
|
||||
return VPX_CODEC_MEM_ERROR;
|
||||
}
|
||||
|
||||
worker_data->pbi->owner_frame_worker = worker;
|
||||
worker_data->worker_id = i;
|
||||
worker_data->frame_context_ready = 0;
|
||||
// If decoding in serial mode, FrameWorker thread could create tile worker
|
||||
// thread or loopfilter thread.
|
||||
worker_data->pbi->max_threads =
|
||||
@@ -579,11 +581,17 @@ static vpx_image_t *decoder_get_frame(vpx_codec_alg_priv_t *ctx,
|
||||
*iter = img;
|
||||
// Decrease reference count of last output frame in frame parallel mode.
|
||||
if (ctx->frame_parallel_decode && ctx->last_show_frame >= 0) {
|
||||
#if CONFIG_MULTITHREAD
|
||||
pthread_mutex_lock(&cm->buffer_pool->pool_mutex);
|
||||
#endif
|
||||
--frame_bufs[ctx->last_show_frame].ref_count;
|
||||
if (frame_bufs[ctx->last_show_frame].ref_count == 0) {
|
||||
pool->release_fb_cb(pool->cb_priv,
|
||||
&frame_bufs[ctx->last_show_frame].raw_frame_buffer);
|
||||
}
|
||||
#if CONFIG_MULTITHREAD
|
||||
pthread_mutex_unlock(&cm->buffer_pool->pool_mutex);
|
||||
#endif
|
||||
}
|
||||
ctx->last_show_frame = worker_data->pbi->common.new_fb_idx;
|
||||
}
|
||||
|
Reference in New Issue
Block a user