All of lore.kernel.org
 help / color / mirror / Atom feed
* [PATCH net-next 0/6] sfc: rework locking around filter management
@ 2018-03-27 16:40 Edward Cree
  2018-03-27 16:41 ` [PATCH net-next 1/6] sfc: replace asynchronous filter operations Edward Cree
                   ` (6 more replies)
  0 siblings, 7 replies; 8+ messages in thread
From: Edward Cree @ 2018-03-27 16:40 UTC (permalink / raw)
  To: linux-net-drivers, David Miller; +Cc: netdev

The use of a spinlock to protect filter state combined with the need for a
 sleeping operation (MCDI) to apply that state to the NIC (on EF10) led to
 unfixable race conditions, around the handling of filter restoration after
 an MC reboot.
So, this patch series removes the requirement to be able to modify the SW
 filter table from atomic context, by using a workqueue to request
 asynchronous filter operations (which are needed for ARFS).  Then, the
 filter table locks are changed to mutexes, replacing the dance of spinlocks
 and 'busy' flags.  Also, a mutex is added to protect the RSS context state,
 since otherwise a similar race is possible around restoring that after an
 MC reboot.  While we're at it, fix a couple of other related bugs.

Edward Cree (6):
  sfc: replace asynchronous filter operations
  sfc: give ef10 its own rwsem in the filter table instead of
    filter_lock
  sfc: use a semaphore to lock farch filters too
  sfc: return a better error if filter insertion collides with MC reboot
  sfc: protect list of RSS contexts under a mutex
  sfc: fix flow type handling for RSS filters

 drivers/net/ethernet/sfc/ef10.c       | 545 ++++++++++++----------------------
 drivers/net/ethernet/sfc/efx.c        |  30 +-
 drivers/net/ethernet/sfc/efx.h        |  11 +-
 drivers/net/ethernet/sfc/ethtool.c    |  77 +++--
 drivers/net/ethernet/sfc/farch.c      |  80 +++--
 drivers/net/ethernet/sfc/net_driver.h |  16 +-
 drivers/net/ethernet/sfc/nic.h        |   5 +-
 drivers/net/ethernet/sfc/rx.c         | 119 +++++---
 drivers/net/ethernet/sfc/siena.c      |   1 -
 9 files changed, 400 insertions(+), 484 deletions(-)

^ permalink raw reply	[flat|nested] 8+ messages in thread

* [PATCH net-next 1/6] sfc: replace asynchronous filter operations
  2018-03-27 16:40 [PATCH net-next 0/6] sfc: rework locking around filter management Edward Cree
