rtc: Pipeline lpf after encode for threads > 1

In multi-threaded encoding scenario, when no encode
job is available for a thread, start loop filtering
of a superblock row.

Change-Id: I32140a5bcfddcd2bb9a9466fcbc22a3b4a7a58d0
diff --git a/av1/encoder/encoder.c b/av1/encoder/encoder.c
index 36881b3..51a2671 100644
--- a/av1/encoder/encoder.c
+++ b/av1/encoder/encoder.c
@@ -1614,12 +1614,17 @@
   MultiThreadInfo *const mt_info = &cpi->mt_info;
 #if CONFIG_MULTITHREAD
   pthread_mutex_t *const enc_row_mt_mutex_ = mt_info->enc_row_mt.mutex_;
+  pthread_cond_t *const enc_row_mt_cond_ = mt_info->enc_row_mt.cond_;
   pthread_mutex_t *const gm_mt_mutex_ = mt_info->gm_sync.mutex_;
   pthread_mutex_t *const pack_bs_mt_mutex_ = mt_info->pack_bs_sync.mutex_;
   if (enc_row_mt_mutex_ != NULL) {
     pthread_mutex_destroy(enc_row_mt_mutex_);
     aom_free(enc_row_mt_mutex_);
   }
+  if (enc_row_mt_cond_ != NULL) {
+    pthread_cond_destroy(enc_row_mt_cond_);
+    aom_free(enc_row_mt_cond_);
+  }
   if (gm_mt_mutex_ != NULL) {
     pthread_mutex_destroy(gm_mt_mutex_);
     aom_free(gm_mt_mutex_);
@@ -2301,45 +2306,6 @@
   }
 }
 
-static void set_postproc_filter_default_params(AV1_COMMON *cm) {
-  struct loopfilter *const lf = &cm->lf;
-  CdefInfo *const cdef_info = &cm->cdef_info;
-  RestorationInfo *const rst_info = cm->rst_info;
-
-  lf->filter_level[0] = 0;
-  lf->filter_level[1] = 0;
-  cdef_info->cdef_bits = 0;
-  cdef_info->cdef_strengths[0] = 0;
-  cdef_info->nb_cdef_strengths = 1;
-  cdef_info->cdef_uv_strengths[0] = 0;
-  rst_info[0].frame_restoration_type = RESTORE_NONE;
-  rst_info[1].frame_restoration_type = RESTORE_NONE;
-  rst_info[2].frame_restoration_type = RESTORE_NONE;
-}
-
-// Checks if post-processing filters need to be applied.
-// NOTE: This function decides if the application of different post-processing
-// filters on the reconstructed frame can be skipped at the encoder side.
-// However the computation of different filter parameters that are signaled in
-// the bitstream is still required.
-static bool should_skip_postproc_filtering(AV1_COMP *cpi, int use_cdef,
-                                           int use_restoration) {
-  if (!cpi->oxcf.algo_cfg.skip_postproc_filtering || cpi->ppi->b_calculate_psnr)
-    return false;
-  assert(cpi->oxcf.mode == ALLINTRA);
-  const AV1_COMMON *const cm = &cpi->common;
-
-  // The post-processing filters are applied one after the other. In case of
-  // ALLINTRA encoding, the reconstructed frame is not used as a reference
-  // frame. Hence, the application of these filters can be skipped when
-  // 1. filter parameters of the subsequent stages are not dependent on the
-  // filtered output of the current stage or
-  // 2. subsequent filtering stages are disabled
-  // Hence, the application of deblocking filters is also skipped if there are
-  // no further filtering stages.
-  return (!use_cdef && !av1_superres_scaled(cm) && !use_restoration);
-}
-
 /*!\brief Select and apply deblocking filters, cdef filters, and restoration
  * filters.
  *
@@ -2350,11 +2316,13 @@
   const int num_workers = mt_info->num_mod_workers[MOD_LPF];
   const int num_planes = av1_num_planes(cm);
   MACROBLOCKD *xd = &cpi->td.mb.e_mbd;
+  cpi->td.mb.rdmult = cpi->rd.RDMULT;
 
   assert(IMPLIES(is_lossless_requested(&cpi->oxcf.rc_cfg),
                  cm->features.coded_lossless && cm->features.all_lossless));
 
-  const int use_loopfilter = is_loopfilter_used(cm);
+  const int use_loopfilter =
+      is_loopfilter_used(cm) && !cpi->mt_info.pipeline_lpf_mt_with_enc;
   const int use_cdef = is_cdef_used(cm);
   const int use_restoration = is_restoration_used(cm);
 
@@ -3160,7 +3128,8 @@
   cm->cur_frame->buf.render_width = cm->render_width;
   cm->cur_frame->buf.render_height = cm->render_height;
 
-  set_postproc_filter_default_params(cm);
+  if (!cpi->mt_info.pipeline_lpf_mt_with_enc)
+    set_postproc_filter_default_params(&cpi->common);
 
   if (!cm->features.allow_intrabc) loopfilter_frame(cpi, cm);
 
diff --git a/av1/encoder/encoder.h b/av1/encoder/encoder.h
index 0d5ee67..80bbdec 100644
--- a/av1/encoder/encoder.h
+++ b/av1/encoder/encoder.h
@@ -1490,11 +1490,27 @@
    */
   int thread_id_to_tile_id[MAX_NUM_THREADS];
 
