Throughput of flag LASP_PARALLEL. If not set, some downstream code will be broken as JobQueues will not go empty for this case. For all code in the library, a serial implementation works off the not present job queue

This commit is contained in:
Anne de Jong 2021-09-15 21:10:27 +02:00
parent b159f4d144
commit 9983ca6096
9 changed files with 495 additions and 427 deletions

View File

@ -3,7 +3,8 @@
// Author: J.A. de Jong - ASCEE
//
// Description: Implementation of an equalizer using the Second Order Sections
// filter bank implementation.
// filter bank implementation. Applies all filterbanks in parallel and
// recombines to create a single output signal.
#pragma once
#ifndef LASP_EQ_H
#define LASP_EQ_H
@ -53,8 +54,6 @@ void Eq_setLevels(Eq* eq, const vd* levels);
void Eq_free(Eq* eq);
#endif // LASP_EQ_H
// //////////////////////////////////////////////////////////////////////

View File

@ -30,12 +30,13 @@ typedef struct {
} Job;
typedef struct JobQueue_s {
#ifdef LASP_PARALLEL
pthread_mutex_t mutex;
pthread_cond_t cv_plus; /**< Condition variable for the
* "workers". */
pthread_cond_t cv_minus; /**< Condition variable for the
* main thread. */
#endif
Job* jobs; /**< Pointer to job vector */
us max_jobs; /**< Stores the maximum number of
* items */
@ -75,20 +76,24 @@ void print_job_queue(JobQueue* jq) {
feTRACE(15);
}
#ifdef LASP_PARALLEL
#define LOCK_MUTEX \
/* Lock the mutex to let the threads wait initially */ \
int rv = pthread_mutex_lock(&jq->mutex); \
if(rv !=0) { \
WARN("Mutex lock failed"); \
}
#define UNLOCK_MUTEX \
rv = pthread_mutex_unlock(&jq->mutex); \
if(rv !=0) { \
WARN("Mutex unlock failed"); \
}
#else
#define LOCK_MUTEX
#define UNLOCK_MUTEX
#endif // LASP_PARALLEL
JobQueue* JobQueue_alloc(const us max_jobs) {
TRACE(15,"JobQueue_alloc");
if(max_jobs > LASP_MAX_NUM_CHANNELS) {
@ -118,6 +123,7 @@ JobQueue* JobQueue_alloc(const us max_jobs) {
j++;
}
#ifdef LASP_PARALLEL
/* Initialize thread mutex */
int rv = pthread_mutex_init(&jq->mutex,NULL);
if(rv !=0) {
@ -136,6 +142,7 @@ JobQueue* JobQueue_alloc(const us max_jobs) {
return NULL;
}
#endif // LASP_PARALLEL
/* print_job_queue(jq); */
return jq;
}
@ -153,6 +160,7 @@ void JobQueue_free(JobQueue* jq) {
a_free(jq->jobs);
#ifdef LASP_PARALLEL
/* Destroy the mutexes and condition variables */
rv = pthread_mutex_destroy(&jq->mutex);
if(rv != 0){
@ -170,6 +178,7 @@ void JobQueue_free(JobQueue* jq) {
WARN("Condition variable destruction failed. "
"Do not know what to do.");
}
#endif // LASP_PARALLEL
}
@ -185,6 +194,7 @@ int JobQueue_push(JobQueue* jq,void* job_ptr) {
us max_jobs = jq->max_jobs;
#ifdef LASP_PARALLEL
/* Check if queue is full */
while(count_jobs(jq) == max_jobs) {
@ -194,6 +204,13 @@ int JobQueue_push(JobQueue* jq,void* job_ptr) {
WARN("Condition variable wait failed");
}
}
#else
/* If job queue is full, not in parallel, we just fail to add something
* without waiting*/
if(count_jobs(jq) == max_jobs) {
return LASP_FAILURE;
}
#endif // LASP_PARALLEL
dbgassert(count_jobs(jq) != max_jobs,
"Queue cannot be full!");
@ -214,6 +231,7 @@ int JobQueue_push(JobQueue* jq,void* job_ptr) {
j->job_ptr = job_ptr;
j->ready = true;
#ifdef LASP_PARALLEL
/* Notify worker threads that a new job has arrived */
if(count_jobs(jq) == max_jobs) {
/* Notify ALL threads. Action required! */
@ -229,8 +247,8 @@ int JobQueue_push(JobQueue* jq,void* job_ptr) {
if(rv !=0) {
WARN("Condition variable signal failed");
}
}
#endif // LASP_PARALLEL
/* print_job_queue(jq); */
@ -244,25 +262,33 @@ void* JobQueue_assign(JobQueue* jq) {
LOCK_MUTEX;
/* Wait until a job is available */
Job* j;
#ifdef LASP_PARALLEL
/* Wait until a job is available */
while ((j=get_ready_job(jq))==NULL) {
TRACE(15,"JobQueue_assign: no ready job");
pthread_cond_wait(&jq->cv_plus,&jq->mutex);
}
#else
if(count_jobs(jq) == 0) { return NULL; }
else { j = get_ready_job(jq); }
#endif // LASP_PARALLEL
TRACE(16,"JobQueue_assign: found ready job. Assigned to:");
#ifdef LASP_DEBUG
#ifdef LASP_DEBUG
#ifdef LASP_PARALLEL
pthread_t thisthread = pthread_self();
iVARTRACE(16,thisthread);
#endif
#endif
#endif
/* print_job_queue(jq); */
/* Find a job from the queue, assign it and return it */
j->running = true;
#ifdef LASP_PARALLEL
if(count_jobs(jq) > 1) {
/* Signal different thread that there is more work to do */
rv = pthread_cond_signal(&jq->cv_plus);
@ -270,6 +296,7 @@ void* JobQueue_assign(JobQueue* jq) {
WARN("Condition variable broadcast failed");
}
}
#endif
UNLOCK_MUTEX;
@ -303,15 +330,18 @@ void JobQueue_done(JobQueue* jq,void* job_ptr) {
/* print_job_queue(jq); */
#ifdef LASP_PARALLEL
/* Job done, broadcast this */
rv = pthread_cond_signal(&jq->cv_minus);
if(rv !=0) {
WARN("Condition variable broadcast failed");
}
#endif
UNLOCK_MUTEX;
}
#ifdef LASP_PARALLEL
void JobQueue_wait_alldone(JobQueue* jq) {
TRACE(15,"JobQueue_wait_alldone");
dbgassert(jq,NULLPTRDEREF "jq in JobQueue_wait_alldone");
@ -331,6 +361,7 @@ void JobQueue_wait_alldone(JobQueue* jq) {
UNLOCK_MUTEX;
}
#endif
//////////////////////////////////////////////////////////////////////

View File

@ -33,7 +33,8 @@ void JobQueue_free(JobQueue* jq);
/**
* Pops a job from the queue. Waits indefinitely until some job is
* available.
* available. If in parallel mode. In serial mode, it returns NULL if no job
* is available (LASP_PARALLEL compilation flag not set).
*
* @param jq: JobQueue handle
* @return Pointer to the job, NULL on error.
@ -68,7 +69,10 @@ int JobQueue_push(JobQueue* jq,void* job_ptr);
* until all task workers are finished.
*
*/
#ifdef LASP_PARALLEL
void JobQueue_wait_alldone(JobQueue*);
#endif // LASP_PARALLEL
#endif // MQ_H
//////////////////////////////////////////////////////////////////////

View File

@ -1,10 +1,10 @@
#include "lasp_nprocs.h"
#ifdef MS_WIN64
#include <windows.h>
#else
// Used for obtaining the number of processors
#include <sys/sysinfo.h>
#endif
#include "lasp_nprocs.h"
us getNumberOfProcs() {
#if MS_WIN64
@ -12,7 +12,7 @@ us getNumberOfProcs() {
SYSTEM_INFO sysinfo;
GetSystemInfo(&sysinfo);
return sysinfo.dwNumberOfProcessors;
#else
#else // Linux, easy
return get_nprocs();
#endif
}

View File

@ -23,9 +23,10 @@ typedef struct Sosfilterbank {
/// Storage for the current state of the output, first axis correspond to
/// the filter number axis, the second axis contains state coefficients
dmat state;
#ifdef LASP_PARALLEL
JobQueue* jq;
Workers* workers;
#endif // LASP_PARALLEL
} Sosfilterbank;
@ -52,7 +53,6 @@ Sosfilterbank* Sosfilterbank_create(
const us filterbank_size,
const us nsections) {
fsTRACE(15);
dbgassert(nthreads_ <= LASP_MAX_NUM_THREADS, "Illegal number of threads");
dbgassert(filterbank_size <= MAX_SOS_FILTER_BANK_SIZE,
"Illegal filterbank size. Max size is "
@ -60,8 +60,6 @@ Sosfilterbank* Sosfilterbank_create(
Sosfilterbank* fb = (Sosfilterbank*) a_malloc(sizeof(Sosfilterbank));
fb->jq = NULL;
fb->workers = NULL;
fb->filterbank_size = filterbank_size;
dbgassert(nsections < MAX_SOS_FILTER_BANK_NSECTIONS,"Illegal number of sections");
fb->nsections = nsections;
@ -89,6 +87,10 @@ Sosfilterbank* Sosfilterbank_create(
// print_dmat(&(fb->sos));
vd_free(&imp_response);
#ifdef LASP_PARALLEL
fb->jq = NULL;
fb->workers = NULL;
dbgassert(nthreads_ <= LASP_MAX_NUM_THREADS, "Illegal number of threads");
us nthreads;
us nprocs = getNumberOfProcs();
@ -118,6 +120,7 @@ Sosfilterbank* Sosfilterbank_create(
}
}
#endif // LASP_PARALLEL
feTRACE(15);
return fb;
}
@ -152,8 +155,10 @@ void Sosfilterbank_free(Sosfilterbank* fb) {
dmat_free(&(fb->sos));
dmat_free(&(fb->state));
#ifdef LASP_PARALLEL
if(fb->workers) Workers_free(fb->workers);
if(fb->jq) JobQueue_free(fb->jq);
#endif // LASP_PARALLEL
a_free(fb);
feTRACE(15);
@ -236,17 +241,24 @@ dmat Sosfilterbank_filter(Sosfilterbank* fb,const vd* xs) {
jobs[filter] = job_template;
jobs[filter].filter_no = filter;
#ifdef LASP_PARALLEL
if(fb->workers) {
assertvalidptr(fb->jq);
JobQueue_push(fb->jq, &(jobs[filter]));
} else {
#endif // LASP_PARALLEL
/* No workers, we have to do it ourselves */
filter_single(NULL,(void*) &(jobs[filter]));
#ifdef LASP_PARALLEL
}
#endif // LASP_PARALLEL
}
#ifdef LASP_PARALLEL
if(fb->workers) {
JobQueue_wait_alldone(fb->jq);
}
#endif // LASP_PARALLEL
feTRACE(15);
return ys;
}

View File

@ -2,8 +2,11 @@
//
// Author: J.A. de Jong - ASCEE
//
// Description: Implemententation of a discrete filterbank using cascaded
// second order sections (sos), also called BiQuads.
// Description: Implemententation of a discrete parallel filterbank using
// cascaded second order sections (sos) for each filter in the filterbank. In
// parallel mode, the filters are allocated over a set of Worker threads that
// actually perform the computation. In serial mode, all filters in the bank
// are computed in series.
//////////////////////////////////////////////////////////////////////
#pragma once
#ifndef LASP_FILTERBANK_H
@ -61,7 +64,7 @@ void Sosfilterbank_setFilter(Sosfilterbank* fb,const us filter_no,
* @return Filtered output in an allocated array. The number of
* columns in this array equals the number of filters in the
* filterbank. The number of output samples is equal to the number of
* input samples in x.
* input samples in x (and is equal to the number of rows in the output).
*/
dmat Sosfilterbank_filter(Sosfilterbank* fb,
const vd* x);

View File

@ -6,7 +6,12 @@
//
//////////////////////////////////////////////////////////////////////
#define TRACERPLUS (-5)
/* The code in this file is not of any use for systems that are not able to do
* simultaneous multithreading, or should be adjusted for it. It is therefore
* only compiled in case LASP_PARALLEL flag is set */
#include "lasp_worker.h"
#ifdef LASP_PARALLEL
#include "lasp_mq.h"
#include "lasp_alloc.h"
#include <pthread.h>
@ -196,5 +201,6 @@ static void* threadfcn(void* thread_global_data) {
return NULL;
}
#endif // LASP_PARALLEL
//////////////////////////////////////////////////////////////////////

View File

@ -10,8 +10,10 @@
#pragma once
#ifndef LASP_WORKER_H
#define LASP_WORKER_H
#include "lasp_config.h"
#include "lasp_types.h"
#ifdef LASP_PARALLEL
typedef struct Workers_s Workers;
typedef struct JobQueue_s JobQueue;
@ -57,5 +59,7 @@ Workers* Workers_create(const us num_workers,
*/
void Workers_free(Workers* w);
#endif // LASP_PARALLEL
#endif // LASP_WORKER_H
//////////////////////////////////////////////////////////////////////

View File

@ -5,14 +5,19 @@
// Description:
//
//////////////////////////////////////////////////////////////////////
#include "lasp_worker.h"
#include "lasp_mq.h"
#include "lasp_config.h"
#include "lasp_tracer.h"
#include "lasp_assert.h"
#include <unistd.h>
#ifdef LASP_PARALLEL
#include "lasp_worker.h"
#include "lasp_mq.h"
static void* walloc(void*);
static int worker(void*,void*);
static void wfree(void*);
#endif // LASP_PARALLEL
int main() {
@ -21,6 +26,7 @@ int main() {
iVARTRACE(15,getTracerLevel());
#ifdef LASP_PARALLEL
us njobs = 4;
JobQueue* jq = JobQueue_alloc(njobs);
dbgassert(jq,NULLPTRDEREF);
@ -42,8 +48,10 @@ int main() {
Workers_free(w);
JobQueue_free(jq);
#endif // LASP_PARALLEL
return 0;
}
#ifdef LASP_PARALLEL
static void* walloc(void* data) {
TRACE(15,"WALLOC");
uVARTRACE(15,(us) data);
@ -63,6 +71,7 @@ static void wfree(void* w_data) {
TRACE(15,"wfree");
}
#endif // LASP_PARALLEL
//////////////////////////////////////////////////////////////////////