summaryrefslogtreecommitdiffstats
path: root/drivers/misc/habanalabs
diff options
context:
space:
mode:
authorfarah kassabri <fkassabri@habana.ai>2021-05-26 10:43:36 +0300
committerOded Gabbay <ogabbay@kernel.org>2021-08-29 09:47:48 +0300
commite4cdccd2ec0d178219dce0707aa2a63746119e3f (patch)
treed20e660a0cd524e0e5cd1ce04aaa4bfcf177b492 /drivers/misc/habanalabs
parentdadf17abb7245d9556591d8cc78bf57462e3b20a (diff)
downloadlinux-e4cdccd2ec0d178219dce0707aa2a63746119e3f.tar.bz2
habanalabs: add support for encapsulated signals submission
This commit is the second part of the encapsulated signals feature. It contains the driver support for submission of cs with encapsulated signals and the wait for them. Signed-off-by: farah kassabri <fkassabri@habana.ai> Reviewed-by: Oded Gabbay <ogabbay@kernel.org> Signed-off-by: Oded Gabbay <ogabbay@kernel.org>
Diffstat (limited to 'drivers/misc/habanalabs')
-rw-r--r--drivers/misc/habanalabs/common/command_submission.c343
-rw-r--r--drivers/misc/habanalabs/common/context.c14
-rw-r--r--drivers/misc/habanalabs/common/habanalabs.h49
-rw-r--r--drivers/misc/habanalabs/common/hw_queue.c104
-rw-r--r--drivers/misc/habanalabs/gaudi/gaudi.c145
-rw-r--r--drivers/misc/habanalabs/goya/goya.c2
6 files changed, 308 insertions, 349 deletions
diff --git a/drivers/misc/habanalabs/common/command_submission.c b/drivers/misc/habanalabs/common/command_submission.c
index 84032b1bae5c..d71bd48cbc44 100644
--- a/drivers/misc/habanalabs/common/command_submission.c
+++ b/drivers/misc/habanalabs/common/command_submission.c
@@ -106,38 +106,6 @@ int hl_gen_sob_mask(u16 sob_base, u8 sob_mask, u8 *mask)
return 0;
}
-static void sob_reset_work(struct work_struct *work)
-{
- struct hl_cs_compl *hl_cs_cmpl =
- container_of(work, struct hl_cs_compl, sob_reset_work);
- struct hl_device *hdev = hl_cs_cmpl->hdev;
-
- /*
- * A signal CS can get completion while the corresponding wait
- * for signal CS is on its way to the PQ. The wait for signal CS
- * will get stuck if the signal CS incremented the SOB to its
- * max value and there are no pending (submitted) waits on this
- * SOB.
- * We do the following to void this situation:
- * 1. The wait for signal CS must get a ref for the signal CS as
- * soon as possible in cs_ioctl_signal_wait() and put it
- * before being submitted to the PQ but after it incremented
- * the SOB refcnt in init_signal_wait_cs().
- * 2. Signal/Wait for signal CS will decrement the SOB refcnt
- * here.
- * These two measures guarantee that the wait for signal CS will
- * reset the SOB upon completion rather than the signal CS and
- * hence the above scenario is avoided.
- */
- kref_put(&hl_cs_cmpl->hw_sob->kref, hl_sob_reset);
-
- if (hl_cs_cmpl->type == CS_TYPE_COLLECTIVE_WAIT)
- hdev->asic_funcs->reset_sob_group(hdev,
- hl_cs_cmpl->sob_group);
-
- kfree(hl_cs_cmpl);
-}
-
static void hl_fence_release(struct kref *kref)
{
struct hl_fence *fence =
@@ -578,12 +546,17 @@ static inline void cs_release_sob_reset_handler(struct hl_device *hdev,
spin_lock(&hl_cs_cmpl->lock);
+ /*
+ * we get refcount upon reservation of signals or signal/wait cs for the
+ * hw_sob object, and need to put it when the first staged cs
+ * (which cotains the encaps signals) or cs signal/wait is completed.
+ */
if ((hl_cs_cmpl->type == CS_TYPE_SIGNAL) ||
(hl_cs_cmpl->type == CS_TYPE_WAIT) ||
(hl_cs_cmpl->type == CS_TYPE_COLLECTIVE_WAIT) ||
(!!hl_cs_cmpl->encaps_signals)) {
dev_dbg(hdev->dev,
- "CS 0x%llx type %d finished, sob_id: %d, sob_val: 0x%x\n",
+ "CS 0x%llx type %d finished, sob_id: %d, sob_val: %u\n",
hl_cs_cmpl->cs_seq,
hl_cs_cmpl->type,
hl_cs_cmpl->hw_sob->sob_id,
@@ -664,8 +637,20 @@ static void cs_do_release(struct kref *ref)
list_del(&cs->staged_cs_node);
spin_unlock(&hdev->cs_mirror_lock);
}
+
+ /* decrement refcount to handle when first staged cs
+ * with encaps signals is completed.
+ */
+ if (hl_cs_cmpl->encaps_signals)
+ kref_put(&hl_cs_cmpl->encaps_sig_hdl->refcount,
+ hl_encaps_handle_do_release);
}
+ if ((cs->type == CS_TYPE_WAIT || cs->type == CS_TYPE_COLLECTIVE_WAIT)
+ && cs->encaps_signals)
+ kref_put(&cs->encaps_sig_hdl->refcount,
+ hl_encaps_handle_do_release);
+
out:
/* Must be called before hl_ctx_put because inside we use ctx to get
* the device
@@ -798,6 +783,7 @@ static int allocate_cs(struct hl_device *hdev, struct hl_ctx *ctx,
cs->completed = false;
cs->type = cs_type;
cs->timestamp = !!(flags & HL_CS_FLAGS_TIMESTAMP);
+ cs->encaps_signals = !!(flags & HL_CS_FLAGS_ENCAP_SIGNALS);
cs->timeout_jiffies = timeout;
cs->skip_reset_on_timeout =
hdev->skip_reset_on_timeout ||
@@ -808,9 +794,9 @@ static int allocate_cs(struct hl_device *hdev, struct hl_ctx *ctx,
kref_init(&cs->refcount);
spin_lock_init(&cs->job_lock);
- cs_cmpl = kmalloc(sizeof(*cs_cmpl), GFP_ATOMIC);
+ cs_cmpl = kzalloc(sizeof(*cs_cmpl), GFP_ATOMIC);
if (!cs_cmpl)
- cs_cmpl = kmalloc(sizeof(*cs_cmpl), GFP_KERNEL);
+ cs_cmpl = kzalloc(sizeof(*cs_cmpl), GFP_KERNEL);
if (!cs_cmpl) {
atomic64_inc(&ctx->cs_counters.out_of_mem_drop_cnt);
@@ -834,9 +820,7 @@ static int allocate_cs(struct hl_device *hdev, struct hl_ctx *ctx,
cs_cmpl->hdev = hdev;
cs_cmpl->type = cs->type;
- cs_cmpl->encaps_signals = false;
spin_lock_init(&cs_cmpl->lock);
- INIT_WORK(&cs_cmpl->sob_reset_work, sob_reset_work);
cs->fence = &cs_cmpl->base_fence;
spin_lock(&ctx->cs_lock);
@@ -933,18 +917,6 @@ void hl_cs_rollback_all(struct hl_device *hdev)
force_complete_multi_cs(hdev);
}
-void hl_pending_cb_list_flush(struct hl_ctx *ctx)
-{
- struct hl_pending_cb *pending_cb, *tmp;
-
- list_for_each_entry_safe(pending_cb, tmp,
- &ctx->pending_cb_list, cb_node) {
- list_del(&pending_cb->cb_node);
- hl_cb_put(pending_cb->cb);
- kfree(pending_cb);
- }
-}
-
static void
wake_pending_user_interrupt_threads(struct hl_user_interrupt *interrupt)
{
@@ -1225,7 +1197,8 @@ static int hl_cs_copy_chunk_array(struct hl_device *hdev,
}
static int cs_staged_submission(struct hl_device *hdev, struct hl_cs *cs,
- u64 sequence, u32 flags)
+ u64 sequence, u32 flags,
+ u32 encaps_signal_handle)
{
if (!(flags & HL_CS_FLAGS_STAGED_SUBMISSION))
return 0;
@@ -1237,6 +1210,9 @@ static int cs_staged_submission(struct hl_device *hdev, struct hl_cs *cs,
/* Staged CS sequence is the first CS sequence */
INIT_LIST_HEAD(&cs->staged_cs_node);
cs->staged_sequence = cs->sequence;
+
+ if (cs->encaps_signals)
+ cs->encaps_sig_hdl_id = encaps_signal_handle;
} else {
/* User sequence will be validated in 'hl_hw_queue_schedule_cs'
* under the cs_mirror_lock
@@ -1254,7 +1230,7 @@ static int cs_staged_submission(struct hl_device *hdev, struct hl_cs *cs,
static int cs_ioctl_default(struct hl_fpriv *hpriv, void __user *chunks,
u32 num_chunks, u64 *cs_seq, u32 flags,
- u32 timeout)
+ u32 encaps_signals_handle, u32 timeout)
{
bool staged_mid, int_queues_only = true;
struct hl_device *hdev = hpriv->hdev;
@@ -1293,7 +1269,8 @@ static int cs_ioctl_default(struct hl_fpriv *hpriv, void __user *chunks,
hl_debugfs_add_cs(cs);
- rc = cs_staged_submission(hdev, cs, user_sequence, flags);
+ rc = cs_staged_submission(hdev, cs, user_sequence, flags,
+ encaps_signals_handle);
if (rc)
goto free_cs_object;
@@ -1431,130 +1408,6 @@ out:
return rc;
}
-static int pending_cb_create_job(struct hl_device *hdev, struct hl_ctx *ctx,
- struct hl_cs *cs, struct hl_cb *cb, u32 size, u32 hw_queue_id)
-{
- struct hw_queue_properties *hw_queue_prop;
- struct hl_cs_counters_atomic *cntr;
- struct hl_cs_job *job;
-
- hw_queue_prop = &hdev->asic_prop.hw_queues_props[hw_queue_id];
- cntr = &hdev->aggregated_cs_counters;
-
- job = hl_cs_allocate_job(hdev, hw_queue_prop->type, true);
- if (!job) {
- atomic64_inc(&ctx->cs_counters.out_of_mem_drop_cnt);
- atomic64_inc(&cntr->out_of_mem_drop_cnt);
- dev_err(hdev->dev, "Failed to allocate a new job\n");
- return -ENOMEM;
- }
-
- job->id = 0;
- job->cs = cs;
- job->user_cb = cb;
- atomic_inc(&job->user_cb->cs_cnt);
- job->user_cb_size = size;
- job->hw_queue_id = hw_queue_id;
- job->patched_cb = job->user_cb;
- job->job_cb_size = job->user_cb_size;
-
- /* increment refcount as for external queues we get completion */
- cs_get(cs);
-
- cs->jobs_in_queue_cnt[job->hw_queue_id]++;
-
- list_add_tail(&job->cs_node, &cs->job_list);
-
- hl_debugfs_add_job(hdev, job);
-
- return 0;
-}
-
-static int hl_submit_pending_cb(struct hl_fpriv *hpriv)
-{
- struct hl_device *hdev = hpriv->hdev;
- struct hl_ctx *ctx = hpriv->ctx;
- struct hl_pending_cb *pending_cb, *tmp;
- struct list_head local_cb_list;
- struct hl_cs *cs;
- struct hl_cb *cb;
- u32 hw_queue_id;
- u32 cb_size;
- int process_list, rc = 0;
-
- if (list_empty(&ctx->pending_cb_list))
- return 0;
-
- process_list = atomic_cmpxchg(&ctx->thread_pending_cb_token, 1, 0);
-
- /* Only a single thread is allowed to process the list */
- if (!process_list)
- return 0;
-
- if (list_empty(&ctx->pending_cb_list))
- goto free_pending_cb_token;
-
- /* move all list elements to a local list */
- INIT_LIST_HEAD(&local_cb_list);
- spin_lock(&ctx->pending_cb_lock);
- list_for_each_entry_safe(pending_cb, tmp, &ctx->pending_cb_list,
- cb_node)
- list_move_tail(&pending_cb->cb_node, &local_cb_list);
- spin_unlock(&ctx->pending_cb_lock);
-
- rc = allocate_cs(hdev, ctx, CS_TYPE_DEFAULT, ULLONG_MAX, &cs, 0,
- hdev->timeout_jiffies);
- if (rc)
- goto add_list_elements;
-
- hl_debugfs_add_cs(cs);
-
- /* Iterate through pending cb list, create jobs and add to CS */
- list_for_each_entry(pending_cb, &local_cb_list, cb_node) {
- cb = pending_cb->cb;
- cb_size = pending_cb->cb_size;
- hw_queue_id = pending_cb->hw_queue_id;
-
- rc = pending_cb_create_job(hdev, ctx, cs, cb, cb_size,
- hw_queue_id);
- if (rc)
- goto free_cs_object;
- }
-
- rc = hl_hw_queue_schedule_cs(cs);
- if (rc) {
- if (rc != -EAGAIN)
- dev_err(hdev->dev,
- "Failed to submit CS %d.%llu (%d)\n",
- ctx->asid, cs->sequence, rc);
- goto free_cs_object;
- }
-
- /* pending cb was scheduled successfully */
- list_for_each_entry_safe(pending_cb, tmp, &local_cb_list, cb_node) {
- list_del(&pending_cb->cb_node);
- kfree(pending_cb);
- }
-
- cs_put(cs);
-
- goto free_pending_cb_token;
-
-free_cs_object:
- cs_rollback(hdev, cs);
- cs_put(cs);
-add_list_elements:
- spin_lock(&ctx->pending_cb_lock);
- list_for_each_entry_safe_reverse(pending_cb, tmp, &local_cb_list,
- cb_node)
- list_move(&pending_cb->cb_node, &ctx->pending_cb_list);
- spin_unlock(&ctx->pending_cb_lock);
-free_pending_cb_token:
- atomic_set(&ctx->thread_pending_cb_token, 1);
-
- return rc;
-}
-
static int hl_cs_ctx_switch(struct hl_fpriv *hpriv, union hl_cs_args *args,
u64 *cs_seq)
{
@@ -1604,7 +1457,7 @@ static int hl_cs_ctx_switch(struct hl_fpriv *hpriv, union hl_cs_args *args,
rc = 0;
} else {
rc = cs_ioctl_default(hpriv, chunks, num_chunks,
- cs_seq, 0, hdev->timeout_jiffies);
+ cs_seq, 0, 0, hdev->timeout_jiffies);
}
mutex_unlock(&hpriv->restore_phase_mutex);
@@ -1705,7 +1558,15 @@ int hl_cs_signal_sob_wraparound_handler(struct hl_device *hdev, u32 q_idx,
return -EINVAL;
}
- prop->next_sob_val = count;
+ /*
+ * next_sob_val always points to the next available signal
+ * in the sob, so in encaps signals it will be the next one
+ * after reserving the required amount.
+ */
+ if (encaps_sig)
+ prop->next_sob_val = count + 1;
+ else
+ prop->next_sob_val = count;
/* only two SOBs are currently in use */
prop->curr_sob_offset = other_sob_offset;
@@ -1721,11 +1582,11 @@ int hl_cs_signal_sob_wraparound_handler(struct hl_device *hdev, u32 q_idx,
* in addition, if we have combination of cs signal and
* encaps, and at the point we need to reset the sob there was
* no more reservations and only signal cs keep coming,
- * in such case we need to signal_cs to put the refcount and
+ * in such case we need signal_cs to put the refcount and
* reset the sob.
*/
if (other_sob->need_reset)
- kref_put(&other_sob->kref, hl_sob_reset);
+ hw_sob_put(other_sob);
if (encaps_sig) {
/* set reset indication for the sob */
@@ -1743,12 +1604,18 @@ int hl_cs_signal_sob_wraparound_handler(struct hl_device *hdev, u32 q_idx,
}
static int cs_ioctl_extract_signal_seq(struct hl_device *hdev,
- struct hl_cs_chunk *chunk, u64 *signal_seq, struct hl_ctx *ctx)
+ struct hl_cs_chunk *chunk, u64 *signal_seq, struct hl_ctx *ctx,
+ bool encaps_signals)
{
u64 *signal_seq_arr = NULL;
u32 size_to_copy, signal_seq_arr_len;
int rc = 0;
+ if (encaps_signals) {
+ *signal_seq = chunk->encaps_signal_seq;
+ return 0;
+ }
+
signal_seq_arr_len = chunk->num_signal_seq_arr;
/* currently only one signal seq is supported */
@@ -1773,7 +1640,7 @@ static int cs_ioctl_extract_signal_seq(struct hl_device *hdev,
return -ENOMEM;
}
- size_to_copy = chunk->num_signal_seq_arr * sizeof(*signal_seq_arr);
+ size_to_copy = signal_seq_arr_len * sizeof(*signal_seq_arr);
if (copy_from_user(signal_seq_arr,
u64_to_user_ptr(chunk->signal_seq_arr),
size_to_copy)) {
@@ -1795,8 +1662,8 @@ out:
}
static int cs_ioctl_signal_wait_create_jobs(struct hl_device *hdev,
- struct hl_ctx *ctx, struct hl_cs *cs, enum hl_queue_type q_type,
- u32 q_idx)
+ struct hl_ctx *ctx, struct hl_cs *cs,
+ enum hl_queue_type q_type, u32 q_idx, u32 encaps_signal_offset)
{
struct hl_cs_counters_atomic *cntr;
struct hl_cs_job *job;
@@ -1834,6 +1701,9 @@ static int cs_ioctl_signal_wait_create_jobs(struct hl_device *hdev,
job->user_cb_size = cb_size;
job->hw_queue_id = q_idx;
+ if ((cs->type == CS_TYPE_WAIT || cs->type == CS_TYPE_COLLECTIVE_WAIT)
+ && cs->encaps_signals)
+ job->encaps_sig_wait_offset = encaps_signal_offset;
/*
* No need in parsing, user CB is the patched CB.
* We call hl_cb_destroy() out of two reasons - we don't need the CB in
@@ -1906,7 +1776,7 @@ static int cs_ioctl_reserve_signals(struct hl_fpriv *hpriv,
mgr = &hpriv->ctx->sig_mgr;
spin_lock(&mgr->lock);
- hdl_id = idr_alloc(&mgr->handles, handle, 1, 0, GFP_KERNEL);
+ hdl_id = idr_alloc(&mgr->handles, handle, 1, 0, GFP_ATOMIC);
spin_unlock(&mgr->lock);
if (hdl_id < 0) {
@@ -1931,14 +1801,13 @@ static int cs_ioctl_reserve_signals(struct hl_fpriv *hpriv,
* value, if yes then switch sob.
*/
rc = hl_cs_signal_sob_wraparound_handler(hdev, q_idx, &hw_sob, count,
- true);
+ true);
if (rc) {
dev_err(hdev->dev, "Failed to switch SOB\n");
hdev->asic_funcs->hw_queues_unlock(hdev);
rc = -EINVAL;
goto remove_idr;
}
-
/* set the hw_sob to the handle after calling the sob wraparound handler
* since sob could have changed.
*/
@@ -1956,9 +1825,9 @@ static int cs_ioctl_reserve_signals(struct hl_fpriv *hpriv,
*handle_id = hdl_id;
dev_dbg(hdev->dev,
- "Signals reserved, sob_id: %d, sob addr: 0x%x, sob val: 0x%x, q_idx: %d, hdl_id: %d\n",
+ "Signals reserved, sob_id: %d, sob addr: 0x%x, last sob_val: %u, q_idx: %d, hdl_id: %d\n",
hw_sob->sob_id, handle->hw_sob->sob_addr,
- prop->next_sob_val, q_idx, hdl_id);
+ prop->next_sob_val - 1, q_idx, hdl_id);
goto out;
remove_idr:
@@ -2041,7 +1910,12 @@ static int cs_ioctl_signal_wait(struct hl_fpriv *hpriv, enum hl_cs_type cs_type,
void __user *chunks, u32 num_chunks,
u64 *cs_seq, u32 flags, u32 timeout)
{
+ struct hl_cs_encaps_sig_handle *encaps_sig_hdl = NULL;
+ bool handle_found = false, is_wait_cs = false,
+ wait_cs_submitted = false,
+ cs_encaps_signals = false;
struct hl_cs_chunk *cs_chunk_array, *chunk;
+ bool staged_cs_with_encaps_signals = false;
struct hw_queue_properties *hw_queue_prop;
struct hl_device *hdev = hpriv->hdev;
struct hl_cs_compl *sig_waitcs_cmpl;
@@ -2050,7 +1924,6 @@ static int cs_ioctl_signal_wait(struct hl_fpriv *hpriv, enum hl_cs_type cs_type,
struct hl_fence *sig_fence = NULL;
struct hl_ctx *ctx = hpriv->ctx;
enum hl_queue_type q_type;
- bool is_wait_cs = false;
struct hl_cs *cs;
u64 signal_seq;
int rc;
@@ -2102,13 +1975,58 @@ static int cs_ioctl_signal_wait(struct hl_fpriv *hpriv, enum hl_cs_type cs_type,
collective_engine_id = chunk->collective_engine_id;
}
- if (cs_type == CS_TYPE_WAIT || cs_type == CS_TYPE_COLLECTIVE_WAIT) {
- is_wait_cs = true;
+ is_wait_cs = !!(cs_type == CS_TYPE_WAIT ||
+ cs_type == CS_TYPE_COLLECTIVE_WAIT);
- rc = cs_ioctl_extract_signal_seq(hdev, chunk, &signal_seq, ctx);
+ cs_encaps_signals = !!(flags & HL_CS_FLAGS_ENCAP_SIGNALS);
+
+ if (is_wait_cs) {
+ rc = cs_ioctl_extract_signal_seq(hdev, chunk, &signal_seq,
+ ctx, cs_encaps_signals);
if (rc)
goto free_cs_chunk_array;
+ if (cs_encaps_signals) {
+ /* check if cs sequence has encapsulated
+ * signals handle
+ */
+ struct idr *idp;
+ u32 id;
+
+ spin_lock(&ctx->sig_mgr.lock);
+ idp = &ctx->sig_mgr.handles;
+ idr_for_each_entry(idp, encaps_sig_hdl, id) {
+ if (encaps_sig_hdl->cs_seq == signal_seq) {
+ handle_found = true;
+ /* get refcount to protect removing
+ * this handle from idr, needed when
+ * multiple wait cs are used with offset
+ * to wait on reserved encaps signals.
+ */
+ kref_get(&encaps_sig_hdl->refcount);
+ break;
+ }
+ }
+ spin_unlock(&ctx->sig_mgr.lock);
+
+ if (!handle_found) {
+ dev_err(hdev->dev, "Cannot find encapsulated signals handle for seq 0x%llx\n",
+ signal_seq);
+ rc = -EINVAL;
+ goto free_cs_chunk_array;
+ }
+
+ /* validate also the signal offset value */
+ if (chunk->encaps_signal_offset >
+ encaps_sig_hdl->count) {
+ dev_err(hdev->dev, "offset(%u) value exceed max reserved signals count(%u)!\n",
+ chunk->encaps_signal_offset,
+ encaps_sig_hdl->count);
+ rc = -EINVAL;
+ goto free_cs_chunk_array;
+ }
+ }
+
sig_fence = hl_ctx_get_fence(ctx, signal_seq);
if (IS_ERR(sig_fence)) {
atomic64_inc(&ctx->cs_counters.validation_drop_cnt);
@@ -2129,11 +2047,16 @@ static int cs_ioctl_signal_wait(struct hl_fpriv *hpriv, enum hl_cs_type cs_type,
sig_waitcs_cmpl =
container_of(sig_fence, struct hl_cs_compl, base_fence);
- if (sig_waitcs_cmpl->type != CS_TYPE_SIGNAL) {
+ staged_cs_with_encaps_signals = !!
+ (sig_waitcs_cmpl->type == CS_TYPE_DEFAULT &&
+ (flags & HL_CS_FLAGS_ENCAP_SIGNALS));
+
+ if (sig_waitcs_cmpl->type != CS_TYPE_SIGNAL &&
+ !staged_cs_with_encaps_signals) {
atomic64_inc(&ctx->cs_counters.validation_drop_cnt);
atomic64_inc(&cntr->validation_drop_cnt);
dev_err(hdev->dev,
- "CS seq 0x%llx is not of a signal CS\n",
+ "CS seq 0x%llx is not of a signal/encaps-signal CS\n",
signal_seq);
hl_fence_put(sig_fence);
rc = -EINVAL;
@@ -2159,9 +2082,18 @@ static int cs_ioctl_signal_wait(struct hl_fpriv *hpriv, enum hl_cs_type cs_type,
/*
* Save the signal CS fence for later initialization right before
* hanging the wait CS on the queue.
+ * for encaps signals case, we save the cs sequence and handle pointer
+ * for later initialization.
*/
- if (cs_type == CS_TYPE_WAIT || cs_type == CS_TYPE_COLLECTIVE_WAIT)
+ if (is_wait_cs) {
cs->signal_fence = sig_fence;
+ /* store the handle pointer, so we don't have to
+ * look for it again, later on the flow
+ * when we need to set SOB info in hw_queue.
+ */
+ if (cs->encaps_signals)
+ cs->encaps_sig_hdl = encaps_sig_hdl;
+ }
hl_debugfs_add_cs(cs);
@@ -2169,10 +2101,11 @@ static int cs_ioctl_signal_wait(struct hl_fpriv *hpriv, enum hl_cs_type cs_type,
if (cs_type == CS_TYPE_WAIT || cs_type == CS_TYPE_SIGNAL)
rc = cs_ioctl_signal_wait_create_jobs(hdev, ctx, cs, q_type,
- q_idx);
+ q_idx, chunk->encaps_signal_offset);
else if (cs_type == CS_TYPE_COLLECTIVE_WAIT)
rc = hdev->asic_funcs->collective_wait_create_jobs(hdev, ctx,
- cs, q_idx, collective_engine_id);
+ cs, q_idx, collective_engine_id,
+ chunk->encaps_signal_offset);
else {
atomic64_inc(&ctx->cs_counters.validation_drop_cnt);
atomic64_inc(&cntr->validation_drop_cnt);
@@ -2198,6 +2131,8 @@ static int cs_ioctl_signal_wait(struct hl_fpriv *hpriv, enum hl_cs_type cs_type,
}
rc = HL_CS_STATUS_SUCCESS;
+ if (is_wait_cs)
+ wait_cs_submitted = true;
goto put_cs;
free_cs_object:
@@ -2208,6 +2143,10 @@ put_cs:
/* We finished with the CS in this function, so put the ref */
cs_put(cs);
free_cs_chunk_array:
+ if (!wait_cs_submitted && cs_encaps_signals && handle_found &&
+ is_wait_cs)
+ kref_put(&encaps_sig_hdl->refcount,
+ hl_encaps_handle_do_release);
kfree(cs_chunk_array);
out:
return rc;
@@ -2231,10 +2170,6 @@ int hl_cs_ioctl(struct hl_fpriv *hpriv, void *data)
if (rc)
goto out;
- rc = hl_submit_pending_cb(hpriv);
- if (rc)
- goto out;
-
cs_type = hl_cs_get_cs_type(args->in.cs_flags &
~HL_CS_FLAGS_FORCE_RESTORE);
chunks = (void __user *) (uintptr_t) args->in.chunks_execute;
@@ -2269,7 +2204,9 @@ int hl_cs_ioctl(struct hl_fpriv *hpriv, void *data)
break;
default:
rc = cs_ioctl_default(hpriv, chunks, num_chunks, &cs_seq,
- args->in.cs_flags, timeout);
+ args->in.cs_flags,
+ args->in.encaps_sig_handle_id,
+ timeout);
break;
}
out:
diff --git a/drivers/misc/habanalabs/common/context.c b/drivers/misc/habanalabs/common/context.c
index abbba4194d3b..007f3a48601c 100644
--- a/drivers/misc/habanalabs/common/context.c
+++ b/drivers/misc/habanalabs/common/context.c
@@ -16,7 +16,10 @@ void hl_encaps_handle_do_release(struct kref *ref)
struct hl_ctx *ctx = handle->hdev->compute_ctx;
struct hl_encaps_signals_mgr *mgr = &ctx->sig_mgr;
+ spin_lock(&mgr->lock);
idr_remove(&mgr->handles, handle->id);
+ spin_unlock(&mgr->lock);
+
kfree(handle);
}
@@ -33,7 +36,10 @@ static void hl_encaps_handle_do_release_sob(struct kref *ref)
*/
hw_sob_put(handle->hw_sob);
+ spin_lock(&mgr->lock);
idr_remove(&mgr->handles, handle->id);
+ spin_unlock(&mgr->lock);
+
kfree(handle);
}
@@ -67,11 +73,6 @@ static void hl_ctx_fini(struct hl_ctx *ctx)
struct hl_device *hdev = ctx->hdev;
int i;
- /* Release all allocated pending cb's, those cb's were never
- * scheduled so it is safe to release them here
- */
- hl_pending_cb_list_flush(ctx);
-
/* Release all allocated HW block mapped list entries and destroy
* the mutex.
*/
@@ -198,11 +199,8 @@ int hl_ctx_init(struct hl_device *hdev, struct hl_ctx *ctx, bool is_kernel_ctx)
kref_init(&ctx->refcount);
ctx->cs_sequence = 1;
- INIT_LIST_HEAD(&ctx->pending_cb_list);
- spin_lock_init(&ctx->pending_cb_lock);
spin_lock_init(&ctx->cs_lock);
atomic_set(&ctx->thread_ctx_switch_token, 1);
- atomic_set(&ctx->thread_pending_cb_token, 1);
ctx->thread_ctx_switch_wait_token = 0;
ctx->cs_pending = kcalloc(hdev->asic_prop.max_pending_cs,
sizeof(struct hl_fence *),
diff --git a/drivers/misc/habanalabs/common/habanalabs.h b/drivers/misc/habanalabs/common/habanalabs.h
index 81b6825e0c1c..b72fcc9255aa 100644
--- a/drivers/misc/habanalabs/common/habanalabs.h
+++ b/drivers/misc/habanalabs/common/habanalabs.h
@@ -605,11 +605,11 @@ struct hl_fence {
/**
* struct hl_cs_compl - command submission completion object.
- * @sob_reset_work: workqueue object to run SOB reset flow.
* @base_fence: hl fence object.
* @lock: spinlock to protect fence.
* @hdev: habanalabs device structure.
* @hw_sob: the H/W SOB used in this signal/wait CS.
+ * @encaps_sig_hdl: encaps signals hanlder.
* @cs_seq: command submission sequence number.
* @type: type of the CS - signal/wait.
* @sob_val: the SOB value that is used in this signal/wait CS.
@@ -618,11 +618,11 @@ struct hl_fence {
* encaps signals or not.
*/
struct hl_cs_compl {
- struct work_struct sob_reset_work;
struct hl_fence base_fence;
spinlock_t lock;
struct hl_device *hdev;
struct hl_hw_sob *hw_sob;
+ struct hl_cs_encaps_sig_handle *encaps_sig_hdl;
u64 cs_seq;
enum hl_cs_type type;
u16 sob_val;
@@ -1267,8 +1267,9 @@ struct hl_asic_funcs {
u64 (*get_device_time)(struct hl_device *hdev);
int (*collective_wait_init_cs)(struct hl_cs *cs);
int (*collective_wait_create_jobs)(struct hl_device *hdev,
- struct hl_ctx *ctx, struct hl_cs *cs, u32 wait_queue_id,
- u32 collective_engine_id);
+ struct hl_ctx *ctx, struct hl_cs *cs,
+ u32 wait_queue_id, u32 collective_engine_id,
+ u32 encaps_signal_offset);
u64 (*scramble_addr)(struct hl_device *hdev, u64 addr);
u64 (*descramble_addr)(struct hl_device *hdev, u64 addr);
void (*ack_protection_bits_errors)(struct hl_device *hdev);
@@ -1340,20 +1341,6 @@ struct hl_cs_counters_atomic {
};
/**
- * struct hl_pending_cb - pending command buffer structure
- * @cb_node: cb node in pending cb list
- * @cb: command buffer to send in next submission
- * @cb_size: command buffer size
- * @hw_queue_id: destination queue id
- */
-struct hl_pending_cb {
- struct list_head cb_node;
- struct hl_cb *cb;
- u32 cb_size;
- u32 hw_queue_id;
-};
-
-/**
* struct hl_ctx - user/kernel context.
* @mem_hash: holds mapping from virtual address to virtual memory area
* descriptor (hl_vm_phys_pg_list or hl_userptr).
@@ -1369,8 +1356,6 @@ struct hl_pending_cb {
* MMU hash or walking the PGT requires talking this lock.
* @hw_block_list_lock: protects the HW block memory list.
* @debugfs_list: node in debugfs list of contexts.
- * pending_cb_list: list of pending command buffers waiting to be sent upon
- * next user command submission context.
* @hw_block_mem_list: list of HW block virtual mapped addresses.
* @cs_counters: context command submission counters.
* @cb_va_pool: device VA pool for command buffers which are mapped to the
@@ -1381,17 +1366,11 @@ struct hl_pending_cb {
* index to cs_pending array.
* @dram_default_hops: array that holds all hops addresses needed for default
* DRAM mapping.
- * @pending_cb_lock: spinlock to protect pending cb list
* @cs_lock: spinlock to protect cs_sequence.
* @dram_phys_mem: amount of used physical DRAM memory by this context.
* @thread_ctx_switch_token: token to prevent multiple threads of the same
* context from running the context switch phase.
* Only a single thread should run it.
- * @thread_pending_cb_token: token to prevent multiple threads from processing
- * the pending CB list. Only a single thread should
- * process the list since it is protected by a
- * spinlock and we don't want to halt the entire
- * command submission sequence.
* @thread_ctx_switch_wait_token: token to prevent the threads that didn't run
* the context switch phase from moving to their
* execution phase before the context switch phase
@@ -1411,18 +1390,15 @@ struct hl_ctx {
struct mutex mmu_lock;
struct mutex hw_block_list_lock;
struct list_head debugfs_list;
- struct list_head pending_cb_list;
struct list_head hw_block_mem_list;
struct hl_cs_counters_atomic cs_counters;
struct gen_pool *cb_va_pool;
struct hl_encaps_signals_mgr sig_mgr;
u64 cs_sequence;
u64 *dram_default_hops;
- spinlock_t pending_cb_lock;
spinlock_t cs_lock;
atomic64_t dram_phys_mem;
atomic_t thread_ctx_switch_token;
- atomic_t thread_pending_cb_token;
u32 thread_ctx_switch_wait_token;
u32 asid;
u32 handle;
@@ -1485,12 +1461,14 @@ struct hl_userptr {
* @mirror_node : node in device mirror list of command submissions.
* @staged_cs_node: node in the staged cs list.
* @debugfs_list: node in debugfs list of command submissions.
+ * @encaps_sig_hdl: holds the encaps signals handle.
* @sequence: the sequence number of this CS.
* @staged_sequence: the sequence of the staged submission this CS is part of,
* relevant only if staged_cs is set.
* @timeout_jiffies: cs timeout in jiffies.
* @submission_time_jiffies: submission time of the cs
* @type: CS_TYPE_*.
+ * @encaps_sig_hdl_id: encaps signals handle id, set for the first staged cs.
* @submitted: true if CS was submitted to H/W.
* @completed: true if CS was completed by device.
* @timedout : true if CS was timedout.
@@ -1504,6 +1482,7 @@ struct hl_userptr {
* @staged_cs: true if this CS is part of a staged submission.
* @skip_reset_on_timeout: true if we shall not reset the device in case
* timeout occurs (debug scenario).
+ * @encaps_signals: true if this CS has encaps reserved signals.
*/
struct hl_cs {
u16 *jobs_in_queue_cnt;
@@ -1518,11 +1497,13 @@ struct hl_cs {
struct list_head mirror_node;
struct list_head staged_cs_node;
struct list_head debugfs_list;
+ struct hl_cs_encaps_sig_handle *encaps_sig_hdl;
u64 sequence;
u64 staged_sequence;
u64 timeout_jiffies;
u64 submission_time_jiffies;
enum hl_cs_type type;
+ u32 encaps_sig_hdl_id;
u8 submitted;
u8 completed;
u8 timedout;
@@ -1533,6 +1514,7 @@ struct hl_cs {
u8 staged_first;
u8 staged_cs;
u8 skip_reset_on_timeout;
+ u8 encaps_signals;
};
/**
@@ -1552,6 +1534,8 @@ struct hl_cs {
* @hw_queue_id: the id of the H/W queue this job is submitted to.
* @user_cb_size: the actual size of the CB we got from the user.
* @job_cb_size: the actual size of the CB that we put on the queue.
+ * @encaps_sig_wait_offset: encapsulated signals offset, which allow user
+ * to wait on part of the reserved signals.
* @is_kernel_allocated_cb: true if the CB handle we got from the user holds a
* handle to a kernel-allocated CB object, false
* otherwise (SRAM/DRAM/host address).
@@ -1576,6 +1560,7 @@ struct hl_cs_job {
u32 hw_queue_id;
u32 user_cb_size;
u32 job_cb_size;
+ u32 encaps_sig_wait_offset;
u8 is_kernel_allocated_cb;
u8 contains_dma_pkt;
};
@@ -2794,7 +2779,6 @@ int hl_cb_va_pool_init(struct hl_ctx *ctx);
void hl_cb_va_pool_fini(struct hl_ctx *ctx);
void hl_cs_rollback_all(struct hl_device *hdev);
-void hl_pending_cb_list_flush(struct hl_ctx *ctx);
struct hl_cs_job *hl_cs_allocate_job(struct hl_device *hdev,
enum hl_queue_type queue_type, bool is_kernel_allocated_cb);
void hl_sob_reset_error(struct kref *ref);
@@ -2935,9 +2919,12 @@ int hl_set_voltage(struct hl_device *hdev,
int sensor_index, u32 attr, long value);
int hl_set_current(struct hl_device *hdev,
int sensor_index, u32 attr, long value);
-void hl_encaps_handle_do_release(struct kref *ref);
void hw_sob_get(struct hl_hw_sob *hw_sob);
void hw_sob_put(struct hl_hw_sob *hw_sob);
+void hl_encaps_handle_do_release(struct kref *ref);
+void hl_hw_queue_encaps_sig_set_sob_info(struct hl_device *hdev,
+ struct hl_cs *cs, struct hl_cs_job *job,
+ struct hl_cs_compl *cs_cmpl);
void hl_release_pending_user_interrupts(struct hl_device *hdev);
int hl_cs_signal_sob_wraparound_handler(struct hl_device *hdev, u32 q_idx,
struct hl_hw_sob **hw_sob, u32 count, bool encaps_sig);
diff --git a/drivers/misc/habanalabs/common/hw_queue.c b/drivers/misc/habanalabs/common/hw_queue.c
index 9a59b8e9bf53..6d3beccad91b 100644
--- a/drivers/misc/habanalabs/common/hw_queue.c
+++ b/drivers/misc/habanalabs/common/hw_queue.c
@@ -416,7 +416,7 @@ static int init_signal_cs(struct hl_device *hdev,
cs_cmpl->sob_val = prop->next_sob_val;
dev_dbg(hdev->dev,
- "generate signal CB, sob_id: %d, sob val: 0x%x, q_idx: %d, seq: %llu\n",
+ "generate signal CB, sob_id: %d, sob val: %u, q_idx: %d, seq: %llu\n",
cs_cmpl->hw_sob->sob_id, cs_cmpl->sob_val, q_idx,
cs_cmpl->cs_seq);
@@ -432,12 +432,31 @@ static int init_signal_cs(struct hl_device *hdev,
return rc;
}
+void hl_hw_queue_encaps_sig_set_sob_info(struct hl_device *hdev,
+ struct hl_cs *cs, struct hl_cs_job *job,
+ struct hl_cs_compl *cs_cmpl)
+{
+ struct hl_cs_encaps_sig_handle *handle = cs->encaps_sig_hdl;
+
+ cs_cmpl->hw_sob = handle->hw_sob;
+
+ /* Note that encaps_sig_wait_offset was validated earlier in the flow
+ * for offset value which exceeds the max reserved signal count.
+ * always decrement 1 of the offset since when the user
+ * set offset 1 for example he mean to wait only for the first
+ * signal only, which will be pre_sob_val, and if he set offset 2
+ * then the value required is (pre_sob_val + 1) and so on...
+ */
+ cs_cmpl->sob_val = handle->pre_sob_val +
+ (job->encaps_sig_wait_offset - 1);
+}
+
static int init_wait_cs(struct hl_device *hdev, struct hl_cs *cs,
struct hl_cs_job *job, struct hl_cs_compl *cs_cmpl)
{
- struct hl_cs_compl *signal_cs_cmpl;
- struct hl_sync_stream_properties *prop;
struct hl_gen_wait_properties wait_prop;
+ struct hl_sync_stream_properties *prop;
+ struct hl_cs_compl *signal_cs_cmpl;
u32 q_idx;
q_idx = job->hw_queue_id;
@@ -447,9 +466,23 @@ static int init_wait_cs(struct hl_device *hdev, struct hl_cs *cs,
struct hl_cs_compl,
base_fence);
- /* copy the SOB id and value of the signal CS */
- cs_cmpl->hw_sob = signal_cs_cmpl->hw_sob;
- cs_cmpl->sob_val = signal_cs_cmpl->sob_val;
+ if (cs->encaps_signals) {
+ /* use the encaps signal handle stored earlier in the flow
+ * and set the SOB information from the encaps
+ * signals handle
+ */
+ hl_hw_queue_encaps_sig_set_sob_info(hdev, cs, job, cs_cmpl);
+
+ dev_dbg(hdev->dev, "Wait for encaps signals handle, qidx(%u), CS sequence(%llu), sob val: 0x%x, offset: %u\n",
+ cs->encaps_sig_hdl->q_idx,
+ cs->encaps_sig_hdl->cs_seq,
+ cs_cmpl->sob_val,
+ job->encaps_sig_wait_offset);
+ } else {
+ /* Copy the SOB id and value of the signal CS */
+ cs_cmpl->hw_sob = signal_cs_cmpl->hw_sob;
+ cs_cmpl->sob_val = signal_cs_cmpl->sob_val;
+ }
/* check again if the signal cs already completed.
* if yes then don't send any wait cs since the hw_sob
@@ -523,6 +556,59 @@ static int init_signal_wait_cs(struct hl_cs *cs)
return rc;
}
+static int encaps_sig_first_staged_cs_handler
+ (struct hl_device *hdev, struct hl_cs *cs)
+{
+ struct hl_cs_compl *cs_cmpl =
+ container_of(cs->fence,
+ struct hl_cs_compl, base_fence);
+ struct hl_cs_encaps_sig_handle *encaps_sig_hdl;
+ struct hl_encaps_signals_mgr *mgr;
+ int rc = 0;
+
+ mgr = &hdev->compute_ctx->sig_mgr;
+
+ spin_lock(&mgr->lock);
+ encaps_sig_hdl = idr_find(&mgr->handles, cs->encaps_sig_hdl_id);
+ if (encaps_sig_hdl) {
+ /*
+ * Set handler CS sequence,
+ * the CS which contains the encapsulated signals.
+ */
+ encaps_sig_hdl->cs_seq = cs->sequence;
+ /* store the handle and set encaps signal indication,
+ * to be used later in cs_do_release to put the last
+ * reference to encaps signals handlers.
+ */
+ cs_cmpl->encaps_signals = true;
+ cs_cmpl->encaps_sig_hdl = encaps_sig_hdl;
+
+ /* set hw_sob pointer in completion object
+ * since it's used in cs_do_release flow to put
+ * refcount to sob
+ */
+ cs_cmpl->hw_sob = encaps_sig_hdl->hw_sob;
+ cs_cmpl->sob_val = encaps_sig_hdl->pre_sob_val +
+ encaps_sig_hdl->count;
+
+ dev_dbg(hdev->dev, "CS seq (%llu) added to encaps signal handler id (%u), count(%u), qidx(%u), sob(%u), val(%u)\n",
+ cs->sequence, encaps_sig_hdl->id,
+ encaps_sig_hdl->count,
+ encaps_sig_hdl->q_idx,
+ cs_cmpl->hw_sob->sob_id,
+ cs_cmpl->sob_val);
+
+ } else {
+ dev_err(hdev->dev, "encaps handle id(%u) wasn't found!\n",
+ cs->encaps_sig_hdl_id);
+ rc = -EINVAL;
+ }
+
+ spin_unlock(&mgr->lock);
+
+ return rc;
+}
+
/*
* hl_hw_queue_schedule_cs - schedule a command submission
* @cs: pointer to the CS
@@ -602,6 +688,12 @@ int hl_hw_queue_schedule_cs(struct hl_cs *cs)
}
+ if (cs->encaps_signals && cs->staged_first) {
+ rc = encaps_sig_first_staged_cs_handler(hdev, cs);
+ if (rc)
+ goto unroll_cq_resv;
+ }
+
spin_lock(&hdev->cs_mirror_lock);
/* Verify staged CS exists and add to the staged list */
diff --git a/drivers/misc/habanalabs/gaudi/gaudi.c b/drivers/misc/habanalabs/gaudi/gaudi.c
index ae1a8b4e694c..6187e2e802bc 100644
--- a/drivers/misc/habanalabs/gaudi/gaudi.c
+++ b/drivers/misc/habanalabs/gaudi/gaudi.c
@@ -456,8 +456,6 @@ static int gaudi_memset_device_memory(struct hl_device *hdev, u64 addr,
u32 size, u64 val);
static int gaudi_memset_registers(struct hl_device *hdev, u64 reg_base,
u32 num_regs, u32 val);
-static int gaudi_schedule_register_memset(struct hl_device *hdev,
- u32 hw_queue_id, u64 reg_base, u32 num_regs, u32 val);
static int gaudi_run_tpc_kernel(struct hl_device *hdev, u64 tpc_kernel,
u32 tpc_id);
static int gaudi_mmu_clear_pgt_range(struct hl_device *hdev);
@@ -468,7 +466,6 @@ static u32 gaudi_gen_signal_cb(struct hl_device *hdev, void *data, u16 sob_id,
u32 size, bool eb);
static u32 gaudi_gen_wait_cb(struct hl_device *hdev,
struct hl_gen_wait_properties *prop);
-
static inline enum hl_collective_mode
get_collective_mode(struct hl_device *hdev, u32 queue_id)
{
@@ -1068,17 +1065,11 @@ static void gaudi_sob_group_hw_reset(struct kref *ref)
struct gaudi_hw_sob_group *hw_sob_group =
container_of(ref, struct gaudi_hw_sob_group, kref);
struct hl_device *hdev = hw_sob_group->hdev;
- u64 base_addr;
- int rc;
+ int i;
- base_addr = CFG_BASE + mmSYNC_MNGR_W_S_SYNC_MNGR_OBJS_SOB_OBJ_0 +
- hw_sob_group->base_sob_id * 4;
- rc = gaudi_schedule_register_memset(hdev, hw_sob_group->queue_id,
- base_addr, NUMBER_OF_SOBS_IN_GRP, 0);
- if (rc)
- dev_err(hdev->dev,
- "failed resetting sob group - sob base %u, count %u",
- hw_sob_group->base_sob_id, NUMBER_OF_SOBS_IN_GRP);
+ for (i = 0 ; i < NUMBER_OF_SOBS_IN_GRP ; i++)
+ WREG32((mmSYNC_MNGR_W_S_SYNC_MNGR_OBJS_SOB_OBJ_0 +
+ (hw_sob_group->base_sob_id * 4) + (i * 4)), 0);
kref_init(&hw_sob_group->kref);
}
@@ -1215,6 +1206,20 @@ static void gaudi_collective_slave_init_job(struct hl_device *hdev,
queue_id = job->hw_queue_id;
prop = &hdev->kernel_queues[queue_id].sync_stream_prop;
+ if (job->cs->encaps_signals) {
+ /* use the encaps signal handle store earlier in the flow
+ * and set the SOB information from the encaps
+ * signals handle
+ */
+ hl_hw_queue_encaps_sig_set_sob_info(hdev, job->cs, job,
+ cs_cmpl);
+
+ dev_dbg(hdev->dev, "collective wait: Sequence %llu found, sob_id: %u, wait for sob_val: %u\n",
+ job->cs->sequence,
+ cs_cmpl->hw_sob->sob_id,
+ cs_cmpl->sob_val);
+ }
+
/* Add to wait CBs using slave monitor */
wait_prop.data = (void *) job->user_cb;
wait_prop.sob_base = cs_cmpl->hw_sob->sob_id;
@@ -1225,7 +1230,7 @@ static void gaudi_collective_slave_init_job(struct hl_device *hdev,
wait_prop.size = cb_size;
dev_dbg(hdev->dev,
- "Generate slave wait CB, sob %d, val:0x%x, mon %d, q %d\n",
+ "Generate slave wait CB, sob %d, val:%x, mon %d, q %d\n",
cs_cmpl->hw_sob->sob_id, cs_cmpl->sob_val,
prop->collective_slave_mon_id, queue_id);
@@ -1257,9 +1262,14 @@ static int gaudi_collective_wait_init_cs(struct hl_cs *cs)
gaudi = hdev->asic_specific;
cprop = &gaudi->collective_props;
- /* copy the SOB id and value of the signal CS */
- cs_cmpl->hw_sob = signal_cs_cmpl->hw_sob;
- cs_cmpl->sob_val = signal_cs_cmpl->sob_val;
+ /* In encaps signals case the SOB info will be retrieved from
+ * the handle in gaudi_collective_slave_init_job.
+ */
+ if (!cs->encaps_signals) {
+ /* copy the SOB id and value of the signal CS */
+ cs_cmpl->hw_sob = signal_cs_cmpl->hw_sob;
+ cs_cmpl->sob_val = signal_cs_cmpl->sob_val;
+ }
/* check again if the signal cs already completed.
* if yes then don't send any wait cs since the hw_sob
@@ -1336,7 +1346,8 @@ static int gaudi_collective_wait_init_cs(struct hl_cs *cs)
static int gaudi_collective_wait_create_job(struct hl_device *hdev,
struct hl_ctx *ctx, struct hl_cs *cs,
- enum hl_collective_mode mode, u32 queue_id, u32 wait_queue_id)
+ enum hl_collective_mode mode, u32 queue_id, u32 wait_queue_id,
+ u32 encaps_signal_offset)
{
struct hw_queue_properties *hw_queue_prop;
struct hl_cs_counters_atomic *cntr;
@@ -1396,6 +1407,13 @@ static int gaudi_collective_wait_create_job(struct hl_device *hdev,
job->user_cb_size = cb_size;
job->hw_queue_id = queue_id;
+ /* since its guaranteed to have only one chunk in the collective wait
+ * cs, we can use this chunk to set the encapsulated signal offset
+ * in the jobs.
+ */
+ if (cs->encaps_signals)
+ job->encaps_sig_wait_offset = encaps_signal_offset;
+
/*
* No need in parsing, user CB is the patched CB.
* We call hl_cb_destroy() out of two reasons - we don't need
@@ -1424,8 +1442,9 @@ static int gaudi_collective_wait_create_job(struct hl_device *hdev,
}
static int gaudi_collective_wait_create_jobs(struct hl_device *hdev,
- struct hl_ctx *ctx, struct hl_cs *cs, u32 wait_queue_id,
- u32 collective_engine_id)
+ struct hl_ctx *ctx, struct hl_cs *cs,
+ u32 wait_queue_id, u32 collective_engine_id,
+ u32 encaps_signal_offset)
{
struct gaudi_device *gaudi = hdev->asic_specific;
struct hw_queue_properties *hw_queue_prop;
@@ -1475,7 +1494,8 @@ static int gaudi_collective_wait_create_jobs(struct hl_device *hdev,
if (i == 0) {
queue_id = wait_queue_id;
rc = gaudi_collective_wait_create_job(hdev, ctx, cs,
- HL_COLLECTIVE_MASTER, queue_id, wait_queue_id);
+ HL_COLLECTIVE_MASTER, queue_id,
+ wait_queue_id, encaps_signal_offset);
} else {
if (nic_idx < NIC_NUMBER_OF_ENGINES) {
if (gaudi->hw_cap_initialized &
@@ -1495,7 +1515,8 @@ static int gaudi_collective_wait_create_jobs(struct hl_device *hdev,
}
rc = gaudi_collective_wait_create_job(hdev, ctx, cs,
- HL_COLLECTIVE_SLAVE, queue_id, wait_queue_id);
+ HL_COLLECTIVE_SLAVE, queue_id,
+ wait_queue_id, encaps_signal_offset);
}
if (rc)
@@ -5909,78 +5930,6 @@ release_cb:
return rc;
}
-static int gaudi_schedule_register_memset(struct hl_device *hdev,
- u32 hw_queue_id, u64 reg_base, u32 num_regs, u32 val)
-{
- struct hl_ctx *ctx;
- struct hl_pending_cb *pending_cb;
- struct packet_msg_long *pkt;
- u32 cb_size, ctl;
- struct hl_cb *cb;
- int i, rc;
-
- mutex_lock(&hdev->fpriv_list_lock);
- ctx = hdev->compute_ctx;
-
- /* If no compute context available or context is going down
- * memset registers directly
- */
- if (!ctx || kref_read(&ctx->refcount) == 0) {
- rc = gaudi_memset_registers(hdev, reg_base, num_regs, val);
- mutex_unlock(&hdev->fpriv_list_lock);
- return rc;
- }
-
- mutex_unlock(&hdev->fpriv_list_lock);
-
- cb_size = (sizeof(*pkt) * num_regs) +
- sizeof(struct packet_msg_prot) * 2;
-
- if (cb_size > SZ_2M) {
- dev_err(hdev->dev, "CB size must be smaller than %uMB", SZ_2M);
- return -ENOMEM;
- }
-
- pending_cb = kzalloc(sizeof(*pending_cb), GFP_KERNEL);
- if (!pending_cb)
- return -ENOMEM;
-
- cb = hl_cb_kernel_create(hdev, cb_size, false);
- if (!cb) {
- kfree(pending_cb);
- return -EFAULT;
- }
-
- pkt = cb->kernel_address;
-
- ctl = FIELD_PREP(GAUDI_PKT_LONG_CTL_OP_MASK, 0); /* write the value */
- ctl |= FIELD_PREP(GAUDI_PKT_CTL_OPCODE_MASK, PACKET_MSG_LONG);
- ctl |= FIELD_PREP(GAUDI_PKT_CTL_EB_MASK, 1);
- ctl |= FIELD_PREP(GAUDI_PKT_CTL_RB_MASK, 1);
- ctl |= FIELD_PREP(GAUDI_PKT_CTL_MB_MASK, 1);
-
- for (i = 0; i < num_regs ; i++, pkt++) {
- pkt->ctl = cpu_to_le32(ctl);
- pkt->value = cpu_to_le32(val);
- pkt->addr = cpu_to_le64(reg_base + (i * 4));
- }
-
- hl_cb_destroy(hdev, &hdev->kernel_cb_mgr, cb->id << PAGE_SHIFT);
-
- pending_cb->cb = cb;
- pending_cb->cb_size = cb_size;
- /* The queue ID MUST be an external queue ID. Otherwise, we will
- * have undefined behavior
- */
- pending_cb->hw_queue_id = hw_queue_id;
-
- spin_lock(&ctx->pending_cb_lock);
- list_add_tail(&pending_cb->cb_node, &ctx->pending_cb_list);
- spin_unlock(&ctx->pending_cb_lock);
-
- return 0;
-}
-
static int gaudi_restore_sm_registers(struct hl_device *hdev)
{
u64 base_addr;
@@ -9031,16 +8980,12 @@ static u32 gaudi_gen_wait_cb(struct hl_device *hdev,
static void gaudi_reset_sob(struct hl_device *hdev, void *data)
{
struct hl_hw_sob *hw_sob = (struct hl_hw_sob *) data;
- int rc;
dev_dbg(hdev->dev, "reset SOB, q_idx: %d, sob_id: %d\n", hw_sob->q_idx,
hw_sob->sob_id);
- rc = gaudi_schedule_register_memset(hdev, hw_sob->q_idx,
- CFG_BASE + mmSYNC_MNGR_W_S_SYNC_MNGR_OBJS_SOB_OBJ_0 +
- hw_sob->sob_id * 4, 1, 0);
- if (rc)
- dev_err(hdev->dev, "failed resetting sob %u", hw_sob->sob_id);
+ WREG32(mmSYNC_MNGR_W_S_SYNC_MNGR_OBJS_SOB_OBJ_0 +
+ hw_sob->sob_id * 4, 0);
kref_init(&hw_sob->kref);
}
diff --git a/drivers/misc/habanalabs/goya/goya.c b/drivers/misc/habanalabs/goya/goya.c
index 8a689bf42397..c070cd14753e 100644
--- a/drivers/misc/habanalabs/goya/goya.c
+++ b/drivers/misc/habanalabs/goya/goya.c
@@ -5487,7 +5487,7 @@ static int goya_collective_wait_init_cs(struct hl_cs *cs)
static int goya_collective_wait_create_jobs(struct hl_device *hdev,
struct hl_ctx *ctx, struct hl_cs *cs, u32 wait_queue_id,
- u32 collective_engine_id)
+ u32 collective_engine_id, u32 encaps_signal_offset)
{
return -EINVAL;
}