+  /*!
+   * num_tile_cols_done[i] indicates the number of tile columns whose encoding
+   * is complete in the ith superblock row.
+   */
+  int *num_tile_cols_done;
+
+  /*!
+   * Number of superblock rows in a frame for which 'num_tile_cols_done' is
+   * allocated.
+   */
+  int allocated_sb_rows;
+
 #if CONFIG_MULTITHREAD
   /*!
    * Mutex lock used while dispatching jobs.
    */
   pthread_mutex_t *mutex_;
+  /*!
+   *  Condition variable used to dispatch loopfilter jobs.
+   */
+  pthread_cond_t *cond_;
 #endif
 
   /**
@@ -1675,6 +1691,12 @@
    * Buffers to be stored/restored before/after parallel encode.
    */
   RestoreStateBuffers restore_state_buf;
+
+  /*!
+   * In multi-threaded realtime encoding with row-mt enabled, pipeline
+   * loop-filtering after encoding.
+   */
+  int pipeline_lpf_mt_with_enc;
 } MultiThreadInfo;
 
 /*!\cond */
@@ -4091,6 +4113,45 @@
          !cm->tiles.large_scale;
 }
 
+// Checks if post-processing filters need to be applied.
+// NOTE: This function decides if the application of different post-processing
+// filters on the reconstructed frame can be skipped at the encoder side.
+// However the computation of different filter parameters that are signaled in
+// the bitstream is still required.
+static INLINE bool should_skip_postproc_filtering(AV1_COMP *cpi, int use_cdef,
+                                                  int use_restoration) {
+  if (!cpi->oxcf.algo_cfg.skip_postproc_filtering || cpi->ppi->b_calculate_psnr)
+    return false;
+  assert(cpi->oxcf.mode == ALLINTRA);
+  const AV1_COMMON *const cm = &cpi->common;
+
+  // The post-processing filters are applied one after the other. In case of
+  // ALLINTRA encoding, the reconstructed frame is not used as a reference
+  // frame. Hence, the application of these filters can be skipped when
+  // 1. filter parameters of the subsequent stages are not dependent on the
+  // filtered output of the current stage or
+  // 2. subsequent filtering stages are disabled
+  // Hence, the application of deblocking filters is also skipped if there are
+  // no further filtering stages.
+  return (!use_cdef && !av1_superres_scaled(cm) && !use_restoration);
+}
+
+static INLINE void set_postproc_filter_default_params(AV1_COMMON *cm) {
+  struct loopfilter *const lf = &cm->lf;
+  CdefInfo *const cdef_info = &cm->cdef_info;
+  RestorationInfo *const rst_info = cm->rst_info;
+
+  lf->filter_level[0] = 0;
+  lf->filter_level[1] = 0;
+  cdef_info->cdef_bits = 0;
+  cdef_info->cdef_strengths[0] = 0;
+  cdef_info->nb_cdef_strengths = 1;
+  cdef_info->cdef_uv_strengths[0] = 0;
+  rst_info[0].frame_restoration_type = RESTORE_NONE;
+  rst_info[1].frame_restoration_type = RESTORE_NONE;
+  rst_info[2].frame_restoration_type = RESTORE_NONE;
+}
+
 static INLINE int is_inter_tx_size_search_level_one(
     const TX_SPEED_FEATURES *tx_sf) {
   return (tx_sf->inter_tx_size_search_init_depth_rect >= 1 &&
diff --git a/av1/encoder/ethread.c b/av1/encoder/ethread.c
index 1701a91..e30010f 100644
--- a/av1/encoder/ethread.c
+++ b/av1/encoder/ethread.c
@@ -24,6 +24,7 @@
 #include "av1/encoder/global_motion.h"
 #include "av1/encoder/global_motion_facade.h"
 #include "av1/encoder/intra_mode_search_utils.h"
+#include "av1/encoder/picklpf.h"
 #include "av1/encoder/rdopt.h"
 #include "aom_dsp/aom_dsp_common.h"
 #include "av1/encoder/temporal_filter.h"
@@ -221,6 +222,11 @@
   }
 }
 
