aboutsummaryrefslogtreecommitdiffstats
path: root/net/core/dev.c
diff options
context:
space:
mode:
Diffstat (limited to '')
-rw-r--r--net/core/dev.c592
1 files changed, 399 insertions, 193 deletions
diff --git a/net/core/dev.c b/net/core/dev.c
index 449b45b843d4..6c5967e80132 100644
--- a/net/core/dev.c
+++ b/net/core/dev.c
@@ -91,6 +91,7 @@
#include <linux/etherdevice.h>
#include <linux/ethtool.h>
#include <linux/skbuff.h>
+#include <linux/kthread.h>
#include <linux/bpf.h>
#include <linux/bpf_trace.h>
#include <net/net_namespace.h>
@@ -101,6 +102,7 @@
#include <net/dsa.h>
#include <net/dst.h>
#include <net/dst_metadata.h>
+#include <net/gro.h>
#include <net/pkt_sched.h>
#include <net/pkt_cls.h>
#include <net/checksum.h>
@@ -1493,6 +1495,27 @@ void netdev_notify_peers(struct net_device *dev)
}
EXPORT_SYMBOL(netdev_notify_peers);
+static int napi_threaded_poll(void *data);
+
+static int napi_kthread_create(struct napi_struct *n)
+{
+ int err = 0;
+
+ /* Create and wake up the kthread once to put it in
+ * TASK_INTERRUPTIBLE mode to avoid the blocked task
+ * warning and work with loadavg.
+ */
+ n->thread = kthread_run(napi_threaded_poll, n, "napi/%s-%d",
+ n->dev->name, n->napi_id);
+ if (IS_ERR(n->thread)) {
+ err = PTR_ERR(n->thread);
+ pr_err("kthread_run failed with err %d\n", err);
+ n->thread = NULL;
+ }
+
+ return err;
+}
+
static int __dev_open(struct net_device *dev, struct netlink_ext_ack *extack)
{
const struct net_device_ops *ops = dev->netdev_ops;
@@ -2194,28 +2217,14 @@ static inline void net_timestamp_set(struct sk_buff *skb)
bool is_skb_forwardable(const struct net_device *dev, const struct sk_buff *skb)
{
- unsigned int len;
-
- if (!(dev->flags & IFF_UP))
- return false;
-
- len = dev->mtu + dev->hard_header_len + VLAN_HLEN;
- if (skb->len <= len)
- return true;
-
- /* if TSO is enabled, we don't care about the length as the packet
- * could be forwarded without being segmented before
- */
- if (skb_is_gso(skb))
- return true;
-
- return false;
+ return __is_skb_forwardable(dev, skb, true);
}
EXPORT_SYMBOL_GPL(is_skb_forwardable);
-int __dev_forward_skb(struct net_device *dev, struct sk_buff *skb)
+static int __dev_forward_skb2(struct net_device *dev, struct sk_buff *skb,
+ bool check_mtu)
{
- int ret = ____dev_forward_skb(dev, skb);
+ int ret = ____dev_forward_skb(dev, skb, check_mtu);
if (likely(!ret)) {
skb->protocol = eth_type_trans(skb, dev);
@@ -2224,6 +2233,11 @@ int __dev_forward_skb(struct net_device *dev, struct sk_buff *skb)
return ret;
}
+
+int __dev_forward_skb(struct net_device *dev, struct sk_buff *skb)
+{
+ return __dev_forward_skb2(dev, skb, true);
+}
EXPORT_SYMBOL_GPL(__dev_forward_skb);
/**
@@ -2250,6 +2264,11 @@ int dev_forward_skb(struct net_device *dev, struct sk_buff *skb)
}
EXPORT_SYMBOL_GPL(dev_forward_skb);
+int dev_forward_skb_nomtu(struct net_device *dev, struct sk_buff *skb)
+{
+ return __dev_forward_skb2(dev, skb, false) ?: netif_rx_internal(skb);
+}
+
static inline int deliver_skb(struct sk_buff *skb,
struct packet_type *pt_prev,
struct net_device *orig_dev)
@@ -3617,11 +3636,22 @@ static struct sk_buff *validate_xmit_vlan(struct sk_buff *skb,
int skb_csum_hwoffload_help(struct sk_buff *skb,
const netdev_features_t features)
{
- if (unlikely(skb->csum_not_inet))
+ if (unlikely(skb_csum_is_sctp(skb)))
return !!(features & NETIF_F_SCTP_CRC) ? 0 :
skb_crc32c_csum_help(skb);
- return !!(features & NETIF_F_CSUM_MASK) ? 0 : skb_checksum_help(skb);
+ if (features & NETIF_F_HW_CSUM)
+ return 0;
+
+ if (features & (NETIF_F_IP_CSUM | NETIF_F_IPV6_CSUM)) {
+ switch (skb->csum_offset) {
+ case offsetof(struct tcphdr, check):
+ case offsetof(struct udphdr, check):
+ return 0;
+ }
+ }
+
+ return skb_checksum_help(skb);
}
EXPORT_SYMBOL(skb_csum_hwoffload_help);
@@ -3878,6 +3908,7 @@ sch_handle_egress(struct sk_buff *skb, int *ret, struct net_device *dev)
/* qdisc_skb_cb(skb)->pkt_len was already set by the caller. */
qdisc_skb_cb(skb)->mru = 0;
+ qdisc_skb_cb(skb)->post_ct = false;
mini_qdisc_bstats_cpu_update(miniq, skb);
switch (tcf_classify(skb, miniq->filter_list, &cl_res, false)) {
@@ -4083,7 +4114,7 @@ static int __dev_queue_xmit(struct sk_buff *skb, struct net_device *sb_dev)
skb_reset_mac_header(skb);
if (unlikely(skb_shinfo(skb)->tx_flags & SKBTX_SCHED_TSTAMP))
- __skb_tstamp_tx(skb, NULL, skb->sk, SCM_TSTAMP_SCHED);
+ __skb_tstamp_tx(skb, NULL, NULL, skb->sk, SCM_TSTAMP_SCHED);
/* Disable soft irqs for various locks below. Also
* stops preemption for RCU.
@@ -4252,6 +4283,22 @@ int gro_normal_batch __read_mostly = 8;
static inline void ____napi_schedule(struct softnet_data *sd,
struct napi_struct *napi)
{
+ struct task_struct *thread;
+
+ if (test_bit(NAPI_STATE_THREADED, &napi->state)) {
+ /* Paired with smp_mb__before_atomic() in
+ * napi_enable()/dev_set_threaded().
+ * Use READ_ONCE() to guarantee a complete
+ * read on napi->thread. Only call
+ * wake_up_process() when it's not NULL.
+ */
+ thread = READ_ONCE(napi->thread);
+ if (thread) {
+ wake_up_process(thread);
+ return;
+ }
+ }
+
list_add_tail(&napi->poll_list, &sd->poll_list);
__raise_softirq_irqoff(NET_RX_SOFTIRQ);
}
@@ -4603,14 +4650,14 @@ static u32 netif_receive_generic_xdp(struct sk_buff *skb,
struct xdp_buff *xdp,
struct bpf_prog *xdp_prog)
{
+ void *orig_data, *orig_data_end, *hard_start;
struct netdev_rx_queue *rxqueue;
- void *orig_data, *orig_data_end;
u32 metalen, act = XDP_DROP;
+ u32 mac_len, frame_sz;
__be16 orig_eth_type;
struct ethhdr *eth;
bool orig_bcast;
- int hlen, off;
- u32 mac_len;
+ int off;
/* Reinjected packets coming from act_mirred or similar should
* not get XDP generic processing.
@@ -4642,15 +4689,16 @@ static u32 netif_receive_generic_xdp(struct sk_buff *skb,
* header.
*/
mac_len = skb->data - skb_mac_header(skb);
- hlen = skb_headlen(skb) + mac_len;
- xdp->data = skb->data - mac_len;
- xdp->data_meta = xdp->data;
- xdp->data_end = xdp->data + hlen;
- xdp->data_hard_start = skb->data - skb_headroom(skb);
+ hard_start = skb->data - skb_headroom(skb);
/* SKB "head" area always have tailroom for skb_shared_info */
- xdp->frame_sz = (void *)skb_end_pointer(skb) - xdp->data_hard_start;
- xdp->frame_sz += SKB_DATA_ALIGN(sizeof(struct skb_shared_info));
+ frame_sz = (void *)skb_end_pointer(skb) - hard_start;
+ frame_sz += SKB_DATA_ALIGN(sizeof(struct skb_shared_info));
+
+ rxqueue = netif_get_rxqueue(skb);
+ xdp_init_buff(xdp, frame_sz, &rxqueue->xdp_rxq);
+ xdp_prepare_buff(xdp, hard_start, skb_headroom(skb) - mac_len,
+ skb_headlen(skb) + mac_len, true);
orig_data_end = xdp->data_end;
orig_data = xdp->data;
@@ -4658,9 +4706,6 @@ static u32 netif_receive_generic_xdp(struct sk_buff *skb,
orig_bcast = is_multicast_ether_addr_64bits(eth->h_dest);
orig_eth_type = eth->h_proto;
- rxqueue = netif_get_rxqueue(skb);
- xdp->rxq = &rxqueue->xdp_rxq;
-
act = bpf_prog_run_xdp(xdp_prog, xdp);
/* check if bpf_xdp_adjust_head was used */
@@ -4895,8 +4940,6 @@ static __latent_entropy void net_tx_action(struct softirq_action *h)
else
__kfree_skb_defer(skb);
}
-
- __kfree_skb_flush();
}
if (sd->output_queue) {
@@ -4962,6 +5005,7 @@ sch_handle_ingress(struct sk_buff *skb, struct packet_type **pt_prev, int *ret,
qdisc_skb_cb(skb)->pkt_len = skb->len;
qdisc_skb_cb(skb)->mru = 0;
+ qdisc_skb_cb(skb)->post_ct = false;
skb->tc_at_ingress = 1;
mini_qdisc_bstats_cpu_update(miniq, skb);
@@ -5151,8 +5195,7 @@ another_round:
skb_reset_mac_len(skb);
}
- if (skb->protocol == cpu_to_be16(ETH_P_8021Q) ||
- skb->protocol == cpu_to_be16(ETH_P_8021AD)) {
+ if (eth_type_vlan(skb->protocol)) {
skb = skb_vlan_untag(skb);
if (unlikely(!skb))
goto out;
@@ -5236,8 +5279,7 @@ check_vlan_id:
* find vlan device.
*/
skb->pkt_type = PACKET_OTHERHOST;
- } else if (skb->protocol == cpu_to_be16(ETH_P_8021Q) ||
- skb->protocol == cpu_to_be16(ETH_P_8021AD)) {
+ } else if (eth_type_vlan(skb->protocol)) {
/* Outer header is 802.1P with vlan 0, inner header is
* 802.1Q or 802.1AD and vlan_do_receive() above could
* not find vlan dev for vlan id 0.
@@ -5713,7 +5755,7 @@ static void flush_all_backlogs(void)
}
/* we can have in flight packet[s] on the cpus we are not flushing,
- * synchronize_net() in rollback_registered_many() will take care of
+ * synchronize_net() in unregister_netdevice_many() will take care of
* them
*/
for_each_cpu(cpu, &flush_cpus)
@@ -5743,8 +5785,6 @@ static void gro_normal_one(struct napi_struct *napi, struct sk_buff *skb, int se
gro_normal_list(napi);
}
-INDIRECT_CALLABLE_DECLARE(int inet_gro_complete(struct sk_buff *, int));
-INDIRECT_CALLABLE_DECLARE(int ipv6_gro_complete(struct sk_buff *, int));
static int napi_gro_complete(struct napi_struct *napi, struct sk_buff *skb)
{
struct packet_offload *ptype;
@@ -5913,10 +5953,6 @@ static void gro_flush_oldest(struct napi_struct *napi, struct list_head *head)
napi_gro_complete(napi, oldest);
}
-INDIRECT_CALLABLE_DECLARE(struct sk_buff *inet_gro_receive(struct list_head *,
- struct sk_buff *));
-INDIRECT_CALLABLE_DECLARE(struct sk_buff *ipv6_gro_receive(struct list_head *,
- struct sk_buff *));
static enum gro_result dev_gro_receive(struct napi_struct *napi, struct sk_buff *skb)
{
u32 hash = skb_get_hash_raw(skb) & (GRO_HASH_BUCKETS - 1);
@@ -6055,13 +6091,6 @@ struct packet_offload *gro_find_complete_by_type(__be16 type)
}
EXPORT_SYMBOL(gro_find_complete_by_type);
-static void napi_skb_free_stolen_head(struct sk_buff *skb)
-{
- skb_dst_drop(skb);
- skb_ext_put(skb);
- kmem_cache_free(skbuff_head_cache, skb);
-}
-
static gro_result_t napi_skb_finish(struct napi_struct *napi,
struct sk_buff *skb,
gro_result_t ret)
@@ -6071,15 +6100,11 @@ static gro_result_t napi_skb_finish(struct napi_struct *napi,
gro_normal_one(napi, skb, 1);
break;
- case GRO_DROP:
- kfree_skb(skb);
- break;
-
case GRO_MERGED_FREE:
if (NAPI_GRO_CB(skb)->free == NAPI_GRO_FREE_STOLEN_HEAD)
napi_skb_free_stolen_head(skb);
else
- __kfree_skb(skb);
+ __kfree_skb_defer(skb);
break;
case GRO_HELD:
@@ -6159,10 +6184,6 @@ static gro_result_t napi_frags_finish(struct napi_struct *napi,
gro_normal_one(napi, skb, 1);
break;
- case GRO_DROP:
- napi_reuse_skb(napi, skb);
- break;
-
case GRO_MERGED_FREE:
if (NAPI_GRO_CB(skb)->free == NAPI_GRO_FREE_STOLEN_HEAD)
napi_skb_free_stolen_head(skb);
@@ -6224,9 +6245,6 @@ gro_result_t napi_gro_frags(struct napi_struct *napi)
gro_result_t ret;
struct sk_buff *skb = napi_frags_skb(napi);
- if (!skb)
- return GRO_DROP;
-
trace_napi_gro_frags_entry(skb);
ret = napi_frags_finish(napi, skb, dev_gro_receive(napi, skb));
@@ -6709,6 +6727,49 @@ static void init_gro_hash(struct napi_struct *napi)
napi->gro_bitmask = 0;
}
+int dev_set_threaded(struct net_device *dev, bool threaded)
+{
+ struct napi_struct *napi;
+ int err = 0;
+
+ if (dev->threaded == threaded)
+ return 0;
+
+ if (threaded) {
+ list_for_each_entry(napi, &dev->napi_list, dev_list) {
+ if (!napi->thread) {
+ err = napi_kthread_create(napi);
+ if (err) {
+ threaded = false;
+ break;
+ }
+ }
+ }
+ }
+
+ dev->threaded = threaded;
+
+ /* Make sure kthread is created before THREADED bit
+ * is set.
+ */
+ smp_mb__before_atomic();
+
+ /* Setting/unsetting threaded mode on a napi might not immediately
+ * take effect, if the current napi instance is actively being
+ * polled. In this case, the switch between threaded mode and
+ * softirq mode will happen in the next round of napi_schedule().
+ * This should not cause hiccups/stalls to the live traffic.
+ */
+ list_for_each_entry(napi, &dev->napi_list, dev_list) {
+ if (threaded)
+ set_bit(NAPI_STATE_THREADED, &napi->state);
+ else
+ clear_bit(NAPI_STATE_THREADED, &napi->state);
+ }
+
+ return err;
+}
+
void netif_napi_add(struct net_device *dev, struct napi_struct *napi,
int (*poll)(struct napi_struct *, int), int weight)
{
@@ -6736,6 +6797,12 @@ void netif_napi_add(struct net_device *dev, struct napi_struct *napi,
set_bit(NAPI_STATE_NPSVC, &napi->state);
list_add_rcu(&napi->dev_list, &dev->napi_list);
napi_hash_add(napi);
+ /* Create kthread for this napi if dev->threaded is set.
+ * Clear dev->threaded if kthread creation failed so that
+ * threaded mode will not be enabled in napi_enable().
+ */
+ if (dev->threaded && napi_kthread_create(napi))
+ dev->threaded = 0;
}
EXPORT_SYMBOL(netif_napi_add);
@@ -6753,9 +6820,28 @@ void napi_disable(struct napi_struct *n)
clear_bit(NAPI_STATE_PREFER_BUSY_POLL, &n->state);
clear_bit(NAPI_STATE_DISABLE, &n->state);
+ clear_bit(NAPI_STATE_THREADED, &n->state);
}
EXPORT_SYMBOL(napi_disable);
+/**
+ * napi_enable - enable NAPI scheduling
+ * @n: NAPI context
+ *
+ * Resume NAPI from being scheduled on this context.
+ * Must be paired with napi_disable.
+ */
+void napi_enable(struct napi_struct *n)
+{
+ BUG_ON(!test_bit(NAPI_STATE_SCHED, &n->state));
+ smp_mb__before_atomic();
+ clear_bit(NAPI_STATE_SCHED, &n->state);
+ clear_bit(NAPI_STATE_NPSVC, &n->state);
+ if (n->dev->threaded && n->thread)
+ set_bit(NAPI_STATE_THREADED, &n->state);
+}
+EXPORT_SYMBOL(napi_enable);
+
static void flush_gro_hash(struct napi_struct *napi)
{
int i;
@@ -6781,18 +6867,18 @@ void __netif_napi_del(struct napi_struct *napi)
flush_gro_hash(napi);
napi->gro_bitmask = 0;
+
+ if (napi->thread) {
+ kthread_stop(napi->thread);
+ napi->thread = NULL;
+ }
}
EXPORT_SYMBOL(__netif_napi_del);
-static int napi_poll(struct napi_struct *n, struct list_head *repoll)
+static int __napi_poll(struct napi_struct *n, bool *repoll)
{
- void *have;
int work, weight;
- list_del_init(&n->poll_list);
-
- have = netpoll_poll_lock(n);
-
weight = n->weight;
/* This NAPI_STATE_SCHED test is for avoiding a race
@@ -6812,7 +6898,7 @@ static int napi_poll(struct napi_struct *n, struct list_head *repoll)
n->poll, work, weight);
if (likely(work < weight))
- goto out_unlock;
+ return work;
/* Drivers must not modify the NAPI state if they
* consume the entire weight. In such cases this code
@@ -6821,7 +6907,7 @@ static int napi_poll(struct napi_struct *n, struct list_head *repoll)
*/
if (unlikely(napi_disable_pending(n))) {
napi_complete(n);
- goto out_unlock;
+ return work;
}
/* The NAPI context has more processing work, but busy-polling
@@ -6834,7 +6920,7 @@ static int napi_poll(struct napi_struct *n, struct list_head *repoll)
*/
napi_schedule(n);
}
- goto out_unlock;
+ return work;
}
if (n->gro_bitmask) {
@@ -6852,17 +6938,78 @@ static int napi_poll(struct napi_struct *n, struct list_head *repoll)
if (unlikely(!list_empty(&n->poll_list))) {
pr_warn_once("%s: Budget exhausted after napi rescheduled\n",
n->dev ? n->dev->name : "backlog");
- goto out_unlock;
+ return work;
}
- list_add_tail(&n->poll_list, repoll);
+ *repoll = true;
+
+ return work;
+}
+
+static int napi_poll(struct napi_struct *n, struct list_head *repoll)
+{
+ bool do_repoll = false;
+ void *have;
+ int work;
+
+ list_del_init(&n->poll_list);
+
+ have = netpoll_poll_lock(n);
+
+ work = __napi_poll(n, &do_repoll);
+
+ if (do_repoll)
+ list_add_tail(&n->poll_list, repoll);
-out_unlock:
netpoll_poll_unlock(have);
return work;
}
+static int napi_thread_wait(struct napi_struct *napi)
+{
+ set_current_state(TASK_INTERRUPTIBLE);
+
+ while (!kthread_should_stop() && !napi_disable_pending(napi)) {
+ if (test_bit(NAPI_STATE_SCHED, &napi->state)) {
+ WARN_ON(!list_empty(&napi->poll_list));
+ __set_current_state(TASK_RUNNING);
+ return 0;
+ }
+
+ schedule();
+ set_current_state(TASK_INTERRUPTIBLE);
+ }
+ __set_current_state(TASK_RUNNING);
+ return -1;
+}
+
+static int napi_threaded_poll(void *data)
+{
+ struct napi_struct *napi = data;
+ void *have;
+
+ while (!napi_thread_wait(napi)) {
+ for (;;) {
+ bool repoll = false;
+
+ local_bh_disable();
+
+ have = netpoll_poll_lock(napi);
+ __napi_poll(napi, &repoll);
+ netpoll_poll_unlock(have);
+
+ local_bh_enable();
+
+ if (!repoll)
+ break;
+
+ cond_resched();
+ }
+ }
+ return 0;
+}
+
static __latent_entropy void net_rx_action(struct softirq_action *h)
{
struct softnet_data *sd = this_cpu_ptr(&softnet_data);
@@ -6881,7 +7028,7 @@ static __latent_entropy void net_rx_action(struct softirq_action *h)
if (list_empty(&list)) {
if (!sd_has_rps_ipi_waiting(sd) && list_empty(&repoll))
- goto out;
+ return;
break;
}
@@ -6908,8 +7055,6 @@ static __latent_entropy void net_rx_action(struct softirq_action *h)
__raise_softirq_irqoff(NET_RX_SOFTIRQ);
net_rps_action_and_irq_enable(sd);
-out:
- __kfree_skb_flush();
}
struct netdev_adjacent {
@@ -8121,6 +8266,39 @@ struct net_device *netdev_get_xmit_slave(struct net_device *dev,
}
EXPORT_SYMBOL(netdev_get_xmit_slave);
+static struct net_device *netdev_sk_get_lower_dev(struct net_device *dev,
+ struct sock *sk)
+{
+ const struct net_device_ops *ops = dev->netdev_ops;
+
+ if (!ops->ndo_sk_get_lower_dev)
+ return NULL;
+ return ops->ndo_sk_get_lower_dev(dev, sk);
+}
+
+/**
+ * netdev_sk_get_lowest_dev - Get the lowest device in chain given device and socket
+ * @dev: device
+ * @sk: the socket
+ *
+ * %NULL is returned if no lower device is found.
+ */
+
+struct net_device *netdev_sk_get_lowest_dev(struct net_device *dev,
+ struct sock *sk)
+{
+ struct net_device *lower;
+
+ lower = netdev_sk_get_lower_dev(dev, sk);
+ while (lower) {
+ dev = lower;
+ lower = netdev_sk_get_lower_dev(dev, sk);
+ }
+
+ return dev;
+}
+EXPORT_SYMBOL(netdev_sk_get_lowest_dev);
+
static void netdev_adjacent_add_links(struct net_device *dev)
{
struct netdev_adjacent *iter;
@@ -8743,6 +8921,48 @@ int dev_set_mac_address(struct net_device *dev, struct sockaddr *sa,
}
EXPORT_SYMBOL(dev_set_mac_address);
+static DECLARE_RWSEM(dev_addr_sem);
+
+int dev_set_mac_address_user(struct net_device *dev, struct sockaddr *sa,
+ struct netlink_ext_ack *extack)
+{
+ int ret;
+
+ down_write(&dev_addr_sem);
+ ret = dev_set_mac_address(dev, sa, extack);
+ up_write(&dev_addr_sem);
+ return ret;
+}
+EXPORT_SYMBOL(dev_set_mac_address_user);
+
+int dev_get_mac_address(struct sockaddr *sa, struct net *net, char *dev_name)
+{
+ size_t size = sizeof(sa->sa_data);
+ struct net_device *dev;
+ int ret = 0;
+
+ down_read(&dev_addr_sem);
+ rcu_read_lock();
+
+ dev = dev_get_by_name_rcu(net, dev_name);
+ if (!dev) {
+ ret = -ENODEV;
+ goto unlock;
+ }
+ if (!dev->addr_len)
+ memset(sa->sa_data, 0, size);
+ else
+ memcpy(sa->sa_data, dev->dev_addr,
+ min_t(size_t, size, dev->addr_len));
+ sa->sa_family = dev->type;
+
+unlock:
+ rcu_read_unlock();
+ up_read(&dev_addr_sem);
+ return ret;
+}
+EXPORT_SYMBOL(dev_get_mac_address);
+
/**
* dev_change_carrier - Change device carrier
* @dev: device
@@ -9442,106 +9662,6 @@ static void net_set_todo(struct net_device *dev)
dev_net(dev)->dev_unreg_count++;
}
-static void rollback_registered_many(struct list_head *head)
-{
- struct net_device *dev, *tmp;
- LIST_HEAD(close_head);
-
- BUG_ON(dev_boot_phase);
- ASSERT_RTNL();
-
- list_for_each_entry_safe(dev, tmp, head, unreg_list) {
- /* Some devices call without registering
- * for initialization unwind. Remove those
- * devices and proceed with the remaining.
- */
- if (dev->reg_state == NETREG_UNINITIALIZED) {
- pr_debug("unregister_netdevice: device %s/%p never was registered\n",
- dev->name, dev);
-
- WARN_ON(1);
- list_del(&dev->unreg_list);
- continue;
- }
- dev->dismantle = true;
- BUG_ON(dev->reg_state != NETREG_REGISTERED);
- }
-
- /* If device is running, close it first. */
- list_for_each_entry(dev, head, unreg_list)
- list_add_tail(&dev->close_list, &close_head);
- dev_close_many(&close_head, true);
-
- list_for_each_entry(dev, head, unreg_list) {
- /* And unlink it from device chain. */
- unlist_netdevice(dev);
-
- dev->reg_state = NETREG_UNREGISTERING;
- }
- flush_all_backlogs();
-
- synchronize_net();
-
- list_for_each_entry(dev, head, unreg_list) {
- struct sk_buff *skb = NULL;
-
- /* Shutdown queueing discipline. */
- dev_shutdown(dev);
-
- dev_xdp_uninstall(dev);
-
- /* Notify protocols, that we are about to destroy
- * this device. They should clean all the things.
- */
- call_netdevice_notifiers(NETDEV_UNREGISTER, dev);
-
- if (!dev->rtnl_link_ops ||
- dev->rtnl_link_state == RTNL_LINK_INITIALIZED)
- skb = rtmsg_ifinfo_build_skb(RTM_DELLINK, dev, ~0U, 0,
- GFP_KERNEL, NULL, 0);
-
- /*
- * Flush the unicast and multicast chains
- */
- dev_uc_flush(dev);
- dev_mc_flush(dev);
-
- netdev_name_node_alt_flush(dev);
- netdev_name_node_free(dev->name_node);
-
- if (dev->netdev_ops->ndo_uninit)
- dev->netdev_ops->ndo_uninit(dev);
-
- if (skb)
- rtmsg_ifinfo_send(skb, dev, GFP_KERNEL);
-
- /* Notifier chain MUST detach us all upper devices. */
- WARN_ON(netdev_has_any_upper_dev(dev));
- WARN_ON(netdev_has_any_lower_dev(dev));
-
- /* Remove entries from kobject tree */
- netdev_unregister_kobject(dev);
-#ifdef CONFIG_XPS
- /* Remove XPS queueing entries */
- netif_reset_xps_queues_gt(dev, 0);
-#endif
- }
-
- synchronize_net();
-
- list_for_each_entry(dev, head, unreg_list)
- dev_put(dev);
-}
-
-static void rollback_registered(struct net_device *dev)
-{
- LIST_HEAD(single);
-
- list_add(&dev->unreg_list, &single);
- rollback_registered_many(&single);
- list_del(&single);
-}
-
static netdev_features_t netdev_sync_upper_features(struct net_device *lower,
struct net_device *upper, netdev_features_t features)
{
@@ -10014,7 +10134,7 @@ int register_netdevice(struct net_device *dev)
dev->hw_features |= (NETIF_F_SOFT_FEATURES | NETIF_F_SOFT_FEATURES_OFF);
dev->features |= NETIF_F_SOFT_FEATURES;
- if (dev->netdev_ops->ndo_udp_tunnel_add) {
+ if (dev->udp_tunnel_nic_info) {
dev->features |= NETIF_F_RX_UDP_TUNNEL_PORT;
dev->hw_features |= NETIF_F_RX_UDP_TUNNEL_PORT;
}
@@ -10091,8 +10211,7 @@ int register_netdevice(struct net_device *dev)
if (ret) {
/* Expect explicit free_netdev() on failure */
dev->needs_free_netdev = false;
- rollback_registered(dev);
- net_set_todo(dev);
+ unregister_netdevice_queue(dev, NULL);
goto out;
}
/*
@@ -10714,9 +10833,10 @@ void unregister_netdevice_queue(struct net_device *dev, struct list_head *head)
if (head) {
list_move_tail(&dev->unreg_list, head);
} else {
- rollback_registered(dev);
- /* Finish processing unregister after unlock */
- net_set_todo(dev);
+ LIST_HEAD(single);
+
+ list_add(&dev->unreg_list, &single);
+ unregister_netdevice_many(&single);
}
}
EXPORT_SYMBOL(unregister_netdevice_queue);
@@ -10730,14 +10850,100 @@ EXPORT_SYMBOL(unregister_netdevice_queue);
*/
void unregister_netdevice_many(struct list_head *head)
{
- struct net_device *dev;
+ struct net_device *dev, *tmp;
+ LIST_HEAD(close_head);
+
+ BUG_ON(dev_boot_phase);
+ ASSERT_RTNL();
+
+ if (list_empty(head))
+ return;
+
+ list_for_each_entry_safe(dev, tmp, head, unreg_list) {
+ /* Some devices call without registering
+ * for initialization unwind. Remove those
+ * devices and proceed with the remaining.
+ */
+ if (dev->reg_state == NETREG_UNINITIALIZED) {
+ pr_debug("unregister_netdevice: device %s/%p never was registered\n",
+ dev->name, dev);
+
+ WARN_ON(1);
+ list_del(&dev->unreg_list);
+ continue;
+ }
+ dev->dismantle = true;
+ BUG_ON(dev->reg_state != NETREG_REGISTERED);
+ }
+
+ /* If device is running, close it first. */
+ list_for_each_entry(dev, head, unreg_list)
+ list_add_tail(&dev->close_list, &close_head);
+ dev_close_many(&close_head, true);
+
+ list_for_each_entry(dev, head, unreg_list) {
+ /* And unlink it from device chain. */
+ unlist_netdevice(dev);
+
+ dev->reg_state = NETREG_UNREGISTERING;
+ }
+ flush_all_backlogs();
+
+ synchronize_net();
+
+ list_for_each_entry(dev, head, unreg_list) {
+ struct sk_buff *skb = NULL;
+
+ /* Shutdown queueing discipline. */
+ dev_shutdown(dev);
+
+ dev_xdp_uninstall(dev);
+
+ /* Notify protocols, that we are about to destroy
+ * this device. They should clean all the things.
+ */
+ call_netdevice_notifiers(NETDEV_UNREGISTER, dev);
+
+ if (!dev->rtnl_link_ops ||
+ dev->rtnl_link_state == RTNL_LINK_INITIALIZED)
+ skb = rtmsg_ifinfo_build_skb(RTM_DELLINK, dev, ~0U, 0,
+ GFP_KERNEL, NULL, 0);
+
+ /*
+ * Flush the unicast and multicast chains
+ */
+ dev_uc_flush(dev);
+ dev_mc_flush(dev);
+
+ netdev_name_node_alt_flush(dev);
+ netdev_name_node_free(dev->name_node);
+
+ if (dev->netdev_ops->ndo_uninit)
+ dev->netdev_ops->ndo_uninit(dev);
+
+ if (skb)
+ rtmsg_ifinfo_send(skb, dev, GFP_KERNEL);
+
+ /* Notifier chain MUST detach us all upper devices. */
+ WARN_ON(netdev_has_any_upper_dev(dev));
+ WARN_ON(netdev_has_any_lower_dev(dev));
- if (!list_empty(head)) {
- rollback_registered_many(head);
- list_for_each_entry(dev, head, unreg_list)
- net_set_todo(dev);
- list_del(head);
+ /* Remove entries from kobject tree */
+ netdev_unregister_kobject(dev);
+#ifdef CONFIG_XPS
+ /* Remove XPS queueing entries */
+ netif_reset_xps_queues_gt(dev, 0);
+#endif
}
+
+ synchronize_net();
+
+ list_for_each_entry(dev, head, unreg_list) {
+ dev_put(dev);
+ net_set_todo(dev);
+ }
+
+ list_del(head);
}
EXPORT_SYMBOL(unregister_netdevice_many);