diff --git a/src/audio/audio_stream.c b/src/audio/audio_stream.c index 53fc62885c25..96f02cff8b90 100644 --- a/src/audio/audio_stream.c +++ b/src/audio/audio_stream.c @@ -37,9 +37,14 @@ static int audio_stream_commit_buffer(struct sof_sink __sparse_cache *sink, size struct audio_stream __sparse_cache *audio_stream = attr_container_of(sink, struct audio_stream __sparse_cache, sink_api, __sparse_cache); + struct comp_buffer __sparse_cache *buffer_c = + attr_container_of(audio_stream, struct comp_buffer __sparse_cache, + stream, __sparse_cache); - if (commit_size) + if (commit_size) { + buffer_stream_writeback(buffer_c, commit_size); audio_stream_produce(audio_stream, commit_size); + } return 0; } @@ -60,9 +65,15 @@ static int audio_stream_get_data(struct sof_source __sparse_cache *source, size_ attr_container_of(source, struct audio_stream __sparse_cache, source_api, __sparse_cache); + struct comp_buffer __sparse_cache *buffer_c = + attr_container_of(audio_stream, struct comp_buffer __sparse_cache, + stream, __sparse_cache); + if (req_size > audio_stream_get_data_available(source)) return -ENODATA; + buffer_stream_invalidate(buffer_c, req_size); + /* get circular buffer parameters */ *data_ptr = audio_stream->r_ptr; *buffer_start = audio_stream->addr; diff --git a/src/audio/dp_queue.c b/src/audio/dp_queue.c new file mode 100644 index 000000000000..817946f8cfe8 --- /dev/null +++ b/src/audio/dp_queue.c @@ -0,0 +1,391 @@ +// SPDX-License-Identifier: BSD-3-Clause +// +// Copyright(c) 2021 Intel Corporation. All rights reserved. +// + +#include +#include +#include +#include + +#include +#include + +struct dp_queue { + struct sof_source source_api; /**< src api handler */ + struct sof_sink sink_api; /**< sink api handler */ + + uint32_t ibs; + uint32_t obs; + uint32_t flags; /* DP_QUEUE_MODE_* */ + + size_t data_buffer_size; + uint8_t __sparse_cache *data_buffer; + + uint32_t write_offset; + uint32_t read_offset; + uint32_t available_data; /* amount of data ready for immediate reading */ + /* free buffer space + * NOTE! + * - when dp queue is shared, available_data + free_space DOES NOT eq data_buffer_size + * - when dp queue is not shared available_data + free_space always == data_buffer_size + */ + uint32_t free_space; + + bool hw_params_configured; + struct sof_audio_stream_params audio_stream_params; + + struct k_spinlock lock; +}; + +union lock_key { + k_spinlock_key_t spinlock; + int irq_lock; +}; + +static inline bool dp_queue_is_shared(struct dp_queue *dp_queue) +{ + return !!(dp_queue->flags & DP_QUEUE_MODE_SHARED); +} + +static inline uint8_t __sparse_cache *dp_queue_buffer_end(struct dp_queue *dp_queue) +{ + return dp_queue->data_buffer + dp_queue->data_buffer_size; +} + +static inline struct dp_queue *dp_queue_get_from_sink(struct sof_sink *sink) +{ + return container_of(sink, struct dp_queue, sink_api); +} + +static inline struct dp_queue *dp_queue_get_from_source(struct sof_source *source) +{ + return container_of(source, struct dp_queue, source_api); +} + +static inline void dp_queue_invalidate_shared(struct dp_queue *dp_queue, + void __sparse_cache *ptr, uint32_t size) +{ + /* wrap-around? */ + if ((uintptr_t)ptr + size > (uintptr_t)dp_queue_buffer_end(dp_queue)) { + /* writeback till the end of circular buffer */ + dcache_invalidate_region + (ptr, (uintptr_t)dp_queue_buffer_end(dp_queue) - (uintptr_t)ptr); + size -= (uintptr_t)dp_queue_buffer_end(dp_queue) - (uintptr_t)ptr; + ptr = (__sparse_force void __sparse_cache *)dp_queue->data_buffer; + } + /* invalidate rest of data */ + dcache_invalidate_region(ptr, size); +} + +static inline union lock_key dp_queue_lock(struct dp_queue *dp_queue) +{ + /* use cross-core spinlock in case of shared queue + * When shared, dp_queue structure is located in not cached memory + * as required by spinlock + */ + union lock_key key; + + if (dp_queue_is_shared(dp_queue)) + key.spinlock = k_spin_lock(&dp_queue->lock); + else + /* use faster irq_lock in case of not shared queue (located in cached mem) */ + key.irq_lock = irq_lock(); + + return key; +} + +static inline void dp_queue_unlock(struct dp_queue *dp_queue, union lock_key key) +{ + if (dp_queue_is_shared(dp_queue)) + k_spin_unlock(&dp_queue->lock, key.spinlock); + else + irq_unlock(key.irq_lock); +} + +struct sof_sink *dp_queue_get_sink(struct dp_queue *dp_queue) +{ + return &dp_queue->sink_api; +} + +struct sof_source *dp_queue_get_source(struct dp_queue *dp_queue) +{ + return &dp_queue->source_api; +} + +static size_t dp_queue_get_free_size(struct sof_sink *sink) +{ + struct dp_queue *dp_queue = dp_queue_get_from_sink(sink); + + return dp_queue->free_space; +} + +static int dp_queue_get_buffer(struct sof_sink *sink, size_t req_size, + void **data_ptr, void **buffer_start, size_t *buffer_size) +{ + struct dp_queue *dp_queue = dp_queue_get_from_sink(sink); + + /* dp_queue_get_free_size will return free size with adjustment for cacheline if needed */ + if (req_size > dp_queue_get_free_size(sink)) + return -ENODATA; + + /* no need to lock, just reading data that may be modified by commit_buffer only + * + * note! a sparse warning will be generated here till + * https://github.com/thesofproject/sof/issues/8006 is implemented + */ + *data_ptr = dp_queue->data_buffer + dp_queue->write_offset; + *buffer_start = dp_queue->data_buffer; + *buffer_size = dp_queue->data_buffer_size; + + /* provided buffer is an empty space, the requester will perform write operations only + * no need to invalidate cache - will be overwritten anyway + */ + return 0; +} + +static int dp_queue_commit_buffer(struct sof_sink *sink, size_t commit_size) +{ + struct dp_queue *dp_queue = dp_queue_get_from_sink(sink); + + if (commit_size) { + union lock_key key = dp_queue_lock(dp_queue); + + if (dp_queue_is_shared(dp_queue)) { + /* a shared queue. We need to go through committed cachelines one-by-one + * and if the whole cacheline is committed - writeback cache + * and mark data as available for reading + * + * first, calculate the current and last committed cacheline + * as offsets from buffer start + */ + uint32_t current_cacheline = dp_queue->write_offset / + PLATFORM_DCACHE_ALIGN; + + /* Last used cacheline may not be filled completely, calculate cacheline + * containing 1st free byte + */ + uint32_t last_cacheline = (dp_queue->write_offset + commit_size + 1) / + PLATFORM_DCACHE_ALIGN; + uint32_t total_num_of_cachelines = dp_queue->data_buffer_size / + PLATFORM_DCACHE_ALIGN; + /* wrap-around? */ + last_cacheline %= total_num_of_cachelines; + + /* now go one by one. + * if current_cacheline == last_full_cacheline - nothing to do + */ + while (current_cacheline != last_cacheline) { + /* writeback / invalidate */ + uint8_t __sparse_cache *ptr = dp_queue->data_buffer + + (current_cacheline * PLATFORM_DCACHE_ALIGN); + dcache_writeback_region(ptr, PLATFORM_DCACHE_ALIGN); + + /* mark data as available to read */ + dp_queue->available_data += PLATFORM_DCACHE_ALIGN; + /* get next cacheline */ + current_cacheline = (current_cacheline + 1) % + total_num_of_cachelines; + } + } else { + /* not shared */ + dp_queue->available_data += commit_size; + } + + /* move write pointer */ + dp_queue->free_space -= commit_size; + dp_queue->write_offset = (dp_queue->write_offset + commit_size) % + dp_queue->data_buffer_size; + + dp_queue_unlock(dp_queue, key); + } + + return 0; +} + +static size_t dp_queue_get_data_available(struct sof_source *source) +{ + struct dp_queue *dp_queue = dp_queue_get_from_source(source); + + /* access is read only, using uncached alias, no need to lock */ + return dp_queue->available_data; +} + +static int dp_queue_get_data(struct sof_source *source, size_t req_size, + void **data_ptr, void **buffer_start, size_t *buffer_size) +{ + struct dp_queue *dp_queue = dp_queue_get_from_source(source); + + /* no need to lock, just reading data */ + if (req_size > dp_queue_get_data_available(source)) + return -ENODATA; + + /* + * note! a sparse warning will be generated here till + * https://github.com/thesofproject/sof/issues/8006 is implemented + */ + *buffer_start = dp_queue->data_buffer; + *buffer_size = dp_queue->data_buffer_size; + *data_ptr = dp_queue->data_buffer + dp_queue->read_offset; + + /* clean cache in provided data range */ + if (dp_queue_is_shared(dp_queue)) + dp_queue_invalidate_shared(dp_queue, + (__sparse_force void __sparse_cache *)*data_ptr, + req_size); + + return 0; +} + +static int dp_queue_release_data(struct sof_source *source, size_t free_size) +{ + struct dp_queue *dp_queue = dp_queue_get_from_source(source); + + if (free_size) { + /* data consumed, free buffer space, no need for any special cache operations */ + union lock_key key = dp_queue_lock(dp_queue); + + dp_queue->available_data -= free_size; + dp_queue->free_space += free_size; + dp_queue->read_offset = + (dp_queue->read_offset + free_size) % dp_queue->data_buffer_size; + + dp_queue_unlock(dp_queue, key); + } + + return 0; +} + +static int dp_queue_set_ipc_params(struct dp_queue *dp_queue, + struct sof_ipc_stream_params *params, + bool force_update) +{ + if (dp_queue->hw_params_configured && !force_update) + return 0; + + dp_queue->audio_stream_params.frame_fmt = params->frame_fmt; + dp_queue->audio_stream_params.rate = params->rate; + dp_queue->audio_stream_params.channels = params->channels; + dp_queue->audio_stream_params.buffer_fmt = params->buffer_fmt; + + dp_queue->hw_params_configured = true; + + return 0; +} + +/* + * note! a sparse warning will be generated here till + * https://github.com/thesofproject/sof/issues/8006 is implemented + */ + +static int dp_queue_set_ipc_params_source(struct sof_source *source, + struct sof_ipc_stream_params *params, + bool force_update) +{ + struct dp_queue *dp_queue = dp_queue_get_from_source(source); + + return dp_queue_set_ipc_params(dp_queue, params, force_update); +} + +static int dp_queue_set_ipc_params_sink(struct sof_sink *sink, + struct sof_ipc_stream_params *params, + bool force_update) +{ + struct dp_queue *dp_queue = dp_queue_get_from_sink(sink); + + return dp_queue_set_ipc_params(dp_queue, params, force_update); +} + +/* + * note! a sparse warning will be generated here till + * https://github.com/thesofproject/sof/issues/8006 is implemented + */ +static const struct source_ops dp_queue_source_ops = { + .get_data_available = dp_queue_get_data_available, + .get_data = dp_queue_get_data, + .release_data = dp_queue_release_data, + .audio_set_ipc_params = dp_queue_set_ipc_params_source, +}; + +static const struct sink_ops dp_queue_sink_ops = { + .get_free_size = dp_queue_get_free_size, + .get_buffer = dp_queue_get_buffer, + .commit_buffer = dp_queue_commit_buffer, + .audio_set_ipc_params = dp_queue_set_ipc_params_sink, +}; + +struct dp_queue *dp_queue_create(uint32_t ibs, uint32_t obs, uint32_t flags) +{ + uint32_t align; + struct dp_queue *dp_queue; + + /* allocate DP structure */ + if (flags & DP_QUEUE_MODE_SHARED) + dp_queue = rzalloc(SOF_MEM_ZONE_RUNTIME_SHARED, 0, SOF_MEM_CAPS_RAM, + sizeof(*dp_queue)); + else + dp_queue = rzalloc(SOF_MEM_ZONE_RUNTIME, 0, SOF_MEM_CAPS_RAM, sizeof(*dp_queue)); + if (!dp_queue) + return NULL; + + dp_queue->flags = flags; + + /* initiate sink/source provide uncached pointers to audio_stream_params + * + * note! a sparse warning will be generated here till + * https://github.com/thesofproject/sof/issues/8006 is implemented + */ + source_init(dp_queue_get_source(dp_queue), &dp_queue_source_ops, + &dp_queue->audio_stream_params); + sink_init(dp_queue_get_sink(dp_queue), &dp_queue_sink_ops, + &dp_queue->audio_stream_params); + + /* set original ibs/obs as required by the module */ + sink_set_obs(&dp_queue->sink_api, ibs); + source_set_ibs(&dp_queue->source_api, obs); + + /* calculate required buffer size */ + if (dp_queue_is_shared(dp_queue)) { + ibs = MAX(ibs, PLATFORM_DCACHE_ALIGN); + obs = MAX(obs, PLATFORM_DCACHE_ALIGN); + align = PLATFORM_DCACHE_ALIGN; + dp_queue->data_buffer_size = 3 * MAX(ibs, obs) + PLATFORM_DCACHE_ALIGN; + /* reserve 1 cacheline to ensure exclusive use */ + dp_queue->free_space = dp_queue->data_buffer_size - PLATFORM_DCACHE_ALIGN; + } else { + if ((ibs % obs == 0) && (obs % ibs == 0)) + dp_queue->data_buffer_size = 2 * MAX(ibs, obs); + else + dp_queue->data_buffer_size = 3 * MAX(ibs, obs); + align = 0; + dp_queue->free_space = dp_queue->data_buffer_size; + } + dp_queue->available_data = 0; + + /* allocate data buffer - always in cached memory */ + dp_queue->data_buffer = (__sparse_force __sparse_cache void *) + rballoc_align(0, 0, dp_queue->data_buffer_size, align); + if (!dp_queue->data_buffer) + goto err; + + /* return allocated structure */ + return dp_queue; +err: + rfree(dp_queue); + return NULL; +} + +void dp_queue_free(struct dp_queue *dp_queue) +{ + dp_queue_invalidate_shared(dp_queue, + (__sparse_force void __sparse_cache *)dp_queue->data_buffer, + dp_queue->data_buffer_size); + + rfree((__sparse_force void *)dp_queue->data_buffer); + rfree(dp_queue); +} + +struct sof_audio_stream_params *dp_queue_get_audio_params(struct dp_queue *dp_queue) +{ + return &dp_queue->audio_stream_params; +} diff --git a/src/audio/module_adapter/module/generic.c b/src/audio/module_adapter/module/generic.c index 760652982245..e78a31286044 100644 --- a/src/audio/module_adapter/module/generic.c +++ b/src/audio/module_adapter/module/generic.c @@ -96,13 +96,24 @@ int module_init(struct processing_module *mod, struct module_interface *interfac } /*check interface, there must be one and only one of processing procedure */ - if (!interface->init || !interface->prepare || - !interface->reset || !interface->free || - (!!interface->process + !!interface->process_audio_stream + - !!interface->process_raw_data != 1)) { - comp_err(dev, "module_init(): comp %d is missing mandatory interfaces", - dev_comp_id(dev)); - return -EIO; + if (dev->ipc_config.proc_domain == COMP_PROCESSING_DOMAIN_LL) { + if (!interface->init || !interface->prepare || + !interface->reset || !interface->free || + (!!interface->process + !!interface->process_audio_stream + + !!interface->process_raw_data != 1)) { + comp_err(dev, "module_init(): LL comp %d is missing mandatory interfaces", + dev_comp_id(dev)); + return -EIO; + } + } else { + /* DP processing */ + if (!interface->init || !interface->prepare || + !interface->reset || !interface->free || + !interface->process || !interface->is_ready_to_process) { + comp_err(dev, "module_init(): DP comp %d is missing mandatory interfaces", + dev_comp_id(dev)); + return -EIO; + } } /* Assign interface */ @@ -282,6 +293,21 @@ int module_process_legacy(struct processing_module *mod, return 0; } +bool module_is_ready_to_process(struct processing_module *mod, + struct sof_source __sparse_cache **sources, + int num_of_sources, + struct sof_sink __sparse_cache **sinks, + int num_of_sinks) +{ + struct module_data *md = &mod->priv; + + /* LL module has to be always ready for processing */ + if (mod->dev->ipc_config.proc_domain == COMP_PROCESSING_DOMAIN_LL) + return true; + + return md->ops->is_ready_to_process(mod, sources, num_of_sources, sinks, num_of_sinks); +} + int module_process_sink_src(struct processing_module *mod, struct sof_source __sparse_cache **sources, int num_of_sources, struct sof_sink __sparse_cache **sinks, int num_of_sinks) diff --git a/src/audio/sink_api_helper.c b/src/audio/sink_api_helper.c index b98e1c3cac7f..4655fdbde347 100644 --- a/src/audio/sink_api_helper.c +++ b/src/audio/sink_api_helper.c @@ -175,3 +175,13 @@ int sink_set_alignment_constants(struct sof_sink __sparse_cache *sink, return sink->ops->set_alignment_constants(sink, byte_align, frame_align_req); return 0; } + +void sink_set_obs(struct sof_sink __sparse_cache *sink, size_t obs) +{ + sink->obs = obs; +} + +size_t sink_get_obs(struct sof_sink __sparse_cache *sink) +{ + return sink->obs; +} diff --git a/src/audio/source_api_helper.c b/src/audio/source_api_helper.c index fbdc74afa397..91985aeed969 100644 --- a/src/audio/source_api_helper.c +++ b/src/audio/source_api_helper.c @@ -165,3 +165,13 @@ int source_set_alignment_constants(struct sof_source __sparse_cache *source, return source->ops->set_alignment_constants(source, byte_align, frame_align_req); return 0; } + +void source_set_ibs(struct sof_source __sparse_cache *source, size_t ibs) +{ + source->ibs = ibs; +} + +size_t source_get_ibs(struct sof_source __sparse_cache *source) +{ + return source->ibs; +} diff --git a/src/audio/src/src.c b/src/audio/src/src.c index 94c0e0aef18a..2c6169b685d1 100644 --- a/src/audio/src/src.c +++ b/src/audio/src/src.c @@ -670,9 +670,9 @@ static int src_verify_params(struct processing_module *mod) return ret; } -static int src_get_copy_limits(struct comp_data *cd, - struct sof_source __sparse_cache *source, - struct sof_sink __sparse_cache *sink) +static bool src_get_copy_limits(struct comp_data *cd, + struct sof_source __sparse_cache *source, + struct sof_sink __sparse_cache *sink) { struct src_param *sp; struct src_stage *s1; @@ -709,9 +709,9 @@ static int src_get_copy_limits(struct comp_data *cd, } if (sp->blk_in == 0 && sp->blk_out == 0) - return -EIO; + return false; - return 0; + return true; } static int src_params_general(struct processing_module *mod, @@ -999,26 +999,31 @@ static int src_prepare(struct processing_module *mod, return src_prepare_general(mod, sources[0], sinks[0]); } + +static bool src_is_ready_to_process(struct processing_module *mod, + struct sof_source __sparse_cache **sources, int num_of_sources, + struct sof_sink __sparse_cache **sinks, int num_of_sinks) +{ + struct comp_data *cd = module_get_private_data(mod); + + return src_get_copy_limits(cd, sources[0], sinks[0]); +} + static int src_process(struct processing_module *mod, struct sof_source __sparse_cache **sources, int num_of_sources, struct sof_sink __sparse_cache **sinks, int num_of_sinks) { struct comp_data *cd = module_get_private_data(mod); - struct comp_dev *dev = mod->dev; - int ret; - comp_dbg(dev, "src_process()"); + comp_dbg(mod->dev, "src_process()"); /* src component needs 1 source and 1 sink */ - ret = src_get_copy_limits(cd, sources[0], sinks[0]); - if (ret) { - comp_dbg(dev, "No data to process."); + if (!src_get_copy_limits(cd, sources[0], sinks[0])) { + comp_dbg(mod->dev, "No data to process."); return 0; } - ret = cd->src_func(cd, sources[0], sinks[0]); - - return ret; + return cd->src_func(cd, sources[0], sinks[0]); } static int src_set_config(struct processing_module *mod, uint32_t config_id, @@ -1064,6 +1069,7 @@ static struct module_interface src_interface = { .init = src_init, .prepare = src_prepare, .process = src_process, + .is_ready_to_process = src_is_ready_to_process, .set_configuration = src_set_config, .get_configuration = src_get_config, .reset = src_reset, diff --git a/src/include/sof/audio/component.h b/src/include/sof/audio/component.h index 89e760b2dd04..61c813da8c16 100644 --- a/src/include/sof/audio/component.h +++ b/src/include/sof/audio/component.h @@ -570,15 +570,6 @@ struct comp_dev { uint32_t frames; /**< number of frames we copy to sink */ struct pipeline *pipeline; /**< pipeline we belong to */ - uint32_t min_sink_bytes; /**< min free sink buffer size measured in - * bytes required to run component's - * processing - */ - uint32_t min_source_bytes; /**< amount of data measured in bytes - * available at source buffer required - * to run component's processing - */ - struct task *task; /**< component's processing task used * 1) for components running on different core * than the rest of the pipeline diff --git a/src/include/sof/audio/dp_queue.h b/src/include/sof/audio/dp_queue.h new file mode 100644 index 000000000000..6adbd2077731 --- /dev/null +++ b/src/include/sof/audio/dp_queue.h @@ -0,0 +1,88 @@ +/* SPDX-License-Identifier: BSD-3-Clause + * + * Copyright(c) 2023 Intel Corporation. All rights reserved. + * + */ + +#ifndef __SOF_DP_QUEUE_H__ +#define __SOF_DP_QUEUE_H__ + +#include +#include +#include + +/** + * DP queue is a circular buffer providing safe consumer/producer cached operations cross cores + * + * prerequisites: + * 1) incoming and outgoing data rate MUST be the same + * 2) Both data consumer and data producer declare max chunk sizes they want to use (IBS/OBS) + * + * The queue may work in 2 modes + * 1) simple mode + * in case both receiver and sender are located on the same core and cache coherency + * does not matter. dp_queue structure is located in cached memory + * In this case DP Queue is a simple ring buffer + * Buffer size must be: + * - 2*MAX(IBS,OBS) if IBS(obs) is a multiplication of OBS(IBS) + * - 3*MAX(IBS,OBS) otherwise + * + * 2) shared mode + * In this case we need to writeback cache when new data arrive and invalidate cache on + * secondary core. That means the whole cacheline must be used exclusively by sink or by source + * incoming data will be available to use when a cacheline is filled completely + * dp_queue structure is located in shared memory + * + * buffer size is always 3*MAX(IBS,OBS,CACHELINE) + CACHELINE + * + */ + +struct dp_queue; +struct sof_audio_stream_params; + +/* DP flags */ +#define DP_QUEUE_MODE_SIMPLE 0 +#define DP_QUEUE_MODE_SHARED BIT(1) + +/** + * @param ibs input buffer size + * the size of data to be produced in 1 cycle + * the data producer declares here how much data it will produce in single cycle + * + * @param obs output buffer size + * the size of data to be consumed in 1 cycle + * the data receiver declares here how much data it will consume in single cycle + * + * @param flags a combinatin of DP_QUEUE_MODE_* flags determining working mode + * + */ +struct dp_queue *dp_queue_create(uint32_t ibs, uint32_t obs, uint32_t flags); +/** + * @brief free dp queue memory + */ +void dp_queue_free(struct dp_queue *dp_queue); + +/** + * @brief return a handler to sink API of dp_queue. + * the handler may be used by helper functions defined in sink_api.h + * note! a sparse warning will be generated here till + * https://github.com/thesofproject/sof/issues/8006 is implemented + */ +struct sof_sink *dp_queue_get_sink(struct dp_queue *dp_queue); + +/** + * @brief return a handler to source API of dp_queue + * the handler may be used by helper functions defined in source_api.h + * note! a sparse warning will be generated here till + * https://github.com/thesofproject/sof/issues/8006 is implemented + */ +struct sof_source *dp_queue_get_source(struct dp_queue *dp_queue); + +/** + * @brief this is a backdoor to get complete audio params structure from dp_queue + * it is needed till pipeline 2.0 is ready + * + */ +struct sof_audio_stream_params *dp_queue_get_audio_params(struct dp_queue *dp_queue); + +#endif /* __SOF_DP_QUEUE_H__ */ diff --git a/src/include/sof/audio/module_adapter/module/generic.h b/src/include/sof/audio/module_adapter/module/generic.h index bfcf32c92a76..83dd35d4efeb 100644 --- a/src/include/sof/audio/module_adapter/module/generic.h +++ b/src/include/sof/audio/module_adapter/module/generic.h @@ -247,6 +247,9 @@ void module_free_all_memory(struct processing_module *mod); int module_prepare(struct processing_module *mod, struct sof_source __sparse_cache **sources, int num_of_sources, struct sof_sink __sparse_cache **sinks, int num_of_sinks); +bool module_is_ready_to_process(struct processing_module *mod, + struct sof_source __sparse_cache **sources, int num_of_sources, + struct sof_sink __sparse_cache **sinks, int num_of_sinks); int module_process_sink_src(struct processing_module *mod, struct sof_source __sparse_cache **sources, int num_of_sources, struct sof_sink __sparse_cache **sinks, int num_of_sinks); diff --git a/src/include/sof/audio/module_adapter/module/module_interface.h b/src/include/sof/audio/module_adapter/module/module_interface.h index 3b915e797b29..82f7681883a8 100644 --- a/src/include/sof/audio/module_adapter/module/module_interface.h +++ b/src/include/sof/audio/module_adapter/module/module_interface.h @@ -161,6 +161,23 @@ struct module_interface { struct sof_source __sparse_cache **sources, int num_of_sources, struct sof_sink __sparse_cache **sinks, int num_of_sinks); + /** + * return true if the module is ready to process + * This procedure should check if the module is ready for immediate + * processing. + * In common case that means quick check if a module is properly + * configured and there's enough data for processing on all sources + * and enough space for result storage in all sinks + * + * NOTE! the call MUST NOT perform any time consuming operations + * + * this procedure is obligatory for DP processing modules + * if not implemented, the wrapper will simply return false + */ + bool (*is_ready_to_process)(struct processing_module *mod, + struct sof_source __sparse_cache **sources, int num_of_sources, + struct sof_sink __sparse_cache **sinks, int num_of_sinks); + /** * Module specific processing procedure * This procedure is responsible to consume diff --git a/src/include/sof/audio/sink_api.h b/src/include/sof/audio/sink_api.h index b8f1b0833f7b..07172f332555 100644 --- a/src/include/sof/audio/sink_api.h +++ b/src/include/sof/audio/sink_api.h @@ -129,6 +129,8 @@ int sink_set_rate(struct sof_sink __sparse_cache *sink, unsigned int rate); int sink_set_channels(struct sof_sink __sparse_cache *sink, unsigned int channels); int sink_set_overrun(struct sof_sink __sparse_cache *sink, bool overrun_permitted); int sink_set_buffer_fmt(struct sof_sink __sparse_cache *sink, uint32_t buffer_fmt); +void sink_set_obs(struct sof_sink __sparse_cache *sink, size_t obs); +size_t sink_get_obs(struct sof_sink __sparse_cache *sink); /** * initial set of audio parameters, provided in sof_ipc_stream_params diff --git a/src/include/sof/audio/sink_api_implementation.h b/src/include/sof/audio/sink_api_implementation.h index eb1c63ce3847..15db32152039 100644 --- a/src/include/sof/audio/sink_api_implementation.h +++ b/src/include/sof/audio/sink_api_implementation.h @@ -69,9 +69,10 @@ struct sink_ops { /** internals of sink API. NOT TO BE MODIFIED OUTSIDE OF sink_api_helper.h */ struct sof_sink { - const struct sink_ops *ops; /** operations interface */ + const struct sink_ops *ops; /** operations interface */ size_t requested_write_frag_size; /** keeps number of bytes requested by get_buffer() */ - size_t num_of_bytes_processed; /** processed bytes counter */ + size_t num_of_bytes_processed; /** processed bytes counter */ + size_t obs; /** output buffer size as declared in module bind IPC */ struct sof_audio_stream_params *audio_stream_params; /** pointer to audio params */ }; diff --git a/src/include/sof/audio/source_api.h b/src/include/sof/audio/source_api.h index ac13c286de64..bfbcf530a81b 100644 --- a/src/include/sof/audio/source_api.h +++ b/src/include/sof/audio/source_api.h @@ -140,6 +140,8 @@ int source_set_rate(struct sof_source __sparse_cache *source, unsigned int rate) int source_set_channels(struct sof_source __sparse_cache *source, unsigned int channels); int source_set_underrun(struct sof_source __sparse_cache *source, bool underrun_permitted); int source_set_buffer_fmt(struct sof_source __sparse_cache *source, uint32_t buffer_fmt); +void source_set_ibs(struct sof_source __sparse_cache *source, size_t ibs); +size_t source_get_ibs(struct sof_source __sparse_cache *source); /** * initial set of audio parameters, provided in sof_ipc_stream_params diff --git a/src/include/sof/audio/source_api_implementation.h b/src/include/sof/audio/source_api_implementation.h index 6007f8c3681c..c75330cde019 100644 --- a/src/include/sof/audio/source_api_implementation.h +++ b/src/include/sof/audio/source_api_implementation.h @@ -70,8 +70,9 @@ struct source_ops { /** internals of source API. NOT TO BE MODIFIED OUTSIDE OF source_api_helper.h */ struct sof_source { const struct source_ops *ops; - size_t requested_read_frag_size; /** keeps size of data obtained by get_data() */ - size_t num_of_bytes_processed; /** processed bytes counter */ + size_t requested_read_frag_size; /** keeps size of data obtained by get_data() */ + size_t num_of_bytes_processed; /** processed bytes counter */ + size_t ibs; /** input buffer size as declared in module bind IPC */ struct sof_audio_stream_params *audio_stream_params; }; diff --git a/src/ipc/ipc4/helper.c b/src/ipc/ipc4/helper.c index 182fb81a1c2f..835b362babc8 100644 --- a/src/ipc/ipc4/helper.c +++ b/src/ipc/ipc4/helper.c @@ -328,20 +328,14 @@ int ipc_pipeline_free(struct ipc *ipc, uint32_t comp_id) } static struct comp_buffer *ipc4_create_buffer(struct comp_dev *src, struct comp_dev *sink, - uint32_t src_queue, uint32_t dst_queue) + uint32_t src_obs, uint32_t src_queue, + uint32_t dst_queue) { - struct ipc4_base_module_cfg src_cfg; struct sof_ipc_buffer ipc_buf; - int buf_size, ret; - - ret = comp_get_attribute(src, COMP_ATTR_BASE_CONFIG, &src_cfg); - if (ret < 0) { - tr_err(&ipc_tr, "failed to get base config for src %#x", dev_comp_id(src)); - return NULL; - } + int buf_size; /* double it since obs is single buffer size */ - buf_size = src_cfg.obs * 2; + buf_size = src_obs * 2; memset(&ipc_buf, 0, sizeof(ipc_buf)); ipc_buf.size = buf_size; @@ -357,6 +351,8 @@ int ipc_comp_connect(struct ipc *ipc, ipc_pipe_comp_connect *_connect) struct comp_buffer *buffer; struct comp_dev *source; struct comp_dev *sink; + struct ipc4_base_module_cfg source_src_cfg; + struct ipc4_base_module_cfg sink_src_cfg; uint32_t flags; int src_id, sink_id; int ret; @@ -376,13 +372,33 @@ int ipc_comp_connect(struct ipc *ipc, ipc_pipe_comp_connect *_connect) if (!cpu_is_me(source->ipc_config.core) && source->ipc_config.core == sink->ipc_config.core) return ipc4_process_on_core(source->ipc_config.core, false); - buffer = ipc4_create_buffer(source, sink, bu->extension.r.src_queue, + ret = comp_get_attribute(source, COMP_ATTR_BASE_CONFIG, &source_src_cfg); + if (ret < 0) { + tr_err(&ipc_tr, "failed to get base config for module %#x", dev_comp_id(source)); + return IPC4_FAILURE; + } + + ret = comp_get_attribute(sink, COMP_ATTR_BASE_CONFIG, &sink_src_cfg); + if (ret < 0) { + tr_err(&ipc_tr, "failed to get base config for module %#x", dev_comp_id(sink)); + return IPC4_FAILURE; + } + + buffer = ipc4_create_buffer(source, sink, source_src_cfg.obs, bu->extension.r.src_queue, bu->extension.r.dst_queue); if (!buffer) { tr_err(&ipc_tr, "failed to allocate buffer to bind %d to %d", src_id, sink_id); return IPC4_OUT_OF_MEMORY; } + /* + * set ibs and obs in sink/src api of created buffer + * IBS of a buffer is OBS of source component + * OBS of a buffer is IBS of destination component + */ + source_set_ibs(audio_stream_get_source(&buffer->stream), source_src_cfg.obs); + sink_set_obs(audio_stream_get_sink(&buffer->stream), sink_src_cfg.ibs); + /* * Connect and bind the buffer to both source and sink components with the interrupts * disabled to prevent the IPC task getting preempted which could result in buffers being @@ -406,6 +422,7 @@ int ipc_comp_connect(struct ipc *ipc, ipc_pipe_comp_connect *_connect) goto e_sink_connect; } + ret = comp_bind(source, bu); if (ret < 0) goto e_src_bind; diff --git a/zephyr/CMakeLists.txt b/zephyr/CMakeLists.txt index 07558fa1f178..1d3c5c20cd18 100644 --- a/zephyr/CMakeLists.txt +++ b/zephyr/CMakeLists.txt @@ -425,6 +425,9 @@ zephyr_library_sources( lib.c ) +if(CONFIG_ZEPHYR_DP_SCHEDULER) + zephyr_library_sources(${SOF_AUDIO_PATH}/dp_queue.c) +endif() if(CONFIG_SCHEDULE_DMA_SINGLE_CHANNEL AND NOT(CONFIG_DMA_DOMAIN)) zephyr_library_sources(${SOF_SRC_PATH}/schedule/dma_single_chan_domain.c) endif()