+static AOM_INLINE int get_sb_rows_in_frame(AV1_COMMON *cm) {
+  return CEIL_POWER_OF_TWO(cm->mi_params.mi_rows,
+                           cm->seq_params->mib_size_log2);
+}
+
 static void row_mt_mem_alloc(AV1_COMP *cpi, int max_rows, int max_cols,
                              int alloc_row_ctx) {
   struct AV1Common *cm = &cpi->common;
@@ -247,10 +253,16 @@
       }
     }
   }
+  const int sb_rows = get_sb_rows_in_frame(cm);
+  CHECK_MEM_ERROR(
+      cm, enc_row_mt->num_tile_cols_done,
+      aom_malloc(sizeof(*enc_row_mt->num_tile_cols_done) * sb_rows));
+
   enc_row_mt->allocated_tile_cols = tile_cols;
   enc_row_mt->allocated_tile_rows = tile_rows;
   enc_row_mt->allocated_rows = max_rows;
   enc_row_mt->allocated_cols = max_cols - 1;
+  enc_row_mt->allocated_sb_rows = sb_rows;
 }
 
 void av1_row_mt_mem_dealloc(AV1_COMP *cpi) {
@@ -270,10 +282,12 @@
       if (cpi->oxcf.algo_cfg.cdf_update_mode) aom_free(this_tile->row_ctx);
     }
   }
+  aom_free(enc_row_mt->num_tile_cols_done);
   enc_row_mt->allocated_rows = 0;
   enc_row_mt->allocated_cols = 0;
   enc_row_mt->allocated_tile_cols = 0;
   enc_row_mt->allocated_tile_rows = 0;
+  enc_row_mt->allocated_sb_rows = 0;
 }
 
 static AOM_INLINE void assign_tile_to_thread(int *thread_id_to_tile_id,
@@ -433,6 +447,40 @@
 }
 #endif
 