@ 2018-03-27 16:41 ` Edward Cree
  2018-03-27 16:42 ` [PATCH net-next 2/6] sfc: give ef10 its own rwsem in the filter table instead of filter_lock Edward Cree
                   ` (5 subsequent siblings)
  6 siblings, 0 replies; 8+ messages in thread
From: Edward Cree @ 2018-03-27 16:41 UTC (permalink / raw)
  To: linux-net-drivers, David Miller; +Cc: netdev

Instead of having an efx->type->filter_rfs_insert() method, just use
 workitems with a worker function that calls efx->type->filter_insert().
The only user of this is efx_filter_rfs(), which now queues a call to
 efx_filter_rfs_work().
Similarly, efx_filter_rfs_expire() is now a worker function called on a
 new channel->filter_work work_struct, so the method
 efx->type->filter_rfs_expire_one() is no longer called in atomic context.
 We also add a new mutex efx->rps_mutex to protect the RPS state (efx->
 rps_expire_channel, efx->rps_expire_index, and channel->rps_flow_id) so
 that the taking of efx->filter_lock can be moved to
 efx->type->filter_rfs_expire_one().
Thus, all filter table functions are now called in a sleepable context,
 allowing them to use sleeping locks in a future patch.

Signed-off-by: Edward Cree <ecree@solarflare.com>
---
 drivers/net/ethernet/sfc/ef10.c       | 161 +++-------------------------------
 drivers/net/ethernet/sfc/efx.c        |  15 +++-
 drivers/net/ethernet/sfc/efx.h        |   7 +-
 drivers/net/ethernet/sfc/farch.c      |  17 ++--
 drivers/net/ethernet/sfc/net_driver.h |  10 +--
 drivers/net/ethernet/sfc/nic.h        |   2 -
 drivers/net/ethernet/sfc/rx.c         | 119 +++++++++++++++++--------
 drivers/net/ethernet/sfc/siena.c      |   1 -
 8 files changed, 124 insertions(+), 208 deletions(-)

diff --git a/drivers/net/ethernet/sfc/ef10.c b/drivers/net/ethernet/sfc/ef10.c
index e100273b623d..bcbaba330fb5 100644
--- a/drivers/net/ethernet/sfc/ef10.c
+++ b/drivers/net/ethernet/sfc/ef10.c
@@ -4758,143 +4758,6 @@ static s32 efx_ef10_filter_get_rx_ids(struct efx_nic *efx,
 
 #ifdef CONFIG_RFS_ACCEL
 
-static efx_mcdi_async_completer efx_ef10_filter_rfs_insert_complete;
-
-static s32 efx_ef10_filter_rfs_insert(struct efx_nic *efx,
-				      struct efx_filter_spec *spec)
-{
-	struct efx_ef10_filter_table *table = efx->filter_state;
-	MCDI_DECLARE_BUF(inbuf, MC_CMD_FILTER_OP_EXT_IN_LEN);
-	struct efx_filter_spec *saved_spec;
-	unsigned int hash, i, depth = 1;
-	bool replacing = false;
-	int ins_index = -1;
-	u64 cookie;
-	s32 rc;
-
-	/* Must be an RX filter without RSS and not for a multicast
-	 * destination address (RFS only works for connected sockets).
-	 * These restrictions allow us to pass only a tiny amount of
-	 * data through to the completion function.
-	 */
-	EFX_WARN_ON_PARANOID(spec->flags !=
-			     (EFX_FILTER_FLAG_RX | EFX_FILTER_FLAG_RX_SCATTER));
-	EFX_WARN_ON_PARANOID(spec->priority != EFX_FILTER_PRI_HINT);
-	EFX_WARN_ON_PARANOID(efx_filter_is_mc_recipient(spec));
-
-	hash = efx_ef10_filter_hash(spec);
-
-	spin_lock_bh(&efx->filter_lock);
-
-	/* Find any existing filter with the same match tuple or else
-	 * a free slot to insert at.  If an existing filter is busy,
-	 * we have to give up.
-	 */
-	for (;;) {
-		i = (hash + depth) & (HUNT_FILTER_TBL_ROWS - 1);
-		saved_spec = efx_ef10_filter_entry_spec(table, i);
-
-		if (!saved_spec) {
-			if (ins_index < 0)
-				ins_index = i;
-		} else if (efx_ef10_filter_equal(spec, saved_spec)) {
-			if (table->entry[i].spec & EFX_EF10_FILTER_FLAG_BUSY) {
-				rc = -EBUSY;
-				goto fail_unlock;
-			}
-			if (spec->priority < saved_spec->priority) {
-				rc = -EPERM;
-				goto fail_unlock;
-			}
-			ins_index = i;
-			break;
-		}
-
-		/* Once we reach the maximum search depth, use the
-		 * first suitable slot or return -EBUSY if there was
-		 * none
-		 */
-		if (depth == EFX_EF10_FILTER_SEARCH_LIMIT) {
-			if (ins_index < 0) {
-				rc = -EBUSY;
-				goto fail_unlock;
-			}
-			break;
-		}
-
-		++depth;
-	}
-
-	/* Create a software table entry if necessary, and mark it
-	 * busy.  We might yet fail to insert, but any attempt to
-	 * insert a conflicting filter while we're waiting for the
-	 * firmware must find the busy entry.
-	 */
-	saved_spec = efx_ef10_filter_entry_spec(table, ins_index);
-	if (saved_spec) {
-		replacing = true;
-	} else {
-		saved_spec = kmalloc(sizeof(*spec), GFP_ATOMIC);
-		if (!saved_spec) {
-			rc = -ENOMEM;
-			goto fail_unlock;
-		}
-		*saved_spec = *spec;
-	}
-	efx_ef10_filter_set_entry(table, ins_index, saved_spec,
-				  EFX_EF10_FILTER_FLAG_BUSY);
-
-	spin_unlock_bh(&efx->filter_lock);
-
-	/* Pack up the variables needed on completion */
-	cookie = replacing << 31 | ins_index << 16 | spec->dmaq_id;
-
-	efx_ef10_filter_push_prep(efx, spec, inbuf,
-				  table->entry[ins_index].handle, NULL,
-				  replacing);
-	efx_mcdi_rpc_async(efx, MC_CMD_FILTER_OP, inbuf, sizeof(inbuf),
-			   MC_CMD_FILTER_OP_OUT_LEN,
-			   efx_ef10_filter_rfs_insert_complete, cookie);
-
-	return ins_index;
-
-fail_unlock:
-	spin_unlock_bh(&efx->filter_lock);
-	return rc;
-}
-
-static void
-efx_ef10_filter_rfs_insert_complete(struct efx_nic *efx, unsigned long cookie,
-				    int rc, efx_dword_t *outbuf,
-				    size_t outlen_actual)
-{
-	struct efx_ef10_filter_table *table = efx->filter_state;
-	unsigned int ins_index, dmaq_id;
-	struct efx_filter_spec *spec;
-	bool replacing;
-
-	/* Unpack the cookie */
-	replacing = cookie >> 31;
-	ins_index = (cookie >> 16) & (HUNT_FILTER_TBL_ROWS - 1);
-	dmaq_id = cookie & 0xffff;
-
-	spin_lock_bh(&efx->filter_lock);
-	spec = efx_ef10_filter_entry_spec(table, ins_index);
-	if (rc == 0) {
-		table->entry[ins_index].handle =
-			MCDI_QWORD(outbuf, FILTER_OP_OUT_HANDLE);
-		if (replacing)
-			spec->dmaq_id = dmaq_id;
-	} else if (!replacing) {
-		kfree(spec);
-		spec = NULL;
-	}
-	efx_ef10_filter_set_entry(table, ins_index, spec, 0);
-	spin_unlock_bh(&efx->filter_lock);
-
-	wake_up_all(&table->waitq);
-}
-
 static void
 efx_ef10_filter_rfs_expire_complete(struct efx_nic *efx,
 				    unsigned long filter_idx,
@@ -4905,18 +4768,22 @@ static bool efx_ef10_filter_rfs_expire_one(struct efx_nic *efx, u32 flow_id,
 					   unsigned int filter_idx)
 {
 	struct efx_ef10_filter_table *table = efx->filter_state;
-	struct efx_filter_spec *spec =
-		efx_ef10_filter_entry_spec(table, filter_idx);
+	struct efx_filter_spec *spec;
 	MCDI_DECLARE_BUF(inbuf,
 			 MC_CMD_FILTER_OP_IN_HANDLE_OFST +
 			 MC_CMD_FILTER_OP_IN_HANDLE_LEN);
+	bool ret = true;
 
+	spin_lock_bh(&efx->filter_lock);
+	spec = efx_ef10_filter_entry_spec(table, filter_idx);
 	if (!spec ||
 	    (table->entry[filter_idx].spec & EFX_EF10_FILTER_FLAG_BUSY) ||
 	    spec->priority != EFX_FILTER_PRI_HINT ||
 	    !rps_may_expire_flow(efx->net_dev, spec->dmaq_id,
-				 flow_id, filter_idx))
-		return false;
+				 flow_id, filter_idx)) {
+		ret = false;
+		goto out_unlock;
+	}
 
 	MCDI_SET_DWORD(inbuf, FILTER_OP_IN_OP,
 		       MC_CMD_FILTER_OP_IN_OP_REMOVE);
@@ -4924,10 +4791,12 @@ static bool efx_ef10_filter_rfs_expire_one(struct efx_nic *efx, u32 flow_id,
 		       table->entry[filter_idx].handle);
 	if (efx_mcdi_rpc_async(efx, MC_CMD_FILTER_OP, inbuf, sizeof(inbuf), 0,
 			       efx_ef10_filter_rfs_expire_complete, filter_idx))
-		return false;
-
-	table->entry[filter_idx].spec |= EFX_EF10_FILTER_FLAG_BUSY;
-	return true;
+		ret = false;
+	else
+		table->entry[filter_idx].spec |= EFX_EF10_FILTER_FLAG_BUSY;
+out_unlock:
+	spin_unlock_bh(&efx->filter_lock);
+	return ret;
 }
 
 static void
@@ -6784,7 +6653,6 @@ const struct efx_nic_type efx_hunt_a0_vf_nic_type = {
 	.filter_get_rx_id_limit = efx_ef10_filter_get_rx_id_limit,
 	.filter_get_rx_ids = efx_ef10_filter_get_rx_ids,
 #ifdef CONFIG_RFS_ACCEL
-	.filter_rfs_insert = efx_ef10_filter_rfs_insert,
 	.filter_rfs_expire_one = efx_ef10_filter_rfs_expire_one,
 #endif
 #ifdef CONFIG_SFC_MTD
@@ -6897,7 +6765,6 @@ const struct efx_nic_type efx_hunt_a0_nic_type = {
 	.filter_get_rx_id_limit = efx_ef10_filter_get_rx_id_limit,
 	.filter_get_rx_ids = efx_ef10_filter_get_rx_ids,
 #ifdef CONFIG_RFS_ACCEL
-	.filter_rfs_insert = efx_ef10_filter_rfs_insert,
 	.filter_rfs_expire_one = efx_ef10_filter_rfs_expire_one,
 #endif
 #ifdef CONFIG_SFC_MTD
diff --git a/drivers/net/ethernet/sfc/efx.c b/drivers/net/ethernet/sfc/efx.c
index 7321a4cf6f4d..0be93abdb2ad 100644
--- a/drivers/net/ethernet/sfc/efx.c
+++ b/drivers/net/ethernet/sfc/efx.c
@@ -340,7 +340,10 @@ static int efx_poll(struct napi_struct *napi, int budget)
 			efx_update_irq_mod(efx, channel);
 		}
 
-		efx_filter_rfs_expire(channel);
+#ifdef CONFIG_RFS_ACCEL
+		/* Perhaps expire some ARFS filters */
+		schedule_work(&channel->filter_work);
+#endif
 
 		/* There is no race here; although napi_disable() will
 		 * only wait for napi_complete(), this isn't a problem
@@ -470,6 +473,10 @@ efx_alloc_channel(struct efx_nic *efx, int i, struct efx_channel *old_channel)
 		tx_queue->channel = channel;
 	}
 
+#ifdef CONFIG_RFS_ACCEL
+	INIT_WORK(&channel->filter_work, efx_filter_rfs_expire);
+#endif
+
 	rx_queue = &channel->rx_queue;
 	rx_queue->efx = efx;
 	timer_setup(&rx_queue->slow_fill, efx_rx_slow_fill, 0);
@@ -512,6 +519,9 @@ efx_copy_channel(const struct efx_channel *old_channel)
 	rx_queue->buffer = NULL;
 	memset(&rx_queue->rxd, 0, sizeof(rx_queue->rxd));
 	timer_setup(&rx_queue->slow_fill, efx_rx_slow_fill, 0);
+#ifdef CONFIG_RFS_ACCEL
+	INIT_WORK(&channel->filter_work, efx_filter_rfs_expire);
+#endif
 
 	return channel;
 }
@@ -3012,6 +3022,9 @@ static int efx_init_struct(struct efx_nic *efx,
 	efx->num_mac_stats = MC_CMD_MAC_NSTATS;
 	BUILD_BUG_ON(MC_CMD_MAC_NSTATS - 1 != MC_CMD_MAC_GENERATION_END);
 	mutex_init(&efx->mac_lock);
+#ifdef CONFIG_RFS_ACCEL
+	mutex_init(&efx->rps_mutex);
+#endif
 	efx->phy_op = &efx_dummy_phy_operations;
 	efx->mdio.dev = net_dev;
 	INIT_WORK(&efx->mac_work, efx_mac_work);
diff --git a/drivers/net/ethernet/sfc/efx.h b/drivers/net/ethernet/sfc/efx.h
index 3429ae3f3b08..545c2ea1622e 100644
--- a/drivers/net/ethernet/sfc/efx.h
+++ b/drivers/net/ethernet/sfc/efx.h
@@ -170,15 +170,18 @@ static inline s32 efx_filter_get_rx_ids(struct efx_nic *efx,
 int efx_filter_rfs(struct net_device *net_dev, const struct sk_buff *skb,
 		   u16 rxq_index, u32 flow_id);
 bool __efx_filter_rfs_expire(struct efx_nic *efx, unsigned quota);
-static inline void efx_filter_rfs_expire(struct efx_channel *channel)
+static inline void efx_filter_rfs_expire(struct work_struct *data)
 {
+	struct efx_channel *channel = container_of(data, struct efx_channel,
+						   filter_work);
+
 	if (channel->rfs_filters_added >= 60 &&
 	    __efx_filter_rfs_expire(channel->efx, 100))
 		channel->rfs_filters_added -= 60;
 }
 #define efx_filter_rfs_enabled() 1
 #else
-static inline void efx_filter_rfs_expire(struct efx_channel *channel) {}
+static inline void efx_filter_rfs_expire(struct work_struct *data) {}
 #define efx_filter_rfs_enabled() 0
 #endif
 bool efx_filter_is_mc_recipient(const struct efx_filter_spec *spec);
diff --git a/drivers/net/ethernet/sfc/farch.c b/drivers/net/ethernet/sfc/farch.c
index ad001e77d554..6b5ca569db5e 100644
--- a/drivers/net/ethernet/sfc/farch.c
+++ b/drivers/net/ethernet/sfc/farch.c
@@ -2901,28 +2901,25 @@ void efx_farch_filter_update_rx_scatter(struct efx_nic *efx)
 
 #ifdef CONFIG_RFS_ACCEL
 
-s32 efx_farch_filter_rfs_insert(struct efx_nic *efx,
-				struct efx_filter_spec *gen_spec)
-{
-	return efx_farch_filter_insert(efx, gen_spec, true);
-}
-
 bool efx_farch_filter_rfs_expire_one(struct efx_nic *efx, u32 flow_id,
 				     unsigned int index)
 {
 	struct efx_farch_filter_state *state = efx->filter_state;
-	struct efx_farch_filter_table *table =
-		&state->table[EFX_FARCH_FILTER_TABLE_RX_IP];
+	struct efx_farch_filter_table *table;
+	bool ret = false;
 
+	spin_lock_bh(&efx->filter_lock);
+	table = &state->table[EFX_FARCH_FILTER_TABLE_RX_IP];
 	if (test_bit(index, table->used_bitmap) &&
 	    table->spec[index].priority == EFX_FILTER_PRI_HINT &&
 	    rps_may_expire_flow(efx->net_dev, table->spec[index].dmaq_id,
 				flow_id, index)) {
 		efx_farch_filter_table_clear_entry(efx, table, index);
-		return true;
+		ret = true;
 	}
 
-	return false;
+	spin_unlock_bh(&efx->filter_lock);
+	return ret;
 }
 
 #endif /* CONFIG_RFS_ACCEL */
diff --git a/drivers/net/ethernet/sfc/net_driver.h b/drivers/net/ethernet/sfc/net_driver.h
index 203d64c88de5..fe0c5322454d 100644
--- a/drivers/net/ethernet/sfc/net_driver.h
+++ b/drivers/net/ethernet/sfc/net_driver.h
@@ -430,6 +430,7 @@ enum efx_sync_events_state {
  * @event_test_cpu: Last CPU to handle interrupt or test event for this channel
  * @irq_count: Number of IRQs since last adaptive moderation decision
  * @irq_mod_score: IRQ moderation score
+ * @filter_work: Work item for efx_filter_rfs_expire()
  * @rps_flow_id: Flow IDs of filters allocated for accelerated RFS,
  *      indexed by filter ID
  * @n_rx_tobe_disc: Count of RX_TOBE_DISC errors
@@ -475,6 +476,7 @@ struct efx_channel {
 	unsigned int irq_mod_score;
 #ifdef CONFIG_RFS_ACCEL
 	unsigned int rfs_filters_added;
+	struct work_struct filter_work;
 #define RPS_FLOW_ID_INVALID 0xFFFFFFFF
 	u32 *rps_flow_id;
 #endif
@@ -837,6 +839,7 @@ struct efx_rss_context {
  * @filter_sem: Filter table rw_semaphore, for freeing the table
  * @filter_lock: Filter table lock, for mere content changes
  * @filter_state: Architecture-dependent filter table state
+ * @rps_mutex: Protects RPS state of all channels
  * @rps_expire_channel: Next channel to check for expiry
  * @rps_expire_index: Next index to check for expiry in
  *	@rps_expire_channel's @rps_flow_id
@@ -990,6 +993,7 @@ struct efx_nic {
 	spinlock_t filter_lock;
 	void *filter_state;
 #ifdef CONFIG_RFS_ACCEL
+	struct mutex rps_mutex;
 	unsigned int rps_expire_channel;
 	unsigned int rps_expire_index;
 #endif
@@ -1144,10 +1148,6 @@ struct efx_udp_tunnel {
  * @filter_count_rx_used: Get the number of filters in use at a given priority
  * @filter_get_rx_id_limit: Get maximum value of a filter id, plus 1
  * @filter_get_rx_ids: Get list of RX filters at a given priority
- * @filter_rfs_insert: Add or replace a filter for RFS.  This must be
- *	atomic.  The hardware change may be asynchronous but should
- *	not be delayed for long.  It may fail if this can't be done
- *	atomically.
  * @filter_rfs_expire_one: Consider expiring a filter inserted for RFS.
  *	This must check whether the specified table entry is used by RFS
  *	and that rps_may_expire_flow() returns true for it.
@@ -1298,8 +1298,6 @@ struct efx_nic_type {
 				 enum efx_filter_priority priority,
 				 u32 *buf, u32 size);
 #ifdef CONFIG_RFS_ACCEL
-	s32 (*filter_rfs_insert)(struct efx_nic *efx,
-				 struct efx_filter_spec *spec);
 	bool (*filter_rfs_expire_one)(struct efx_nic *efx, u32 flow_id,
 				      unsigned int index);
 #endif
diff --git a/drivers/net/ethernet/sfc/nic.h b/drivers/net/ethernet/sfc/nic.h
index d080a414e8f2..ac59afad6e5c 100644
--- a/drivers/net/ethernet/sfc/nic.h
+++ b/drivers/net/ethernet/sfc/nic.h
@@ -601,8 +601,6 @@ s32 efx_farch_filter_get_rx_ids(struct efx_nic *efx,
 				enum efx_filter_priority priority, u32 *buf,
 				u32 size);
 #ifdef CONFIG_RFS_ACCEL
-s32 efx_farch_filter_rfs_insert(struct efx_nic *efx,
-				struct efx_filter_spec *spec);
 bool efx_farch_filter_rfs_expire_one(struct efx_nic *efx, u32 flow_id,
 				     unsigned int index);
 #endif
diff --git a/drivers/net/ethernet/sfc/rx.c b/drivers/net/ethernet/sfc/rx.c
index cfe76aad79ee..95682831484e 100644
--- a/drivers/net/ethernet/sfc/rx.c
+++ b/drivers/net/ethernet/sfc/rx.c
@@ -827,14 +827,67 @@ MODULE_PARM_DESC(rx_refill_threshold,
 
 #ifdef CONFIG_RFS_ACCEL
 
+/**
+ * struct efx_async_filter_insertion - Request to asynchronously insert a filter
+ * @net_dev: Reference to the netdevice
+ * @spec: The filter to insert
+ * @work: Workitem for this request
+ * @rxq_index: Identifies the channel for which this request was made
+ * @flow_id: Identifies the kernel-side flow for which this request was made
+ */
+struct efx_async_filter_insertion {
+	struct net_device *net_dev;
+	struct efx_filter_spec spec;
+	struct work_struct work;
+	u16 rxq_index;
+	u32 flow_id;
+};
+
+static void efx_filter_rfs_work(struct work_struct *data)
+{
+	struct efx_async_filter_insertion *req = container_of(data, struct efx_async_filter_insertion,
+							      work);
+	struct efx_nic *efx = netdev_priv(req->net_dev);
+	struct efx_channel *channel = efx_get_channel(efx, req->rxq_index);
+	int rc;
+
+	rc = efx->type->filter_insert(efx, &req->spec, false);
+	if (rc >= 0) {
+		/* Remember this so we can check whether to expire the filter
+		 * later.
+		 */
+		mutex_lock(&efx->rps_mutex);
+		channel->rps_flow_id[rc] = req->flow_id;
+		++channel->rfs_filters_added;
+		mutex_unlock(&efx->rps_mutex);
+
+		if (req->spec.ether_type == htons(ETH_P_IP))
+			netif_info(efx, rx_status, efx->net_dev,
+				   "steering %s %pI4:%u:%pI4:%u to queue %u [flow %u filter %d]\n",
+				   (req->spec.ip_proto == IPPROTO_TCP) ? "TCP" : "UDP",
+				   req->spec.rem_host, ntohs(req->spec.rem_port),
+				   req->spec.loc_host, ntohs(req->spec.loc_port),
+				   req->rxq_index, req->flow_id, rc);
+		else
+			netif_info(efx, rx_status, efx->net_dev,
+				   "steering %s [%pI6]:%u:[%pI6]:%u to queue %u [flow %u filter %d]\n",
+				   (req->spec.ip_proto == IPPROTO_TCP) ? "TCP" : "UDP",
+				   req->spec.rem_host, ntohs(req->spec.rem_port),
+				   req->spec.loc_host, ntohs(req->spec.loc_port),
+				   req->rxq_index, req->flow_id, rc);
+	}
+
+	/* Release references */
+	dev_put(req->net_dev);
+	kfree(req);
+}
+
 int efx_filter_rfs(struct net_device *net_dev, const struct sk_buff *skb,
 		   u16 rxq_index, u32 flow_id)
 {
 	struct efx_nic *efx = netdev_priv(net_dev);
-	struct efx_channel *channel;
-	struct efx_filter_spec spec;
+	struct efx_async_filter_insertion *req;
 	struct flow_keys fk;
-	int rc;
 
 	if (flow_id == RPS_FLOW_ID_INVALID)
 		return -EINVAL;
@@ -847,50 +900,39 @@ int efx_filter_rfs(struct net_device *net_dev, const struct sk_buff *skb,
 	if (fk.control.flags & FLOW_DIS_IS_FRAGMENT)
 		return -EPROTONOSUPPORT;
 
-	efx_filter_init_rx(&spec, EFX_FILTER_PRI_HINT,
+	req = kmalloc(sizeof(*req), GFP_ATOMIC);
+	if (!req)
+		return -ENOMEM;
+
+	efx_filter_init_rx(&req->spec, EFX_FILTER_PRI_HINT,
 			   efx->rx_scatter ? EFX_FILTER_FLAG_RX_SCATTER : 0,
 			   rxq_index);
-	spec.match_flags =
+	req->spec.match_flags =
 		EFX_FILTER_MATCH_ETHER_TYPE | EFX_FILTER_MATCH_IP_PROTO |
 		EFX_FILTER_MATCH_LOC_HOST | EFX_FILTER_MATCH_LOC_PORT |
 		EFX_FILTER_MATCH_REM_HOST | EFX_FILTER_MATCH_REM_PORT;
-	spec.ether_type = fk.basic.n_proto;
-	spec.ip_proto = fk.basic.ip_proto;
+	req->spec.ether_type = fk.basic.n_proto;
+	req->spec.ip_proto = fk.basic.ip_proto;
 
 	if (fk.basic.n_proto == htons(ETH_P_IP)) {
-		spec.rem_host[0] = fk.addrs.v4addrs.src;
-		spec.loc_host[0] = fk.addrs.v4addrs.dst;
+		req->spec.rem_host[0] = fk.addrs.v4addrs.src;
+		req->spec.loc_host[0] = fk.addrs.v4addrs.dst;
 	} else {
-		memcpy(spec.rem_host, &fk.addrs.v6addrs.src, sizeof(struct in6_addr));
-		memcpy(spec.loc_host, &fk.addrs.v6addrs.dst, sizeof(struct in6_addr));
+		memcpy(req->spec.rem_host, &fk.addrs.v6addrs.src,
+		       sizeof(struct in6_addr));
+		memcpy(req->spec.loc_host, &fk.addrs.v6addrs.dst,
+		       sizeof(struct in6_addr));
 	}
 
-	spec.rem_port = fk.ports.src;
-	spec.loc_port = fk.ports.dst;
-
-	rc = efx->type->filter_rfs_insert(efx, &spec);
-	if (rc < 0)
-		return rc;
+	req->spec.rem_port = fk.ports.src;
+	req->spec.loc_port = fk.ports.dst;
 
-	/* Remember this so we can check whether to expire the filter later */
-	channel = efx_get_channel(efx, rxq_index);
-	channel->rps_flow_id[rc] = flow_id;
-	++channel->rfs_filters_added;
-
-	if (spec.ether_type == htons(ETH_P_IP))
-		netif_info(efx, rx_status, efx->net_dev,
-			   "steering %s %pI4:%u:%pI4:%u to queue %u [flow %u filter %d]\n",
-			   (spec.ip_proto == IPPROTO_TCP) ? "TCP" : "UDP",
-			   spec.rem_host, ntohs(spec.rem_port), spec.loc_host,
-			   ntohs(spec.loc_port), rxq_index, flow_id, rc);
-	else
-		netif_info(efx, rx_status, efx->net_dev,
-			   "steering %s [%pI6]:%u:[%pI6]:%u to queue %u [flow %u filter %d]\n",
-			   (spec.ip_proto == IPPROTO_TCP) ? "TCP" : "UDP",
-			   spec.rem_host, ntohs(spec.rem_port), spec.loc_host,
-			   ntohs(spec.loc_port), rxq_index, flow_id, rc);
-
-	return rc;
+	dev_hold(req->net_dev = net_dev);
+	INIT_WORK(&req->work, efx_filter_rfs_work);
+	req->rxq_index = rxq_index;
+	req->flow_id = flow_id;
+	schedule_work(&req->work);
+	return 0;
 }
 
 bool __efx_filter_rfs_expire(struct efx_nic *efx, unsigned int quota)
@@ -899,9 +941,8 @@ bool __efx_filter_rfs_expire(struct efx_nic *efx, unsigned int quota)
 	unsigned int channel_idx, index, size;
 	u32 flow_id;
 
-	if (!spin_trylock_bh(&efx->filter_lock))
+	if (!mutex_trylock(&efx->rps_mutex))
 		return false;
-
 	expire_one = efx->type->filter_rfs_expire_one;
 	channel_idx = efx->rps_expire_channel;
 	index = efx->rps_expire_index;
@@ -926,7 +967,7 @@ bool __efx_filter_rfs_expire(struct efx_nic *efx, unsigned int quota)
 	efx->rps_expire_channel = channel_idx;
 	efx->rps_expire_index = index;
 
-	spin_unlock_bh(&efx->filter_lock);
+	mutex_unlock(&efx->rps_mutex);
 	return true;
 }
 
diff --git a/drivers/net/ethernet/sfc/siena.c b/drivers/net/ethernet/sfc/siena.c
index 18aab25234ba..65161f68265a 100644
--- a/drivers/net/ethernet/sfc/siena.c
+++ b/drivers/net/ethernet/sfc/siena.c
@@ -1035,7 +1035,6 @@ const struct efx_nic_type siena_a0_nic_type = {
 	.filter_get_rx_id_limit = efx_farch_filter_get_rx_id_limit,
 	.filter_get_rx_ids = efx_farch_filter_get_rx_ids,
 #ifdef CONFIG_RFS_ACCEL
-	.filter_rfs_insert = efx_farch_filter_rfs_insert,
 	.filter_rfs_expire_one = efx_farch_filter_rfs_expire_one,
 #endif
 #ifdef CONFIG_SFC_MTD

^ permalink raw reply related	[flat|nested] 8+ messages in thread

* [PATCH net-next 2/6] sfc: give ef10 its own rwsem in the filter table instead of filter_lock
  2018-03-27 16:40 [PATCH net-next 0/6] sfc: rework locking around filter management Edward Cree
  2018-03-27 16:41 ` [PATCH net-next 1/6] sfc: replace asynchronous filter operations Edward Cree
@ 2018-03-27 16:42 ` Edward Cree
  2018-03-27 16:42 ` [PATCH net-next 3/6] sfc: use a semaphore to lock farch filters too Edward Cree
                   ` (4 subsequent siblings)
  6 siblings, 0 replies; 8+ messages in thread
From: Edward Cree @ 2018-03-27 16:42 UTC (permalink / raw)
  To: linux-net-drivers, David Miller; +Cc: netdev

efx->filter_lock remains in place for use on farch, but EF10 now ignores it.
EFX_EF10_FILTER_FLAG_BUSY is no longer needed, hence it is removed.

Signed-off-by: Edward Cree <ecree@solarflare.com>
---
 drivers/net/ethernet/sfc/ef10.c       | 361 ++++++++++++++--------------------
 drivers/net/ethernet/sfc/efx.c        |   1 -
 drivers/net/ethernet/sfc/net_driver.h |   2 +-
 3 files changed, 151 insertions(+), 213 deletions(-)

diff --git a/drivers/net/ethernet/sfc/ef10.c b/drivers/net/ethernet/sfc/ef10.c
index bcbaba330fb5..9db1b9144e70 100644
--- a/drivers/net/ethernet/sfc/ef10.c
+++ b/drivers/net/ethernet/sfc/ef10.c
@@ -96,17 +96,15 @@ struct efx_ef10_filter_table {
 		MC_CMD_GET_PARSER_DISP_INFO_OUT_SUPPORTED_MATCHES_MAXNUM * 2];
 	unsigned int rx_match_count;
 
+	struct rw_semaphore lock; /* Protects entries */
 	struct {
 		unsigned long spec;	/* pointer to spec plus flag bits */
-/* BUSY flag indicates that an update is in progress.  AUTO_OLD is
- * used to mark and sweep MAC filters for the device address lists.
- */
-#define EFX_EF10_FILTER_FLAG_BUSY	1UL
+/* AUTO_OLD is used to mark and sweep MAC filters for the device address lists. */
+/* unused flag	1UL */
 #define EFX_EF10_FILTER_FLAG_AUTO_OLD	2UL
 #define EFX_EF10_FILTER_FLAGS		3UL
 		u64 handle;		/* firmware handle */
 	} *entry;
-	wait_queue_head_t waitq;
 /* Shadow of net_device address lists, guarded by mac_lock */
 	struct efx_ef10_dev_addr dev_uc_list[EFX_EF10_FILTER_DEV_UC_MAX];
 	struct efx_ef10_dev_addr dev_mc_list[EFX_EF10_FILTER_DEV_MC_MAX];
@@ -4302,26 +4300,33 @@ static s32 efx_ef10_filter_insert(struct efx_nic *efx,
 				  struct efx_filter_spec *spec,
 				  bool replace_equal)
 {
-	struct efx_ef10_filter_table *table = efx->filter_state;
 	DECLARE_BITMAP(mc_rem_map, EFX_EF10_FILTER_SEARCH_LIMIT);
+	struct efx_ef10_filter_table *table;
 	struct efx_filter_spec *saved_spec;
 	struct efx_rss_context *ctx = NULL;
 	unsigned int match_pri, hash;
 	unsigned int priv_flags;
 	bool replacing = false;
+	unsigned int depth, i;
 	int ins_index = -1;
 	DEFINE_WAIT(wait);
 	bool is_mc_recip;
 	s32 rc;
 
+	down_read(&efx->filter_sem);
+	table = efx->filter_state;
+	down_write(&table->lock);
+
 	/* For now, only support RX filters */
 	if ((spec->flags & (EFX_FILTER_FLAG_RX | EFX_FILTER_FLAG_TX)) !=
-	    EFX_FILTER_FLAG_RX)
-		return -EINVAL;
+	    EFX_FILTER_FLAG_RX) {
+		rc = -EINVAL;
+		goto out_unlock;
+	}
 
 	rc = efx_ef10_filter_pri(table, spec);
 	if (rc < 0)
-		return rc;
+		goto out_unlock;
 	match_pri = rc;
 
 	hash = efx_ef10_filter_hash(spec);
@@ -4335,86 +4340,64 @@ static s32 efx_ef10_filter_insert(struct efx_nic *efx,
 							 &efx->rss_context.list);
 		else
 			ctx = &efx->rss_context;
-		if (!ctx)
-			return -ENOENT;
-		if (ctx->context_id == EFX_EF10_RSS_CONTEXT_INVALID)
-			return -EOPNOTSUPP;
+		if (!ctx) {
+			rc = -ENOENT;
+			goto out_unlock;
+		}
+		if (ctx->context_id == EFX_EF10_RSS_CONTEXT_INVALID) {
+			rc = -EOPNOTSUPP;
+			goto out_unlock;
+		}
 	}
 
 	/* Find any existing filters with the same match tuple or
-	 * else a free slot to insert at.  If any of them are busy,
-	 * we have to wait and retry.
+	 * else a free slot to insert at.
 	 */
-	for (;;) {
-		unsigned int depth = 1;
-		unsigned int i;
-
-		spin_lock_bh(&efx->filter_lock);
+	for (depth = 1; depth < EFX_EF10_FILTER_SEARCH_LIMIT; depth++) {
+		i = (hash + depth) & (HUNT_FILTER_TBL_ROWS - 1);
+		saved_spec = efx_ef10_filter_entry_spec(table, i);
 
-		for (;;) {
-			i = (hash + depth) & (HUNT_FILTER_TBL_ROWS - 1);
-			saved_spec = efx_ef10_filter_entry_spec(table, i);
-
-			if (!saved_spec) {
-				if (ins_index < 0)
-					ins_index = i;
-			} else if (efx_ef10_filter_equal(spec, saved_spec)) {
-				if (table->entry[i].spec &
-				    EFX_EF10_FILTER_FLAG_BUSY)
-					break;
-				if (spec->priority < saved_spec->priority &&
-				    spec->priority != EFX_FILTER_PRI_AUTO) {
-					rc = -EPERM;
-					goto out_unlock;
-				}
-				if (!is_mc_recip) {
-					/* This is the only one */
-					if (spec->priority ==
-					    saved_spec->priority &&
-					    !replace_equal) {
-						rc = -EEXIST;
-						goto out_unlock;
-					}
-					ins_index = i;
-					goto found;
-				} else if (spec->priority >
-					   saved_spec->priority ||
-					   (spec->priority ==
-					    saved_spec->priority &&
-					    replace_equal)) {
-					if (ins_index < 0)
-						ins_index = i;
-					else
-						__set_bit(depth, mc_rem_map);
-				}
+		if (!saved_spec) {
+			if (ins_index < 0)
+				ins_index = i;
+		} else if (efx_ef10_filter_equal(spec, saved_spec)) {
+			if (spec->priority < saved_spec->priority &&
+			    spec->priority != EFX_FILTER_PRI_AUTO) {
+				rc = -EPERM;
+				goto out_unlock;
 			}
-
-			/* Once we reach the maximum search depth, use
-			 * the first suitable slot or return -EBUSY if
-			 * there was none
-			 */
-			if (depth == EFX_EF10_FILTER_SEARCH_LIMIT) {
-				if (ins_index < 0) {
-					rc = -EBUSY;
+			if (!is_mc_recip) {
+				/* This is the only one */
+				if (spec->priority ==
+				    saved_spec->priority &&
+				    !replace_equal) {
+					rc = -EEXIST;
 					goto out_unlock;
 				}
-				goto found;
+				ins_index = i;
+				break;
+			} else if (spec->priority >
+				   saved_spec->priority ||
+				   (spec->priority ==
+				    saved_spec->priority &&
+				    replace_equal)) {
+				if (ins_index < 0)
+					ins_index = i;
+				else
+					__set_bit(depth, mc_rem_map);
 			}
-
-			++depth;
 		}
-
-		prepare_to_wait(&table->waitq, &wait, TASK_UNINTERRUPTIBLE);
-		spin_unlock_bh(&efx->filter_lock);
-		schedule();
 	}
 
-found:
-	/* Create a software table entry if necessary, and mark it
-	 * busy.  We might yet fail to insert, but any attempt to
-	 * insert a conflicting filter while we're waiting for the
-	 * firmware must find the busy entry.
+	/* Once we reach the maximum search depth, use the first suitable
+	 * slot, or return -EBUSY if there was none
 	 */
+	if (ins_index < 0) {
+		rc = -EBUSY;
+		goto out_unlock;
+	}
+
+	/* Create a software table entry if necessary. */
 	saved_spec = efx_ef10_filter_entry_spec(table, ins_index);
 	if (saved_spec) {
 		if (spec->priority == EFX_FILTER_PRI_AUTO &&
@@ -4438,28 +4421,13 @@ static s32 efx_ef10_filter_insert(struct efx_nic *efx,
 		*saved_spec = *spec;
 		priv_flags = 0;
 	}
-	efx_ef10_filter_set_entry(table, ins_index, saved_spec,
-				  priv_flags | EFX_EF10_FILTER_FLAG_BUSY);
-
-	/* Mark lower-priority multicast recipients busy prior to removal */
-	if (is_mc_recip) {
-		unsigned int depth, i;
-
-		for (depth = 0; depth < EFX_EF10_FILTER_SEARCH_LIMIT; depth++) {
-			i = (hash + depth) & (HUNT_FILTER_TBL_ROWS - 1);
-			if (test_bit(depth, mc_rem_map))
-				table->entry[i].spec |=
-					EFX_EF10_FILTER_FLAG_BUSY;
-		}
-	}
-
-	spin_unlock_bh(&efx->filter_lock);
+	efx_ef10_filter_set_entry(table, ins_index, saved_spec, priv_flags);
 
+	/* Actually insert the filter on the HW */
 	rc = efx_ef10_filter_push(efx, spec, &table->entry[ins_index].handle,
 				  ctx, replacing);
 
 	/* Finalise the software table entry */
-	spin_lock_bh(&efx->filter_lock);
 	if (rc == 0) {
 		if (replacing) {
 			/* Update the fields that may differ */
@@ -4475,6 +4443,12 @@ static s32 efx_ef10_filter_insert(struct efx_nic *efx,
 	} else if (!replacing) {
 		kfree(saved_spec);
 		saved_spec = NULL;
+	} else {
+		/* We failed to replace, so the old filter is still present.
+		 * Roll back the software table to reflect this.  In fact the
+		 * efx_ef10_filter_set_entry() call below will do the right
+		 * thing, so nothing extra is needed here.
+		 */
 	}
 	efx_ef10_filter_set_entry(table, ins_index, saved_spec, priv_flags);
 
@@ -4496,7 +4470,6 @@ static s32 efx_ef10_filter_insert(struct efx_nic *efx,
 			priv_flags = efx_ef10_filter_entry_flags(table, i);
 
 			if (rc == 0) {
-				spin_unlock_bh(&efx->filter_lock);
 				MCDI_SET_DWORD(inbuf, FILTER_OP_IN_OP,
 					       MC_CMD_FILTER_OP_IN_OP_UNSUBSCRIBE);
 				MCDI_SET_QWORD(inbuf, FILTER_OP_IN_HANDLE,
@@ -4504,15 +4477,12 @@ static s32 efx_ef10_filter_insert(struct efx_nic *efx,
 				rc = efx_mcdi_rpc(efx, MC_CMD_FILTER_OP,
 						  inbuf, sizeof(inbuf),
 						  NULL, 0, NULL);
-				spin_lock_bh(&efx->filter_lock);
 			}
 
 			if (rc == 0) {
 				kfree(saved_spec);
 				saved_spec = NULL;
 				priv_flags = 0;
-			} else {
-				priv_flags &= ~EFX_EF10_FILTER_FLAG_BUSY;
 			}
 			efx_ef10_filter_set_entry(table, i, saved_spec,
 						  priv_flags);
@@ -4523,10 +4493,9 @@ static s32 efx_ef10_filter_insert(struct efx_nic *efx,
 	if (rc == 0)
 		rc = efx_ef10_make_filter_id(match_pri, ins_index);
 
-	wake_up_all(&table->waitq);
 out_unlock:
-	spin_unlock_bh(&efx->filter_lock);
-	finish_wait(&table->waitq, &wait);
+	up_write(&table->lock);
+	up_read(&efx->filter_sem);
 	return rc;
 }
 
@@ -4539,6 +4508,8 @@ static void efx_ef10_filter_update_rx_scatter(struct efx_nic *efx)
  * If !by_index, remove by ID
  * If by_index, remove by index
  * Filter ID may come from userland and must be range-checked.
+ * Caller must hold efx->filter_sem for read, and efx->filter_state->lock
+ * for write.
  */
 static int efx_ef10_filter_remove_internal(struct efx_nic *efx,
 					   unsigned int priority_mask,
@@ -4553,45 +4524,23 @@ static int efx_ef10_filter_remove_internal(struct efx_nic *efx,
 	DEFINE_WAIT(wait);
 	int rc;
 
-	/* Find the software table entry and mark it busy.  Don't
-	 * remove it yet; any attempt to update while we're waiting
-	 * for the firmware must find the busy entry.
-	 */
-	for (;;) {
-		spin_lock_bh(&efx->filter_lock);
-		if (!(table->entry[filter_idx].spec &
-		      EFX_EF10_FILTER_FLAG_BUSY))
-			break;
-		prepare_to_wait(&table->waitq, &wait, TASK_UNINTERRUPTIBLE);
-		spin_unlock_bh(&efx->filter_lock);
-		schedule();
-	}
-
 	spec = efx_ef10_filter_entry_spec(table, filter_idx);
 	if (!spec ||
 	    (!by_index &&
 	     efx_ef10_filter_pri(table, spec) !=
-	     efx_ef10_filter_get_unsafe_pri(filter_id))) {
-		rc = -ENOENT;
-		goto out_unlock;
-	}
+	     efx_ef10_filter_get_unsafe_pri(filter_id)))
+		return -ENOENT;
 
 	if (spec->flags & EFX_FILTER_FLAG_RX_OVER_AUTO &&
 	    priority_mask == (1U << EFX_FILTER_PRI_AUTO)) {
 		/* Just remove flags */
 		spec->flags &= ~EFX_FILTER_FLAG_RX_OVER_AUTO;
 		table->entry[filter_idx].spec &= ~EFX_EF10_FILTER_FLAG_AUTO_OLD;
-		rc = 0;
-		goto out_unlock;
-	}
-
-	if (!(priority_mask & (1U << spec->priority))) {
-		rc = -ENOENT;
-		goto out_unlock;
+		return 0;
 	}
 
-	table->entry[filter_idx].spec |= EFX_EF10_FILTER_FLAG_BUSY;
-	spin_unlock_bh(&efx->filter_lock);
+	if (!(priority_mask & (1U << spec->priority)))
+		return -ENOENT;
 
 	if (spec->flags & EFX_FILTER_FLAG_RX_OVER_AUTO) {
 		/* Reset to an automatic filter */
@@ -4609,7 +4558,6 @@ static int efx_ef10_filter_remove_internal(struct efx_nic *efx,
 					  &efx->rss_context,
 					  true);
 
-		spin_lock_bh(&efx->filter_lock);
 		if (rc == 0)
 			*spec = new_spec;
 	} else {
@@ -4624,7 +4572,6 @@ static int efx_ef10_filter_remove_internal(struct efx_nic *efx,
 		rc = efx_mcdi_rpc_quiet(efx, MC_CMD_FILTER_OP,
 					inbuf, sizeof(inbuf), NULL, 0, NULL);
 
-		spin_lock_bh(&efx->filter_lock);
 		if ((rc == 0) || (rc == -ENOENT)) {
 			/* Filter removed OK or didn't actually exist */
 			kfree(spec);
@@ -4636,11 +4583,6 @@ static int efx_ef10_filter_remove_internal(struct efx_nic *efx,
 		}
 	}
 
-	table->entry[filter_idx].spec &= ~EFX_EF10_FILTER_FLAG_BUSY;
-	wake_up_all(&table->waitq);
-out_unlock:
-	spin_unlock_bh(&efx->filter_lock);
-	finish_wait(&table->waitq, &wait);
 	return rc;
 }
 
@@ -4648,17 +4590,33 @@ static int efx_ef10_filter_remove_safe(struct efx_nic *efx,
 				       enum efx_filter_priority priority,
 				       u32 filter_id)
 {
-	return efx_ef10_filter_remove_internal(efx, 1U << priority,
-					       filter_id, false);
+	struct efx_ef10_filter_table *table;
+	int rc;
+
+	down_read(&efx->filter_sem);
+	table = efx->filter_state;
+	down_write(&table->lock);
+	rc = efx_ef10_filter_remove_internal(efx, 1U << priority, filter_id,
+					     false);
+	up_write(&table->lock);
+	up_read(&efx->filter_sem);
+	return rc;
 }
 
+/* Caller must hold efx->filter_sem for read */
 static void efx_ef10_filter_remove_unsafe(struct efx_nic *efx,
 					  enum efx_filter_priority priority,
 					  u32 filter_id)
 {
+	struct efx_ef10_filter_table *table = efx->filter_state;
+
 	if (filter_id == EFX_EF10_FILTER_ID_INVALID)
 		return;
-	efx_ef10_filter_remove_internal(efx, 1U << priority, filter_id, true);
+
+	down_write(&table->lock);
+	efx_ef10_filter_remove_internal(efx, 1U << priority, filter_id,
+					true);
+	up_write(&table->lock);
 }
 
 static int efx_ef10_filter_get_safe(struct efx_nic *efx,
@@ -4666,11 +4624,13 @@ static int efx_ef10_filter_get_safe(struct efx_nic *efx,
 				    u32 filter_id, struct efx_filter_spec *spec)
 {
 	unsigned int filter_idx = efx_ef10_filter_get_unsafe_id(filter_id);
-	struct efx_ef10_filter_table *table = efx->filter_state;
 	const struct efx_filter_spec *saved_spec;
+	struct efx_ef10_filter_table *table;
 	int rc;
 
-	spin_lock_bh(&efx->filter_lock);
+	down_read(&efx->filter_sem);
+	table = efx->filter_state;
+	down_read(&table->lock);
 	saved_spec = efx_ef10_filter_entry_spec(table, filter_idx);
 	if (saved_spec && saved_spec->priority == priority &&
 	    efx_ef10_filter_pri(table, saved_spec) ==
@@ -4680,13 +4640,15 @@ static int efx_ef10_filter_get_safe(struct efx_nic *efx,
 	} else {
 		rc = -ENOENT;
 	}
-	spin_unlock_bh(&efx->filter_lock);
+	up_read(&table->lock);
+	up_read(&efx->filter_sem);
 	return rc;
 }
 
 static int efx_ef10_filter_clear_rx(struct efx_nic *efx,
-				     enum efx_filter_priority priority)
+				    enum efx_filter_priority priority)
 {
+	struct efx_ef10_filter_table *table;
 	unsigned int priority_mask;
 	unsigned int i;
 	int rc;
@@ -4694,31 +4656,40 @@ static int efx_ef10_filter_clear_rx(struct efx_nic *efx,
 	priority_mask = (((1U << (priority + 1)) - 1) &
 			 ~(1U << EFX_FILTER_PRI_AUTO));
 
+	down_read(&efx->filter_sem);
+	table = efx->filter_state;
+	down_write(&table->lock);
 	for (i = 0; i < HUNT_FILTER_TBL_ROWS; i++) {
 		rc = efx_ef10_filter_remove_internal(efx, priority_mask,
 						     i, true);
 		if (rc && rc != -ENOENT)
-			return rc;
+			break;
+		rc = 0;
 	}
 
-	return 0;
+	up_write(&table->lock);
+	up_read(&efx->filter_sem);
+	return rc;
 }
 
 static u32 efx_ef10_filter_count_rx_used(struct efx_nic *efx,
 					 enum efx_filter_priority priority)
 {
-	struct efx_ef10_filter_table *table = efx->filter_state;
+	struct efx_ef10_filter_table *table;
 	unsigned int filter_idx;
 	s32 count = 0;
 
-	spin_lock_bh(&efx->filter_lock);
+	down_read(&efx->filter_sem);
+	table = efx->filter_state;
+	down_read(&table->lock);
 	for (filter_idx = 0; filter_idx < HUNT_FILTER_TBL_ROWS; filter_idx++) {
 		if (table->entry[filter_idx].spec &&
 		    efx_ef10_filter_entry_spec(table, filter_idx)->priority ==
 		    priority)
 			++count;
 	}
-	spin_unlock_bh(&efx->filter_lock);
+	up_read(&table->lock);
+	up_read(&efx->filter_sem);
 	return count;
 }
 
@@ -4733,12 +4704,15 @@ static s32 efx_ef10_filter_get_rx_ids(struct efx_nic *efx,
 				      enum efx_filter_priority priority,
 				      u32 *buf, u32 size)
 {
-	struct efx_ef10_filter_table *table = efx->filter_state;
+	struct efx_ef10_filter_table *table;
 	struct efx_filter_spec *spec;
 	unsigned int filter_idx;
 	s32 count = 0;
 
-	spin_lock_bh(&efx->filter_lock);
+	down_read(&efx->filter_sem);
+	table = efx->filter_state;
+	down_read(&table->lock);
+
 	for (filter_idx = 0; filter_idx < HUNT_FILTER_TBL_ROWS; filter_idx++) {
 		spec = efx_ef10_filter_entry_spec(table, filter_idx);
 		if (spec && spec->priority == priority) {
@@ -4752,73 +4726,44 @@ static s32 efx_ef10_filter_get_rx_ids(struct efx_nic *efx,
 					filter_idx);
 		}
 	}
-	spin_unlock_bh(&efx->filter_lock);
+	up_read(&table->lock);
+	up_read(&efx->filter_sem);
 	return count;
 }
 
 #ifdef CONFIG_RFS_ACCEL
 
-static void
-efx_ef10_filter_rfs_expire_complete(struct efx_nic *efx,
-				    unsigned long filter_idx,
-				    int rc, efx_dword_t *outbuf,
-				    size_t outlen_actual);
-
 static bool efx_ef10_filter_rfs_expire_one(struct efx_nic *efx, u32 flow_id,
 					   unsigned int filter_idx)
 {
-	struct efx_ef10_filter_table *table = efx->filter_state;
+	struct efx_ef10_filter_table *table;
 	struct efx_filter_spec *spec;
-	MCDI_DECLARE_BUF(inbuf,
-			 MC_CMD_FILTER_OP_IN_HANDLE_OFST +
-			 MC_CMD_FILTER_OP_IN_HANDLE_LEN);
-	bool ret = true;
+	bool ret;
 
-	spin_lock_bh(&efx->filter_lock);
+	down_read(&efx->filter_sem);
+	table = efx->filter_state;
+	down_write(&table->lock);
 	spec = efx_ef10_filter_entry_spec(table, filter_idx);
-	if (!spec ||
-	    (table->entry[filter_idx].spec & EFX_EF10_FILTER_FLAG_BUSY) ||
-	    spec->priority != EFX_FILTER_PRI_HINT ||
-	    !rps_may_expire_flow(efx->net_dev, spec->dmaq_id,
+
+	if (!spec || spec->priority != EFX_FILTER_PRI_HINT) {
+		ret = true;
+		goto out_unlock;
+	}
+
+	if (!rps_may_expire_flow(efx->net_dev, spec->dmaq_id,
 				 flow_id, filter_idx)) {
 		ret = false;
 		goto out_unlock;
 	}
 
-	MCDI_SET_DWORD(inbuf, FILTER_OP_IN_OP,
-		       MC_CMD_FILTER_OP_IN_OP_REMOVE);
-	MCDI_SET_QWORD(inbuf, FILTER_OP_IN_HANDLE,
-		       table->entry[filter_idx].handle);
-	if (efx_mcdi_rpc_async(efx, MC_CMD_FILTER_OP, inbuf, sizeof(inbuf), 0,
-			       efx_ef10_filter_rfs_expire_complete, filter_idx))
-		ret = false;
-	else
-		table->entry[filter_idx].spec |= EFX_EF10_FILTER_FLAG_BUSY;
+	ret = efx_ef10_filter_remove_internal(efx, 1U << spec->priority,
+					      filter_idx, true) == 0;
 out_unlock:
-	spin_unlock_bh(&efx->filter_lock);
+	up_write(&table->lock);
+	up_read(&efx->filter_sem);
 	return ret;
 }
 
-static void
-efx_ef10_filter_rfs_expire_complete(struct efx_nic *efx,
-				    unsigned long filter_idx,
-				    int rc, efx_dword_t *outbuf,
-				    size_t outlen_actual)
-{
-	struct efx_ef10_filter_table *table = efx->filter_state;
-	struct efx_filter_spec *spec =
-		efx_ef10_filter_entry_spec(table, filter_idx);
-
-	spin_lock_bh(&efx->filter_lock);
-	if (rc == 0) {
-		kfree(spec);
-		efx_ef10_filter_set_entry(table, filter_idx, NULL, 0);
-	}
-	table->entry[filter_idx].spec &= ~EFX_EF10_FILTER_FLAG_BUSY;
-	wake_up_all(&table->waitq);
-	spin_unlock_bh(&efx->filter_lock);
-}
-
 #endif /* CONFIG_RFS_ACCEL */
 
 static int efx_ef10_filter_match_flags_from_mcdi(bool encap, u32 mcdi_flags)
@@ -5011,9 +4956,9 @@ static int efx_ef10_filter_table_probe(struct efx_nic *efx)
 	table->vlan_filter =
 		!!(efx->net_dev->features & NETIF_F_HW_VLAN_CTAG_FILTER);
 	INIT_LIST_HEAD(&table->vlan_list);
+	init_rwsem(&table->lock);
 
 	efx->filter_state = table;
-	init_waitqueue_head(&table->waitq);
 
 	list_for_each_entry(vlan, &nic_data->vlan_list, list) {
 		rc = efx_ef10_filter_add_vlan(efx, vlan->vid);
@@ -5055,7 +5000,7 @@ static void efx_ef10_filter_table_restore(struct efx_nic *efx)
 	if (!table)
 		return;
 
-	spin_lock_bh(&efx->filter_lock);
+	down_write(&table->lock);
 
 	for (filter_idx = 0; filter_idx < HUNT_FILTER_TBL_ROWS; filter_idx++) {
 		spec = efx_ef10_filter_entry_spec(table, filter_idx);
@@ -5093,15 +5038,11 @@ static void efx_ef10_filter_table_restore(struct efx_nic *efx)
 			}
 		}
 
-		table->entry[filter_idx].spec |= EFX_EF10_FILTER_FLAG_BUSY;
-		spin_unlock_bh(&efx->filter_lock);
-
 		rc = efx_ef10_filter_push(efx, spec,
 					  &table->entry[filter_idx].handle,
 					  ctx, false);
 		if (rc)
 			failed++;
-		spin_lock_bh(&efx->filter_lock);
 
 		if (rc) {
 not_restored:
@@ -5113,13 +5054,10 @@ static void efx_ef10_filter_table_restore(struct efx_nic *efx)
 
 			kfree(spec);
 			efx_ef10_filter_set_entry(table, filter_idx, NULL, 0);
-		} else {
-			table->entry[filter_idx].spec &=
-				~EFX_EF10_FILTER_FLAG_BUSY;
 		}
 	}
 
-	spin_unlock_bh(&efx->filter_lock);
+	up_write(&table->lock);
 
 	/* This can happen validly if the MC's capabilities have changed, so
 	 * is not an error.
@@ -5187,6 +5125,8 @@ static void efx_ef10_filter_mark_one_old(struct efx_nic *efx, uint16_t *id)
 	struct efx_ef10_filter_table *table = efx->filter_state;
 	unsigned int filter_idx;
 
+	efx_rwsem_assert_write_locked(&table->lock);
+
 	if (*id != EFX_EF10_FILTER_ID_INVALID) {
 		filter_idx = efx_ef10_filter_get_unsafe_id(*id);
 		if (!table->entry[filter_idx].spec)
@@ -5222,10 +5162,10 @@ static void efx_ef10_filter_mark_old(struct efx_nic *efx)
 	struct efx_ef10_filter_table *table = efx->filter_state;
 	struct efx_ef10_filter_vlan *vlan;
 
-	spin_lock_bh(&efx->filter_lock);
+	down_write(&table->lock);
 	list_for_each_entry(vlan, &table->vlan_list, list)
 		_efx_ef10_filter_vlan_mark_old(efx, vlan);
-	spin_unlock_bh(&efx->filter_lock);
+	up_write(&table->lock);
 }
 
 static void efx_ef10_filter_uc_addr_list(struct efx_nic *efx)
@@ -5502,10 +5442,7 @@ static int efx_ef10_filter_insert_def(struct efx_nic *efx,
 	return rc;
 }
 
-/* Remove filters that weren't renewed.  Since nothing else changes the AUTO_OLD
- * flag or removes these filters, we don't need to hold the filter_lock while
- * scanning for these filters.
- */
+/* Remove filters that weren't renewed. */
 static void efx_ef10_filter_remove_old(struct efx_nic *efx)
 {
 	struct efx_ef10_filter_table *table = efx->filter_state;
@@ -5514,6 +5451,7 @@ static void efx_ef10_filter_remove_old(struct efx_nic *efx)
 	int rc;
 	int i;
 
+	down_write(&table->lock);
 	for (i = 0; i < HUNT_FILTER_TBL_ROWS; i++) {
 		if (READ_ONCE(table->entry[i].spec) &
 		    EFX_EF10_FILTER_FLAG_AUTO_OLD) {
@@ -5525,6 +5463,7 @@ static void efx_ef10_filter_remove_old(struct efx_nic *efx)
 				remove_failed++;
 		}
 	}
+	up_write(&table->lock);
 
 	if (remove_failed)
 		netif_info(efx, drv, efx->net_dev,
diff --git a/drivers/net/ethernet/sfc/efx.c b/drivers/net/ethernet/sfc/efx.c
index 0be93abdb2ad..cb9273016916 100644
--- a/drivers/net/ethernet/sfc/efx.c
+++ b/drivers/net/ethernet/sfc/efx.c
@@ -1783,7 +1783,6 @@ static int efx_probe_filters(struct efx_nic *efx)
 {
 	int rc;
 
-	spin_lock_init(&efx->filter_lock);
 	init_rwsem(&efx->filter_sem);
 	mutex_lock(&efx->mac_lock);
 	down_write(&efx->filter_sem);
diff --git a/drivers/net/ethernet/sfc/net_driver.h b/drivers/net/ethernet/sfc/net_driver.h
index fe0c5322454d..6f37a4dbe268 100644
--- a/drivers/net/ethernet/sfc/net_driver.h
+++ b/drivers/net/ethernet/sfc/net_driver.h
@@ -836,7 +836,7 @@ struct efx_rss_context {
  * @loopback_mode: Loopback status
  * @loopback_modes: Supported loopback mode bitmask
  * @loopback_selftest: Offline self-test private state
- * @filter_sem: Filter table rw_semaphore, for freeing the table
+ * @filter_sem: Filter table rw_semaphore, protects existence of @filter_state
  * @filter_lock: Filter table lock, for mere content changes
  * @filter_state: Architecture-dependent filter table state
  * @rps_mutex: Protects RPS state of all channels

^ permalink raw reply related	[flat|nested] 8+ messages in thread

* [PATCH net-next 3/6] sfc: use a semaphore to lock farch filters too
  2018-03-27 16:40 [PATCH net-next 0/6] sfc: rework locking around filter management Edward Cree
  2018-03-27 16:41 ` [PATCH net-next 1/6] sfc: replace asynchronous filter operations Edward Cree
  2018-03-27 16:42 ` [PATCH net-next 2/6] sfc: give ef10 its own rwsem in the filter table instead of filter_lock Edward Cree
@ 2018-03-27 16:42 ` Edward Cree
  2018-03-27 16:44 ` [PATCH net-next 4/6] sfc: return a better error if filter insertion collides with MC reboot Edward Cree
                   ` (3 subsequent siblings)
  6 siblings, 0 replies; 8+ messages in thread