+static void launch_loop_filter_rows(AV1_COMMON *cm, EncWorkerData *thread_data,
+                                    AV1EncRowMultiThreadInfo *enc_row_mt,
+                                    int mib_size_log2) {
+  AV1LfSync *const lf_sync = (AV1LfSync *)thread_data->lf_sync;
+  const int sb_rows = get_sb_rows_in_frame(cm);
+  AV1LfMTInfo *cur_job_info;
+  (void)enc_row_mt;
+#if CONFIG_MULTITHREAD
+  pthread_mutex_t *enc_row_mt_mutex_ = enc_row_mt->mutex_;
+#endif
+
+  while ((cur_job_info = get_lf_job_info(lf_sync)) != NULL) {
+    LFWorkerData *const lf_data = (LFWorkerData *)thread_data->lf_data;
+    const int lpf_opt_level = cur_job_info->lpf_opt_level;
+    (void)sb_rows;
+#if CONFIG_MULTITHREAD
+    const int cur_sb_row = cur_job_info->mi_row >> mib_size_log2;
+    const int next_sb_row = AOMMIN(sb_rows - 1, cur_sb_row + 1);
+    // Wait for current and next superblock row to finish encoding.
+    pthread_mutex_lock(enc_row_mt_mutex_);
+    while (enc_row_mt->num_tile_cols_done[cur_sb_row] < cm->tiles.cols ||
+           enc_row_mt->num_tile_cols_done[next_sb_row] < cm->tiles.cols) {
+      pthread_cond_wait(enc_row_mt->cond_, enc_row_mt_mutex_);
+    }
+    pthread_mutex_unlock(enc_row_mt_mutex_);
+#endif
+    av1_thread_loop_filter_rows(
+        lf_data->frame_buffer, lf_data->cm, lf_data->planes, lf_data->xd,
+        cur_job_info->mi_row, cur_job_info->plane, cur_job_info->dir,
+        lpf_opt_level, lf_sync, lf_data->params_buf, lf_data->tx_buf,
+        mib_size_log2);
+  }
+}
+
 static int enc_row_mt_worker_hook(void *arg1, void *unused) {
   EncWorkerData *const thread_data = (EncWorkerData *)arg1;
   AV1_COMP *const cpi = thread_data->cpi;
@@ -440,6 +488,7 @@
   int thread_id = thread_data->thread_id;
   AV1EncRowMultiThreadInfo *const enc_row_mt = &cpi->mt_info.enc_row_mt;
   int cur_tile_id = enc_row_mt->thread_id_to_tile_id[thread_id];
+  const int mib_size_log2 = cm->seq_params->mib_size_log2;
 #if CONFIG_MULTITHREAD
   pthread_mutex_t *enc_row_mt_mutex_ = enc_row_mt->mutex_;
 #endif
@@ -486,6 +535,7 @@
     const int tile_row = tile_info->tile_row;
     const int tile_col = tile_info->tile_col;
     ThreadData *td = thread_data->td;
+    const int sb_row = current_mi_row >> mib_size_log2;
 
     assert(current_mi_row != -1 && current_mi_row <= tile_info->mi_row_end);
 
@@ -516,11 +566,21 @@
 #endif
     this_tile->abs_sum_level += td->abs_sum_level;
     row_mt_sync->num_threads_working--;
+    enc_row_mt->num_tile_cols_done[sb_row]++;
 #if CONFIG_MULTITHREAD
+    pthread_cond_broadcast(enc_row_mt->cond_);
     pthread_mutex_unlock(enc_row_mt_mutex_);
 #endif
   }
-
+  if (cpi->mt_info.pipeline_lpf_mt_with_enc &&
+      (cm->lf.filter_level[PLANE_TYPE_Y] ||
+       cm->lf.filter_level[PLANE_TYPE_UV])) {
+    // Loop-filter a superblock row if encoding of the current and next
+    // superblock row is complete.
+    // TODO(deepa.kg @ittiam.com) Evaluate encoder speed by interleaving
+    // encoding and loop filter stage.
+    launch_loop_filter_rows(cm, thread_data, enc_row_mt, mib_size_log2);
+  }
   av1_free_pc_tree_recursive(thread_data->td->rt_pc_root, av1_num_planes(cm), 0,
                              0);
   return 1;
@@ -612,6 +672,11 @@
                       aom_malloc(sizeof(*(enc_row_mt->mutex_))));
       if (enc_row_mt->mutex_) pthread_mutex_init(enc_row_mt->mutex_, NULL);
     }
+    if (enc_row_mt->cond_ == NULL) {
+      CHECK_MEM_ERROR(cm, enc_row_mt->cond_,
+                      aom_malloc(sizeof(*(enc_row_mt->cond_))));
+      if (enc_row_mt->cond_) pthread_cond_init(enc_row_mt->cond_, NULL);
+    }
   }
 
   if (!is_first_pass) {
@@ -1245,10 +1310,16 @@
                                            int num_workers) {
   MultiThreadInfo *const mt_info = &cpi->mt_info;
   AV1_COMMON *const cm = &cpi->common;
+  MACROBLOCKD *xd = &cpi->td.mb.e_mbd;
   for (int i = num_workers - 1; i >= 0; i--) {
     AVxWorker *const worker = &mt_info->workers[i];
     EncWorkerData *const thread_data = &mt_info->tile_thr_data[i];
 
+    // Initialize loopfilter data
+    thread_data->lf_sync = &mt_info->lf_row_sync;
+    thread_data->lf_data = &thread_data->lf_sync->lfdata[i];
+    loop_filter_data_reset(thread_data->lf_data, &cm->cur_frame->buf, cm, xd);
+
     worker->hook = hook;
     worker->data1 = thread_data;
     worker->data2 = NULL;
@@ -1524,12 +1595,67 @@
 }
 #endif
 
+static void lpf_pipeline_mt_init(AV1_COMP *cpi) {
+  // Pipelining of loop-filtering after encoding is enabled when loop-filter
+  // level is chosen based on quantizer and frame type. It is disabled in case
+  // of 'LOOPFILTER_SELECTIVELY' as the stats collected during encoding stage
+  // decides the filter level. Loop-filtering is disabled in case
+  // of non-reference frames and for frames with intra block copy tool enabled.
+  AV1_COMMON *cm = &cpi->common;
+  const int use_cdef = is_cdef_used(cm);
+  const int use_restoration = is_restoration_used(cm);
+  const int skip_postproc_filtering =
+      should_skip_postproc_filtering(cpi, use_cdef, use_restoration);
+  // TODO(deepa.kg @ittiam.com) Enable for rt speed 5, 6.
+  cpi->mt_info.pipeline_lpf_mt_with_enc =
+      (cpi->oxcf.mode == REALTIME) && (cpi->oxcf.speed >= 7) &&
+      (cpi->sf.lpf_sf.lpf_pick == LPF_PICK_FROM_Q) &&
+      (cpi->oxcf.algo_cfg.loopfilter_control != LOOPFILTER_SELECTIVELY) &&
+      !cpi->ppi->rtc_ref.non_reference_frame && !cm->features.allow_intrabc &&
+      !skip_postproc_filtering;
+
+  if (!cpi->mt_info.pipeline_lpf_mt_with_enc) return;
+
+  set_postproc_filter_default_params(cm);
+
+  if (!is_loopfilter_used(cm)) return;
+
+  const LPF_PICK_METHOD method = cpi->sf.lpf_sf.lpf_pick;
+  assert(method == LPF_PICK_FROM_Q);
+  assert(cpi->oxcf.algo_cfg.loopfilter_control != LOOPFILTER_SELECTIVELY);
+
+  av1_pick_filter_level(cpi->source, cpi, method);
+
+  struct loopfilter *lf = &cm->lf;
+  const int plane_start = 0;
+  const int plane_end = av1_num_planes(cm);
+  int planes_to_lf[MAX_MB_PLANE];
+  if ((lf->filter_level[PLANE_TYPE_Y] || lf->filter_level[PLANE_TYPE_UV]) &&
+      check_planes_to_loop_filter(lf, planes_to_lf, plane_start, plane_end)) {
+    int lpf_opt_level = get_lpf_opt_level(&cpi->sf);
+    assert(lpf_opt_level == 2);
+
+    const int start_mi_row = 0;
+    const int end_mi_row = start_mi_row + cm->mi_params.mi_rows;
+
+    av1_loop_filter_frame_init(cm, plane_start, plane_end);
+
+    assert(cpi->mt_info.num_mod_workers[MOD_ENC] ==
+           cpi->mt_info.num_mod_workers[MOD_LPF]);
+    loop_filter_frame_mt_init(cm, start_mi_row, end_mi_row, planes_to_lf,
+                              cpi->mt_info.num_mod_workers[MOD_LPF],
+                              &cpi->mt_info.lf_row_sync, lpf_opt_level,
+                              cm->seq_params->mib_size_log2);
+  }
+}
+
 void av1_encode_tiles_row_mt(AV1_COMP *cpi) {
   AV1_COMMON *const cm = &cpi->common;
   MultiThreadInfo *const mt_info = &cpi->mt_info;
   AV1EncRowMultiThreadInfo *const enc_row_mt = &mt_info->enc_row_mt;
   const int tile_cols = cm->tiles.cols;
   const int tile_rows = cm->tiles.rows;
+  const int sb_rows = get_sb_rows_in_frame(cm);
   int *thread_id_to_tile_id = enc_row_mt->thread_id_to_tile_id;
   int max_sb_rows = 0, max_sb_cols = 0;
   int num_workers = mt_info->num_mod_workers[MOD_ENC];
@@ -1541,6 +1667,8 @@
     av1_alloc_tile_data(cpi);
   }
 
+  lpf_pipeline_mt_init(cpi);
+
   av1_init_tile_data(cpi);
 
   compute_max_sb_rows_cols(cpi, &max_sb_rows, &max_sb_cols);
@@ -1548,7 +1676,8 @@
   if (enc_row_mt->allocated_tile_cols != tile_cols ||
       enc_row_mt->allocated_tile_rows != tile_rows ||
       enc_row_mt->allocated_rows != max_sb_rows ||
-      enc_row_mt->allocated_cols != (max_sb_cols - 1)) {
+      enc_row_mt->allocated_cols != (max_sb_cols - 1) ||
+      enc_row_mt->allocated_sb_rows != sb_rows) {
     av1_row_mt_mem_dealloc(cpi);
     row_mt_mem_alloc(cpi, max_sb_rows, max_sb_cols,
                      cpi->oxcf.algo_cfg.cdf_update_mode);
@@ -1556,6 +1685,8 @@
 
   memset(thread_id_to_tile_id, -1,
          sizeof(*thread_id_to_tile_id) * MAX_NUM_THREADS);
+  memset(enc_row_mt->num_tile_cols_done, 0,
+         sizeof(*enc_row_mt->num_tile_cols_done) * sb_rows);
 
   for (int tile_row = 0; tile_row < tile_rows; tile_row++) {
     for (int tile_col = 0; tile_col < tile_cols; tile_col++) {
diff --git a/av1/encoder/ethread.h b/av1/encoder/ethread.h
index a1de988..25589f8 100644
--- a/av1/encoder/ethread.h
+++ b/av1/encoder/ethread.h
@@ -23,6 +23,8 @@
   struct AV1_COMP *cpi;
   struct ThreadData *td;
   struct ThreadData *original_td;
+  AV1LfSync *lf_sync;
+  LFWorkerData *lf_data;
   int start;
   int thread_id;
 } EncWorkerData;
diff --git a/av1/encoder/picklpf.c b/av1/encoder/picklpf.c
index 21aa5c6..90c3c1a 100644
--- a/av1/encoder/picklpf.c
+++ b/av1/encoder/picklpf.c
@@ -202,7 +202,6 @@
   (void)sd;
 
   lf->sharpness_level = 0;
-  cpi->td.mb.rdmult = cpi->rd.RDMULT;
 
   if (cpi->oxcf.tune_cfg.content == AOM_CONTENT_SCREEN &&
       cpi->oxcf.q_cfg.aq_mode == CYCLIC_REFRESH_AQ &&