From: Edward Cree @ 2018-03-27 16:42 UTC (permalink / raw)
  To: linux-net-drivers, David Miller; +Cc: netdev

With this change, the spinlock efx->filter_lock is no longer used and is
 thus removed.

Signed-off-by: Edward Cree <ecree@solarflare.com>
---
 drivers/net/ethernet/sfc/farch.c      | 67 +++++++++++++++++------------------
 drivers/net/ethernet/sfc/net_driver.h |  2 --
 2 files changed, 33 insertions(+), 36 deletions(-)

diff --git a/drivers/net/ethernet/sfc/farch.c b/drivers/net/ethernet/sfc/farch.c
index 6b5ca569db5e..4a19c7efdf8d 100644
--- a/drivers/net/ethernet/sfc/farch.c
+++ b/drivers/net/ethernet/sfc/farch.c
@@ -1878,6 +1878,7 @@ struct efx_farch_filter_table {
 };
 
 struct efx_farch_filter_state {
+	struct rw_semaphore lock; /* Protects table contents */
 	struct efx_farch_filter_table table[EFX_FARCH_FILTER_TABLE_COUNT];
 };
 
@@ -2397,9 +2398,13 @@ s32 efx_farch_filter_insert(struct efx_nic *efx,
 	if (rc)
 		return rc;
 
+	down_write(&state->lock);
+
 	table = &state->table[efx_farch_filter_spec_table_id(&spec)];
-	if (table->size == 0)
-		return -EINVAL;
+	if (table->size == 0) {
+		rc = -EINVAL;
+		goto out_unlock;
+	}
 
 	netif_vdbg(efx, hw, efx->net_dev,
 		   "%s: type %d search_limit=%d", __func__, spec.type,
@@ -2412,8 +2417,6 @@ s32 efx_farch_filter_insert(struct efx_nic *efx,
 			     EFX_FARCH_FILTER_MC_DEF - EFX_FARCH_FILTER_UC_DEF);
 		rep_index = spec.type - EFX_FARCH_FILTER_UC_DEF;
 		ins_index = rep_index;
-
-		spin_lock_bh(&efx->filter_lock);
 	} else {
 		/* Search concurrently for
 		 * (1) a filter to be replaced (rep_index): any filter
@@ -2443,8 +2446,6 @@ s32 efx_farch_filter_insert(struct efx_nic *efx,
 		ins_index = -1;
 		depth = 1;
 
-		spin_lock_bh(&efx->filter_lock);
-
 		for (;;) {
 			if (!test_bit(i, table->used_bitmap)) {
 				if (ins_index < 0)
@@ -2463,7 +2464,7 @@ s32 efx_farch_filter_insert(struct efx_nic *efx,
 				/* Case (b) */
 				if (ins_index < 0) {
 					rc = -EBUSY;
-					goto out;
+					goto out_unlock;
 				}
 				rep_index = -1;
 				break;
@@ -2483,11 +2484,11 @@ s32 efx_farch_filter_insert(struct efx_nic *efx,
 
 		if (spec.priority == saved_spec->priority && !replace_equal) {
 			rc = -EEXIST;
-			goto out;
+			goto out_unlock;
 		}
 		if (spec.priority < saved_spec->priority) {
 			rc = -EPERM;
-			goto out;
+			goto out_unlock;
 		}
 		if (saved_spec->priority == EFX_FILTER_PRI_AUTO ||
 		    saved_spec->flags & EFX_FILTER_FLAG_RX_OVER_AUTO)
@@ -2528,8 +2529,8 @@ s32 efx_farch_filter_insert(struct efx_nic *efx,
 		   __func__, spec.type, ins_index, spec.dmaq_id);
 	rc = efx_farch_filter_make_id(&spec, ins_index);
 
-out:
-	spin_unlock_bh(&efx->filter_lock);
+out_unlock:
+	up_write(&state->lock);
 	return rc;
 }
 
@@ -2604,11 +2605,11 @@ int efx_farch_filter_remove_safe(struct efx_nic *efx,
 	filter_idx = efx_farch_filter_id_index(filter_id);
 	if (filter_idx >= table->size)
 		return -ENOENT;
+	down_write(&state->lock);
 	spec = &table->spec[filter_idx];
 
-	spin_lock_bh(&efx->filter_lock);
 	rc = efx_farch_filter_remove(efx, table, filter_idx, priority);
-	spin_unlock_bh(&efx->filter_lock);
+	up_write(&state->lock);
 
 	return rc;
 }
@@ -2622,30 +2623,28 @@ int efx_farch_filter_get_safe(struct efx_nic *efx,
 	struct efx_farch_filter_table *table;
 	struct efx_farch_filter_spec *spec;
 	unsigned int filter_idx;
-	int rc;
+	int rc = -ENOENT;
+
+	down_read(&state->lock);
 
 	table_id = efx_farch_filter_id_table_id(filter_id);
 	if ((unsigned int)table_id >= EFX_FARCH_FILTER_TABLE_COUNT)
-		return -ENOENT;
+		goto out_unlock;
 	table = &state->table[table_id];
 
 	filter_idx = efx_farch_filter_id_index(filter_id);
 	if (filter_idx >= table->size)
-		return -ENOENT;
+		goto out_unlock;
 	spec = &table->spec[filter_idx];
 
-	spin_lock_bh(&efx->filter_lock);
-
 	if (test_bit(filter_idx, table->used_bitmap) &&
 	    spec->priority == priority) {
 		efx_farch_filter_to_gen_spec(spec_buf, spec);
 		rc = 0;
-	} else {
-		rc = -ENOENT;
 	}
 
-	spin_unlock_bh(&efx->filter_lock);
-
+out_unlock:
+	up_read(&state->lock);
 	return rc;
 }
 
@@ -2658,13 +2657,13 @@ efx_farch_filter_table_clear(struct efx_nic *efx,
 	struct efx_farch_filter_table *table = &state->table[table_id];
 	unsigned int filter_idx;
 
-	spin_lock_bh(&efx->filter_lock);
+	down_write(&state->lock);
 	for (filter_idx = 0; filter_idx < table->size; ++filter_idx) {
 		if (table->spec[filter_idx].priority != EFX_FILTER_PRI_AUTO)
 			efx_farch_filter_remove(efx, table,
 						filter_idx, priority);
 	}
-	spin_unlock_bh(&efx->filter_lock);
+	up_write(&state->lock);
 }
 
 int efx_farch_filter_clear_rx(struct efx_nic *efx,
@@ -2688,7 +2687,7 @@ u32 efx_farch_filter_count_rx_used(struct efx_nic *efx,
 	unsigned int filter_idx;
 	u32 count = 0;
 
-	spin_lock_bh(&efx->filter_lock);
+	down_read(&state->lock);
 
 	for (table_id = EFX_FARCH_FILTER_TABLE_RX_IP;
 	     table_id <= EFX_FARCH_FILTER_TABLE_RX_DEF;
@@ -2701,7 +2700,7 @@ u32 efx_farch_filter_count_rx_used(struct efx_nic *efx,
 		}
 	}
 
-	spin_unlock_bh(&efx->filter_lock);
+	up_read(&state->lock);
 
 	return count;
 }
@@ -2716,7 +2715,7 @@ s32 efx_farch_filter_get_rx_ids(struct efx_nic *efx,
 	unsigned int filter_idx;
 	s32 count = 0;
 
-	spin_lock_bh(&efx->filter_lock);
+	down_read(&state->lock);
 
 	for (table_id = EFX_FARCH_FILTER_TABLE_RX_IP;
 	     table_id <= EFX_FARCH_FILTER_TABLE_RX_DEF;
@@ -2735,7 +2734,7 @@ s32 efx_farch_filter_get_rx_ids(struct efx_nic *efx,
 		}
 	}
 out:
-	spin_unlock_bh(&efx->filter_lock);
+	up_read(&state->lock);
 
 	return count;
 }
@@ -2749,7 +2748,7 @@ void efx_farch_filter_table_restore(struct efx_nic *efx)
 	efx_oword_t filter;
 	unsigned int filter_idx;
 
-	spin_lock_bh(&efx->filter_lock);
+	down_write(&state->lock);
 
 	for (table_id = 0; table_id < EFX_FARCH_FILTER_TABLE_COUNT; table_id++) {
 		table = &state->table[table_id];
@@ -2770,7 +2769,7 @@ void efx_farch_filter_table_restore(struct efx_nic *efx)
 	efx_farch_filter_push_rx_config(efx);
 	efx_farch_filter_push_tx_limits(efx);
 
-	spin_unlock_bh(&efx->filter_lock);
+	up_write(&state->lock);
 }
 
 void efx_farch_filter_table_remove(struct efx_nic *efx)
@@ -2864,7 +2863,7 @@ void efx_farch_filter_update_rx_scatter(struct efx_nic *efx)
 	efx_oword_t filter;
 	unsigned int filter_idx;
 
-	spin_lock_bh(&efx->filter_lock);
+	down_write(&state->lock);
 
 	for (table_id = EFX_FARCH_FILTER_TABLE_RX_IP;
 	     table_id <= EFX_FARCH_FILTER_TABLE_RX_DEF;
@@ -2896,7 +2895,7 @@ void efx_farch_filter_update_rx_scatter(struct efx_nic *efx)
 
 	efx_farch_filter_push_rx_config(efx);
 
-	spin_unlock_bh(&efx->filter_lock);
+	up_write(&state->lock);
 }
 
 #ifdef CONFIG_RFS_ACCEL
@@ -2908,7 +2907,7 @@ bool efx_farch_filter_rfs_expire_one(struct efx_nic *efx, u32 flow_id,
 	struct efx_farch_filter_table *table;
 	bool ret = false;
 
-	spin_lock_bh(&efx->filter_lock);
+	down_write(&state->lock);
 	table = &state->table[EFX_FARCH_FILTER_TABLE_RX_IP];
 	if (test_bit(index, table->used_bitmap) &&
 	    table->spec[index].priority == EFX_FILTER_PRI_HINT &&
@@ -2918,7 +2917,7 @@ bool efx_farch_filter_rfs_expire_one(struct efx_nic *efx, u32 flow_id,
 		ret = true;
 	}
 
-	spin_unlock_bh(&efx->filter_lock);
+	up_write(&state->lock);
 	return ret;
 }
 
diff --git a/drivers/net/ethernet/sfc/net_driver.h b/drivers/net/ethernet/sfc/net_driver.h
index 6f37a4dbe268..ef6fc931afb8 100644
--- a/drivers/net/ethernet/sfc/net_driver.h
+++ b/drivers/net/ethernet/sfc/net_driver.h
@@ -837,7 +837,6 @@ struct efx_rss_context {
  * @loopback_modes: Supported loopback mode bitmask
  * @loopback_selftest: Offline self-test private state
  * @filter_sem: Filter table rw_semaphore, protects existence of @filter_state
- * @filter_lock: Filter table lock, for mere content changes
  * @filter_state: Architecture-dependent filter table state
  * @rps_mutex: Protects RPS state of all channels
  * @rps_expire_channel: Next channel to check for expiry
@@ -990,7 +989,6 @@ struct efx_nic {
 	void *loopback_selftest;
 
 	struct rw_semaphore filter_sem;
-	spinlock_t filter_lock;
 	void *filter_state;
 #ifdef CONFIG_RFS_ACCEL
 	struct mutex rps_mutex;

^ permalink raw reply related	[flat|nested] 8+ messages in thread

* [PATCH net-next 4/6] sfc: return a better error if filter insertion collides with MC reboot
  2018-03-27 16:40 [PATCH net-next 0/6] sfc: rework locking around filter management Edward Cree
                   ` (2 preceding siblings ...)
  2018-03-27 16:42 ` [PATCH net-next 3/6] sfc: use a semaphore to lock farch filters too Edward Cree
@ 2018-03-27 16:44 ` Edward Cree
  2018-03-27 16:44 ` [PATCH net-next 5/6] sfc: protect list of RSS contexts under a mutex Edward Cree
                   ` (2 subsequent siblings)
  6 siblings, 0 replies; 8+ messages in thread
From: Edward Cree @ 2018-03-27 16:44 UTC (permalink / raw)
  To: linux-net-drivers, David Miller; +Cc: netdev

If some other operation gets the MCDI lock ahead of us and performs an MC
 reboot, then our attempt to insert the filter will fail with EINVAL,
 because the destination VI (spec->dmaq_id, MC_CMD_FILTER_OP_IN_RX_QUEUE) does
 not exist.  But the caller's request (which might e.g. be an ethtool ntuple
 request from userland) isn't invalid, it just got unlucky; so return EAGAIN.

Signed-off-by: Edward Cree <ecree@solarflare.com>
---
 drivers/net/ethernet/sfc/ef10.c | 7 +++++++
 1 file changed, 7 insertions(+)

diff --git a/drivers/net/ethernet/sfc/ef10.c b/drivers/net/ethernet/sfc/ef10.c
index 9db1b9144e70..9a139c390037 100644
--- a/drivers/net/ethernet/sfc/ef10.c
+++ b/drivers/net/ethernet/sfc/ef10.c
@@ -4301,6 +4301,7 @@ static s32 efx_ef10_filter_insert(struct efx_nic *efx,
 				  bool replace_equal)
 {
 	DECLARE_BITMAP(mc_rem_map, EFX_EF10_FILTER_SEARCH_LIMIT);
+	struct efx_ef10_nic_data *nic_data = efx->nic_data;
 	struct efx_ef10_filter_table *table;
 	struct efx_filter_spec *saved_spec;
 	struct efx_rss_context *ctx = NULL;
@@ -4427,6 +4428,12 @@ static s32 efx_ef10_filter_insert(struct efx_nic *efx,
 	rc = efx_ef10_filter_push(efx, spec, &table->entry[ins_index].handle,
 				  ctx, replacing);
 
+	if (rc == -EINVAL && nic_data->must_realloc_vis)
+		/* The MC rebooted under us, causing it to reject our filter
+		 * insertion as pointing to an invalid VI (spec->dmaq_id).
+		 */
+		rc = -EAGAIN;
+
 	/* Finalise the software table entry */
 	if (rc == 0) {
 		if (replacing) {

^ permalink raw reply related	[flat|nested] 8+ messages in thread

* [PATCH net-next 5/6] sfc: protect list of RSS contexts under a mutex
  2018-03-27 16:40 [PATCH net-next 0/6] sfc: rework locking around filter management Edward Cree
                   ` (3 preceding siblings ...)
  2018-03-27 16:44 ` [PATCH net-next 4/6] sfc: return a better error if filter insertion collides with MC reboot Edward Cree
@ 2018-03-27 16:44 ` Edward Cree
  2018-03-27 16:44 ` [PATCH net-next 6/6] sfc: fix flow type handling for RSS filters Edward Cree
  2018-03-27 17:34 ` [PATCH net-next 0/6] sfc: rework locking around filter management David Miller
  6 siblings, 0 replies; 8+ messages in thread
From: Edward Cree @ 2018-03-27 16:44 UTC (permalink / raw)
  To: linux-net-drivers, David Miller; +Cc: netdev

Otherwise races are possible between ethtool ops and
 efx_ef10_rx_restore_rss_contexts().
Also, don't try to perform the restore on every reset, only after an MC
 reboot, otherwise we'll leak RSS contexts on the NIC.

Fixes: 42356d9a137b ("sfc: support RSS spreading of ethtool ntuple filters")
Signed-off-by: Edward Cree <ecree@solarflare.com>
---
 drivers/net/ethernet/sfc/ef10.c       | 32 ++++++++++++++---
 drivers/net/ethernet/sfc/efx.c        | 14 ++++++--
 drivers/net/ethernet/sfc/efx.h        |  4 +--
 drivers/net/ethernet/sfc/ethtool.c    | 66 +++++++++++++++++++++++------------
 drivers/net/ethernet/sfc/net_driver.h |  2 ++
 drivers/net/ethernet/sfc/nic.h        |  3 ++
 6 files changed, 89 insertions(+), 32 deletions(-)

diff --git a/drivers/net/ethernet/sfc/ef10.c b/drivers/net/ethernet/sfc/ef10.c
index 9a139c390037..c4c45c94da77 100644
--- a/drivers/net/ethernet/sfc/ef10.c
+++ b/drivers/net/ethernet/sfc/ef10.c
@@ -1499,6 +1499,7 @@ static void efx_ef10_reset_mc_allocations(struct efx_nic *efx)
 
 	/* All our allocations have been reset */
 	nic_data->must_realloc_vis = true;
+	nic_data->must_restore_rss_contexts = true;
 	nic_data->must_restore_filters = true;
 	nic_data->must_restore_piobufs = true;
 	efx_ef10_forget_old_piobufs(efx);
@@ -2899,6 +2900,8 @@ static int efx_ef10_rx_push_rss_context_config(struct efx_nic *efx,
 {
 	int rc;
 
+	WARN_ON(!mutex_is_locked(&efx->rss_lock));
+
 	if (ctx->context_id == EFX_EF10_RSS_CONTEXT_INVALID) {
 		rc = efx_ef10_alloc_rss_context(efx, true, ctx, NULL);
 		if (rc)
@@ -2929,6 +2932,8 @@ static int efx_ef10_rx_pull_rss_context_config(struct efx_nic *efx,
 	size_t outlen;
 	int rc, i;
 
+	WARN_ON(!mutex_is_locked(&efx->rss_lock));
+
 	BUILD_BUG_ON(MC_CMD_RSS_CONTEXT_GET_TABLE_IN_LEN !=
 		     MC_CMD_RSS_CONTEXT_GET_KEY_IN_LEN);
 
@@ -2972,14 +2977,25 @@ static int efx_ef10_rx_pull_rss_context_config(struct efx_nic *efx,
 
 static int efx_ef10_rx_pull_rss_config(struct efx_nic *efx)
 {
-	return efx_ef10_rx_pull_rss_context_config(efx, &efx->rss_context);
+	int rc;
+
+	mutex_lock(&efx->rss_lock);
+	rc = efx_ef10_rx_pull_rss_context_config(efx, &efx->rss_context);
+	mutex_unlock(&efx->rss_lock);
+	return rc;
 }
 
 static void efx_ef10_rx_restore_rss_contexts(struct efx_nic *efx)
 {
+	struct efx_ef10_nic_data *nic_data = efx->nic_data;
 	struct efx_rss_context *ctx;
 	int rc;
 
+	WARN_ON(!mutex_is_locked(&efx->rss_lock));
+
+	if (!nic_data->must_restore_rss_contexts)
+		return;
+
 	list_for_each_entry(ctx, &efx->rss_context.list, list) {
 		/* previous NIC RSS context is gone */
 		ctx->context_id = EFX_EF10_RSS_CONTEXT_INVALID;
@@ -2993,6 +3009,7 @@ static void efx_ef10_rx_restore_rss_contexts(struct efx_nic *efx)
 				   "; RSS filters may fail to be applied\n",
 				   ctx->user_id, rc);
 	}
+	nic_data->must_restore_rss_contexts = false;
 }
 
 static int efx_ef10_pf_rx_push_rss_config(struct efx_nic *efx, bool user,
@@ -4307,6 +4324,7 @@ static s32 efx_ef10_filter_insert(struct efx_nic *efx,
 	struct efx_rss_context *ctx = NULL;
 	unsigned int match_pri, hash;
 	unsigned int priv_flags;
+	bool rss_locked = false;
 	bool replacing = false;
 	unsigned int depth, i;
 	int ins_index = -1;
@@ -4336,9 +4354,10 @@ static s32 efx_ef10_filter_insert(struct efx_nic *efx,
 		bitmap_zero(mc_rem_map, EFX_EF10_FILTER_SEARCH_LIMIT);
 
 	if (spec->flags & EFX_FILTER_FLAG_RX_RSS) {
+		mutex_lock(&efx->rss_lock);
+		rss_locked = true;
 		if (spec->rss_context)
-			ctx = efx_find_rss_context_entry(spec->rss_context,
-							 &efx->rss_context.list);
+			ctx = efx_find_rss_context_entry(efx, spec->rss_context);
 		else
 			ctx = &efx->rss_context;
 		if (!ctx) {
@@ -4501,6 +4520,8 @@ static s32 efx_ef10_filter_insert(struct efx_nic *efx,
 		rc = efx_ef10_make_filter_id(match_pri, ins_index);
 
 out_unlock:
+	if (rss_locked)
+		mutex_unlock(&efx->rss_lock);
 	up_write(&table->lock);
 	up_read(&efx->filter_sem);
 	return rc;
@@ -5008,6 +5029,7 @@ static void efx_ef10_filter_table_restore(struct efx_nic *efx)
 		return;
 
 	down_write(&table->lock);
+	mutex_lock(&efx->rss_lock);
 
 	for (filter_idx = 0; filter_idx < HUNT_FILTER_TBL_ROWS; filter_idx++) {
 		spec = efx_ef10_filter_entry_spec(table, filter_idx);
@@ -5024,8 +5046,7 @@ static void efx_ef10_filter_table_restore(struct efx_nic *efx)
 			goto not_restored;
 		}
 		if (spec->rss_context)
-			ctx = efx_find_rss_context_entry(spec->rss_context,
-							 &efx->rss_context.list);
+			ctx = efx_find_rss_context_entry(efx, spec->rss_context);
 		else
 			ctx = &efx->rss_context;
 		if (spec->flags & EFX_FILTER_FLAG_RX_RSS) {
@@ -5064,6 +5085,7 @@ static void efx_ef10_filter_table_restore(struct efx_nic *efx)
 		}
 	}
 
+	mutex_unlock(&efx->rss_lock);
 	up_write(&table->lock);
 
 	/* This can happen validly if the MC's capabilities have changed, so
diff --git a/drivers/net/ethernet/sfc/efx.c b/drivers/net/ethernet/sfc/efx.c
index cb9273016916..692dd729ee2a 100644
--- a/drivers/net/ethernet/sfc/efx.c
+++ b/drivers/net/ethernet/sfc/efx.c
@@ -2657,6 +2657,7 @@ void efx_reset_down(struct efx_nic *efx, enum reset_type method)
 	efx_disable_interrupts(efx);
 
 	mutex_lock(&efx->mac_lock);
+	mutex_lock(&efx->rss_lock);
 	if (efx->port_initialized && method != RESET_TYPE_INVISIBLE &&
 	    method != RESET_TYPE_DATAPATH)
 		efx->phy_op->fini(efx);
@@ -2712,6 +2713,7 @@ int efx_reset_up(struct efx_nic *efx, enum reset_type method, bool ok)
 
 	if (efx->type->rx_restore_rss_contexts)
 		efx->type->rx_restore_rss_contexts(efx);
+	mutex_unlock(&efx->rss_lock);
 	down_read(&efx->filter_sem);
 	efx_restore_filters(efx);
 	up_read(&efx->filter_sem);
@@ -2730,6 +2732,7 @@ int efx_reset_up(struct efx_nic *efx, enum reset_type method, bool ok)
 fail:
 	efx->port_initialized = false;
 
+	mutex_unlock(&efx->rss_lock);
 	mutex_unlock(&efx->mac_lock);
 
 	return rc;
@@ -3016,6 +3019,7 @@ static int efx_init_struct(struct efx_nic *efx,
 	efx->rx_packet_ts_offset =
 		efx->type->rx_ts_offset - efx->type->rx_prefix_size;
 	INIT_LIST_HEAD(&efx->rss_context.list);
+	mutex_init(&efx->rss_lock);
 	spin_lock_init(&efx->stats_lock);
 	efx->vi_stride = EFX_DEFAULT_VI_STRIDE;
 	efx->num_mac_stats = MC_CMD_MAC_NSTATS;
@@ -3091,11 +3095,14 @@ void efx_update_sw_stats(struct efx_nic *efx, u64 *stats)
 /* RSS contexts.  We're using linked lists and crappy O(n) algorithms, because
  * (a) this is an infrequent control-plane operation and (b) n is small (max 64)
  */
-struct efx_rss_context *efx_alloc_rss_context_entry(struct list_head *head)
+struct efx_rss_context *efx_alloc_rss_context_entry(struct efx_nic *efx)
 {
+	struct list_head *head = &efx->rss_context.list;
 	struct efx_rss_context *ctx, *new;
 	u32 id = 1; /* Don't use zero, that refers to the master RSS context */
 
+	WARN_ON(!mutex_is_locked(&efx->rss_lock));
+
 	/* Search for first gap in the numbering */
 	list_for_each_entry(ctx, head, list) {
 		if (ctx->user_id != id)
@@ -3121,10 +3128,13 @@ struct efx_rss_context *efx_alloc_rss_context_entry(struct list_head *head)
 	return new;
 }
 
-struct efx_rss_context *efx_find_rss_context_entry(u32 id, struct list_head *head)
+struct efx_rss_context *efx_find_rss_context_entry(struct efx_nic *efx, u32 id)
 {
+	struct list_head *head = &efx->rss_context.list;
 	struct efx_rss_context *ctx;
 
+	WARN_ON(!mutex_is_locked(&efx->rss_lock));
+
 	list_for_each_entry(ctx, head, list)
 		if (ctx->user_id == id)
 			return ctx;
diff --git a/drivers/net/ethernet/sfc/efx.h b/drivers/net/ethernet/sfc/efx.h
index 545c2ea1622e..a3140e16fcef 100644
--- a/drivers/net/ethernet/sfc/efx.h
+++ b/drivers/net/ethernet/sfc/efx.h
@@ -187,8 +187,8 @@ static inline void efx_filter_rfs_expire(struct work_struct *data) {}
 bool efx_filter_is_mc_recipient(const struct efx_filter_spec *spec);
 
 /* RSS contexts */
-struct efx_rss_context *efx_alloc_rss_context_entry(struct list_head *list);
-struct efx_rss_context *efx_find_rss_context_entry(u32 id, struct list_head *list);
+struct efx_rss_context *efx_alloc_rss_context_entry(struct efx_nic *efx);
+struct efx_rss_context *efx_find_rss_context_entry(struct efx_nic *efx, u32 id);
 void efx_free_rss_context_entry(struct efx_rss_context *ctx);
 static inline bool efx_rss_active(struct efx_rss_context *ctx)
 {
diff --git a/drivers/net/ethernet/sfc/ethtool.c b/drivers/net/ethernet/sfc/ethtool.c
index 64049e71e6e7..de1fac6cd34f 100644
--- a/drivers/net/ethernet/sfc/ethtool.c
+++ b/drivers/net/ethernet/sfc/ethtool.c
@@ -979,7 +979,7 @@ efx_ethtool_get_rxnfc(struct net_device *net_dev,
 {
 	struct efx_nic *efx = netdev_priv(net_dev);
 	u32 rss_context = 0;
-	s32 rc;
+	s32 rc = 0;
 
 	switch (info->cmd) {
 	case ETHTOOL_GRXRINGS:
@@ -989,15 +989,17 @@ efx_ethtool_get_rxnfc(struct net_device *net_dev,
 	case ETHTOOL_GRXFH: {
 		struct efx_rss_context *ctx = &efx->rss_context;
 
+		mutex_lock(&efx->rss_lock);
 		if (info->flow_type & FLOW_RSS && info->rss_context) {
-			ctx = efx_find_rss_context_entry(info->rss_context,
-							 &efx->rss_context.list);
-			if (!ctx)
-				return -ENOENT;
+			ctx = efx_find_rss_context_entry(efx, info->rss_context);
+			if (!ctx) {
+				rc = -ENOENT;
+				goto out_unlock;
+			}
 		}
 		info->data = 0;
 		if (!efx_rss_active(ctx)) /* No RSS */
-			return 0;
+			goto out_unlock;
 		switch (info->flow_type & ~FLOW_RSS) {
 		case UDP_V4_FLOW:
 			if (ctx->rx_hash_udp_4tuple)
@@ -1024,7 +1026,9 @@ efx_ethtool_get_rxnfc(struct net_device *net_dev,
 		default:
 			break;
 		}
-		return 0;
+out_unlock:
+		mutex_unlock(&efx->rss_lock);
+		return rc;
 	}
 
 	case ETHTOOL_GRXCLSRLCNT:
@@ -1366,16 +1370,20 @@ static int efx_ethtool_get_rxfh_context(struct net_device *net_dev, u32 *indir,
 {
 	struct efx_nic *efx = netdev_priv(net_dev);
 	struct efx_rss_context *ctx;
-	int rc;
+	int rc = 0;
 
 	if (!efx->type->rx_pull_rss_context_config)
 		return -EOPNOTSUPP;
-	ctx = efx_find_rss_context_entry(rss_context, &efx->rss_context.list);
-	if (!ctx)
-		return -ENOENT;
+
+	mutex_lock(&efx->rss_lock);
+	ctx = efx_find_rss_context_entry(efx, rss_context);
+	if (!ctx) {
+		rc = -ENOENT;
+		goto out_unlock;
+	}
 	rc = efx->type->rx_pull_rss_context_config(efx, ctx);
 	if (rc)
-		return rc;
+		goto out_unlock;
 
 	if (hfunc)
 		*hfunc = ETH_RSS_HASH_TOP;
@@ -1383,7 +1391,9 @@ static int efx_ethtool_get_rxfh_context(struct net_device *net_dev, u32 *indir,
 		memcpy(indir, ctx->rx_indir_table, sizeof(ctx->rx_indir_table));
 	if (key)
 		memcpy(key, ctx->rx_hash_key, efx->type->rx_hash_key_size);
-	return 0;
+out_unlock:
+	mutex_unlock(&efx->rss_lock);
+	return rc;
 }
 
 static int efx_ethtool_set_rxfh_context(struct net_device *net_dev,
@@ -1401,23 +1411,31 @@ static int efx_ethtool_set_rxfh_context(struct net_device *net_dev,
 	/* Hash function is Toeplitz, cannot be changed */
 	if (hfunc != ETH_RSS_HASH_NO_CHANGE && hfunc != ETH_RSS_HASH_TOP)
 		return -EOPNOTSUPP;
+
+	mutex_lock(&efx->rss_lock);
+
 	if (*rss_context == ETH_RXFH_CONTEXT_ALLOC) {
-		if (delete)
+		if (delete) {
 			/* alloc + delete == Nothing to do */
-			return -EINVAL;
-		ctx = efx_alloc_rss_context_entry(&efx->rss_context.list);
-		if (!ctx)
-			return -ENOMEM;
+			rc = -EINVAL;
+			goto out_unlock;
+		}
+		ctx = efx_alloc_rss_context_entry(efx);
+		if (!ctx) {
+			rc = -ENOMEM;
+			goto out_unlock;
+		}
 		ctx->context_id = EFX_EF10_RSS_CONTEXT_INVALID;
 		/* Initialise indir table and key to defaults */
 		efx_set_default_rx_indir_table(efx, ctx);
 		netdev_rss_key_fill(ctx->rx_hash_key, sizeof(ctx->rx_hash_key));
 		allocated = true;
 	} else {
-		ctx = efx_find_rss_context_entry(*rss_context,
-						 &efx->rss_context.list);
-		if (!ctx)
-			return -ENOENT;
+		ctx = efx_find_rss_context_entry(efx, *rss_context);
+		if (!ctx) {
+			rc = -ENOENT;
+			goto out_unlock;
+		}
 	}
 
 	if (delete) {
@@ -1425,7 +1443,7 @@ static int efx_ethtool_set_rxfh_context(struct net_device *net_dev,
 		rc = efx->type->rx_push_rss_context_config(efx, ctx, NULL, NULL);
 		if (!rc)
 			efx_free_rss_context_entry(ctx);
-		return rc;
+		goto out_unlock;
 	}
 
 	if (!key)
@@ -1438,6 +1456,8 @@ static int efx_ethtool_set_rxfh_context(struct net_device *net_dev,
 		efx_free_rss_context_entry(ctx);
 	else
 		*rss_context = ctx->user_id;
+out_unlock:
+	mutex_unlock(&efx->rss_lock);
 	return rc;
 }
 
diff --git a/drivers/net/ethernet/sfc/net_driver.h b/drivers/net/ethernet/sfc/net_driver.h
index ef6fc931afb8..15edf98114dd 100644
--- a/drivers/net/ethernet/sfc/net_driver.h
+++ b/drivers/net/ethernet/sfc/net_driver.h
@@ -791,6 +791,7 @@ struct efx_rss_context {
  * @rx_scatter: Scatter mode enabled for receives
  * @rss_context: Main RSS context.  Its @list member is the head of the list of
  *	RSS contexts created by user requests
+ * @rss_lock: Protects custom RSS context software state in @rss_context.list
  * @int_error_count: Number of internal errors seen recently
  * @int_error_expire: Time at which error count will be expired
  * @irq_soft_enabled: Are IRQs soft-enabled? If not, IRQ handler will
@@ -933,6 +934,7 @@ struct efx_nic {
 	int rx_packet_ts_offset;
 	bool rx_scatter;
 	struct efx_rss_context rss_context;
+	struct mutex rss_lock;
 
 	unsigned int_error_count;
 	unsigned long int_error_expire;
diff --git a/drivers/net/ethernet/sfc/nic.h b/drivers/net/ethernet/sfc/nic.h
index ac59afad6e5c..5640034bda10 100644
--- a/drivers/net/ethernet/sfc/nic.h
+++ b/drivers/net/ethernet/sfc/nic.h
@@ -365,6 +365,8 @@ enum {
  * @vi_base: Absolute index of first VI in this function
  * @n_allocated_vis: Number of VIs allocated to this function
  * @must_realloc_vis: Flag: VIs have yet to be reallocated after MC reboot
+ * @must_restore_rss_contexts: Flag: RSS contexts have yet to be restored after
+ *	MC reboot
  * @must_restore_filters: Flag: filters have yet to be restored after MC reboot
  * @n_piobufs: Number of PIO buffers allocated to this function
  * @wc_membase: Base address of write-combining mapping of the memory BAR
@@ -407,6 +409,7 @@ struct efx_ef10_nic_data {
 	unsigned int vi_base;
 	unsigned int n_allocated_vis;
 	bool must_realloc_vis;
+	bool must_restore_rss_contexts;
 	bool must_restore_filters;
 	unsigned int n_piobufs;
 	void __iomem *wc_membase, *pio_write_base;

^ permalink raw reply related	[flat|nested] 8+ messages in thread

* [PATCH net-next 6/6] sfc: fix flow type handling for RSS filters
  2018-03-27 16:40 [PATCH net-next 0/6] sfc: rework locking around filter management Edward Cree
                   ` (4 preceding siblings ...)
  2018-03-27 16:44 ` [PATCH net-next 5/6] sfc: protect list of RSS contexts under a mutex Edward Cree
@ 2018-03-27 16:44 ` Edward Cree
  2018-03-27 17:34 ` [PATCH net-next 0/6] sfc: rework locking around filter management David Miller
  6 siblings, 0 replies; 8+ messages in thread
From: Edward Cree @ 2018-03-27 16:44 UTC (permalink / raw)
  To: linux-net-drivers, David Miller; +Cc: netdev

The FLOW_RSS flag was causing us to insert UDP filters when TCP was wanted.

Fixes: 42356d9a137b ("sfc: support RSS spreading of ethtool ntuple filters")
Signed-off-by: Edward Cree <ecree@solarflare.com>
---
 drivers/net/ethernet/sfc/ethtool.c | 11 ++++++-----
 1 file changed, 6 insertions(+), 5 deletions(-)

diff --git a/drivers/net/ethernet/sfc/ethtool.c b/drivers/net/ethernet/sfc/ethtool.c
index de1fac6cd34f..d3ee9df6f4f0 100644
--- a/drivers/net/ethernet/sfc/ethtool.c
+++ b/drivers/net/ethernet/sfc/ethtool.c
@@ -1088,6 +1088,7 @@ static int efx_ethtool_set_class_rule(struct efx_nic *efx,
 	struct ethtool_tcpip6_spec *ip6_mask = &rule->m_u.tcp_ip6_spec;
 	struct ethtool_usrip6_spec *uip6_entry = &rule->h_u.usr_ip6_spec;
 	struct ethtool_usrip6_spec *uip6_mask = &rule->m_u.usr_ip6_spec;
+	u32 flow_type = rule->flow_type & ~(FLOW_EXT | FLOW_RSS);
 	struct ethhdr *mac_entry = &rule->h_u.ether_spec;
 	struct ethhdr *mac_mask = &rule->m_u.ether_spec;
 	enum efx_filter_flags flags = 0;
@@ -1121,14 +1122,14 @@ static int efx_ethtool_set_class_rule(struct efx_nic *efx,
 	if (rule->flow_type & FLOW_RSS)
 		spec.rss_context = rss_context;
 
-	switch (rule->flow_type & ~(FLOW_EXT | FLOW_RSS)) {
+	switch (flow_type) {
 	case TCP_V4_FLOW:
 	case UDP_V4_FLOW:
 		spec.match_flags = (EFX_FILTER_MATCH_ETHER_TYPE |
 				    EFX_FILTER_MATCH_IP_PROTO);
 		spec.ether_type = htons(ETH_P_IP);
-		spec.ip_proto = ((rule->flow_type & ~FLOW_EXT) == TCP_V4_FLOW ?
-				 IPPROTO_TCP : IPPROTO_UDP);
+		spec.ip_proto = flow_type == TCP_V4_FLOW ? IPPROTO_TCP
+							 : IPPROTO_UDP;
 		if (ip_mask->ip4dst) {
 			if (ip_mask->ip4dst != IP4_ADDR_FULL_MASK)
 				return -EINVAL;
@@ -1162,8 +1163,8 @@ static int efx_ethtool_set_class_rule(struct efx_nic *efx,
 		spec.match_flags = (EFX_FILTER_MATCH_ETHER_TYPE |
 				    EFX_FILTER_MATCH_IP_PROTO);
 		spec.ether_type = htons(ETH_P_IPV6);
-		spec.ip_proto = ((rule->flow_type & ~FLOW_EXT) == TCP_V6_FLOW ?
-				 IPPROTO_TCP : IPPROTO_UDP);
+		spec.ip_proto = flow_type == TCP_V6_FLOW ? IPPROTO_TCP
+							 : IPPROTO_UDP;
 		if (!ip6_mask_is_empty(ip6_mask->ip6dst)) {
 			if (!ip6_mask_is_full(ip6_mask->ip6dst))
 				return -EINVAL;

^ permalink raw reply related	[flat|nested] 8+ messages in thread

* Re: [PATCH net-next 0/6] sfc: rework locking around filter management
  2018-03-27 16:40 [PATCH net-next 0/6] sfc: rework locking around filter management Edward Cree
                   ` (5 preceding siblings ...)
  2018-03-27 16:44 ` [PATCH net-next 6/6] sfc: fix flow type handling for RSS filters Edward Cree
@ 2018-03-27 17:34 ` David Miller
  6 siblings, 0 replies; 8+ messages in thread
From: David Miller @ 2018-03-27 17:34 UTC (permalink / raw)
  To: ecree; +Cc: linux-net-drivers, netdev

From: Edward Cree <ecree@solarflare.com>
Date: Tue, 27 Mar 2018 17:40:11 +0100

> The use of a spinlock to protect filter state combined with the need for a
>  sleeping operation (MCDI) to apply that state to the NIC (on EF10) led to
>  unfixable race conditions, around the handling of filter restoration after
>  an MC reboot.
> So, this patch series removes the requirement to be able to modify the SW
>  filter table from atomic context, by using a workqueue to request
>  asynchronous filter operations (which are needed for ARFS).  Then, the
>  filter table locks are changed to mutexes, replacing the dance of spinlocks
>  and 'busy' flags.  Also, a mutex is added to protect the RSS context state,
>  since otherwise a similar race is possible around restoring that after an
>  MC reboot.  While we're at it, fix a couple of other related bugs.

Series applied, thank you.

^ permalink raw reply	[flat|nested] 8+ messages in thread

end of thread, other threads:[~2018-03-27 17:34 UTC | newest]

Thread overview: 8+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2018-03-27 16:40 [PATCH net-next 0/6] sfc: rework locking around filter management Edward Cree
2018-03-27 16:41 ` [PATCH net-next 1/6] sfc: replace asynchronous filter operations Edward Cree
2018-03-27 16:42 ` [PATCH net-next 2/6] sfc: give ef10 its own rwsem in the filter table instead of filter_lock Edward Cree
2018-03-27 16:42 ` [PATCH net-next 3/6] sfc: use a semaphore to lock farch filters too Edward Cree
2018-03-27 16:44 ` [PATCH net-next 4/6] sfc: return a better error if filter insertion collides with MC reboot Edward Cree
2018-03-27 16:44 ` [PATCH net-next 5/6] sfc: protect list of RSS contexts under a mutex Edward Cree
2018-03-27 16:44 ` [PATCH net-next 6/6] sfc: fix flow type handling for RSS filters Edward Cree
2018-03-27 17:34 ` [PATCH net-next 0/6] sfc: rework locking around filter management David Miller

This is an external index of several public inboxes,
see mirroring instructions on how to clone and mirror
all data and code used by this external index.