When an application calls write() on a socket, it triggers a deep journey inside the Linux kernel. What looks like a simple function call actually flows through the VFS layer, socket abstractions, security checks, protocol handlers, and packet-construction logic before anything ever reaches the network interface. In this first part of the series, we break down the write path piece by piece.
When an application calls write(fd, buf, len), the CPU is still running in user mode. To enter the kernel, the C library (glibc) issues a syscall instruction (on x86-64). This triggers a controlled transition:
syscall and switches from user mode → kernel modewrite() dispatches us to sys_write()At this point, the kernel runs with full privileges. But it doesn't blindly trust user-supplied values — the buffer pointer and file descriptor will be validated before any real work begins.
User Space
--------------------------------------------
write(fd, buf, len)
↓ triggers syscall instruction
--------------------------------------------
Kernel Space (privileged)
sys_write()
Inside sys_write(), the kernel starts by figuring out what kind of object the file descriptor refers to. If it's a socket, the call gets routed into the networking stack. That's where things get interesting.
A file descriptor is just an integer. The kernel's job is to turn it into a usable kernel object. sys_write() does some quick sanity checks and then hands off to its workhorse, ksys_write():
ssize_t sys_write(unsigned int fd, const char __user *buf, size_t count)
{
/* minimal validation */
return ksys_write(fd, buf, count);
}
ksys_write() uses fdget_pos() to retrieve the struct file corresponding to the file descriptor:
ssize_t ksys_write(unsigned int fd, const char __user *buf, size_t count)
{
struct fd f = fdget_pos(fd); // FD lookup
if (!f.file)
return -EBADF; // Bad file descriptor
// Also verifies f.file->f_mode & FMODE_WRITE
// Returns -EBADF if file wasn't opened for writing
}
This is the moment the kernel learns what it's dealing with — a regular file, a pipe, a device, or in our case, a socket.
struct file?Here's a simplified view of the structure:
struct file {
const struct file_operations *f_op; // Operations table (read/write/ioctl, etc.)
void *private_data; // Subsystem-specific data (filesystem, drivers, socket pointer, etc.)
struct inode *f_inode; // Underlying inode
loff_t f_pos; // Current file position
unsigned int f_flags; // Open flags (O_WRONLY, O_NONBLOCK, etc.)
fmode_t f_mode; // Read/write mode
atomic_long_t f_count; // Reference count
// ... more fields
};
The key field here is f_op — it points to a table of function pointers that tell the kernel how to read, write, or otherwise interact with this object. For sockets, this table is socket_file_ops.
With struct file in hand, ksys_write() calls:
ret = vfs_write(file, buf, count, &pos);
The Virtual File System (VFS) acts as a dispatcher. It doesn't know or care whether you're writing to a disk file, a terminal, or a network socket. It simply calls the appropriate handler via the operations table. For sockets, this is where we leave the generic file world and enter the networking subsystem.
Once vfs_write() has the struct file, it performs a dynamic dispatch through the file's operations table:
ret = file->f_op->write_iter(file, &iter);
For sockets, the operations table looks like this:
file->f_op = &socket_file_ops;
socket_file_ops.write_iter = sock_write_iter;
So write_iter() resolves to sock_write_iter() — our entry point into socket land.
sock_write_iter() — The VFS-to-Socket BridgeThis function is a thin wrapper. It extracts the socket from the file and forwards the data:
ssize_t sock_write_iter(struct file *file, struct iov_iter *from)
{
struct socket *sock = file->private_data; // Get the socket
return sock_sendmsg(sock, from, 0); // Hand off to socket layer
}
Notice how private_data in struct file now becomes meaningful — it holds a pointer to struct socket. This is the glue between the VFS abstraction and the actual networking object.
We've now entered the socket layer. This is an intermediate abstraction that sits between the VFS and the actual transport protocols like TCP or UDP.
struct socket?struct socket {
socket_state state; // SS_CONNECTED, SS_UNCONNECTED, etc.
short type; // SOCK_STREAM, SOCK_DGRAM
unsigned long flags;
struct file *file; // Back-pointer to VFS file
struct sock *sk; // The real networking socket (more on this below)
const struct proto_ops *ops; // Socket-layer operations
};
Two fields matter most here:
ops: A table of socket-level operations (connect, accept, sendmsg, etc.)sk: A pointer to struct sock, which holds the actual protocol stateproto_ops?proto_ops defines the socket layer interface — the operations you can perform on a socket from user space:
struct proto_ops {
int (*sendmsg)(struct socket *, struct msghdr *, size_t);
int (*recvmsg)(struct socket *, struct msghdr *, size_t, int);
int (*connect)(struct socket *, struct sockaddr *, int, int);
int (*accept)(struct socket *, struct socket *, int, bool);
int (*release)(struct socket *);
// ... more operations
};
Think of proto_ops as the BSD socket API(Berkeley sockets) translated into kernel function pointers. When you call connect() from user space, it eventually lands on sock->ops->connect().
For TCP sockets, ops points to inet_stream_ops. For UDP, it's inet_dgram_ops.
sock_sendmsg() is protocol-agnostic. It handles the common concerns before handing off to the specific transport:
current->task_structret = sock->ops->sendmsg(sock, msg, flags);
For a TCP socket, sock->ops points to inet_stream_ops, so this call resolves to inet_sendmsg(), which eventually calls into TCP-specific code.
We've passed through VFS and the socket layer. Now we're entering the transport layer — this is where TCP, UDP, and other protocols actually live.
struct sock?While struct socket is a high-level BSD socket abstraction, struct sock is the core networking object. It holds protocol state, buffers, memory accounting, and everything the kernel needs to manage a network connection:
struct sock {
/* Protocol identity */
int sk_family; // AF_INET, AF_INET6
int sk_type; // SOCK_STREAM, SOCK_DGRAM
int sk_protocol; // IPPROTO_TCP, IPPROTO_UDP
/* Synchronization */
spinlock_t sk_lock;
refcount_t sk_refcnt;
/* Packet queues */
struct sk_buff_head sk_receive_queue; // Incoming data
struct sk_buff_head sk_write_queue; // Outgoing data
struct sk_buff_head sk_error_queue;
/* Memory management */
atomic_t sk_wmem_alloc; // Write memory in use
atomic_t sk_rmem_alloc; // Read memory in use
int sk_sndbuf; // Send buffer limit
int sk_rcvbuf; // Receive buffer limit
/* Waiting */
wait_queue_head_t sk_sleep; // For poll/select/epoll
/* Protocol hooks */
struct proto *sk_prot; // Transport-layer operations
/* Back-pointer */
struct socket *sk_socket;
};
The most important field for our journey is sk_prot — this points to struct proto, the transport-layer operations table.
struct proto?If proto_ops is the socket-layer interface, then struct proto is the transport-layer interface. It's where TCP and UDP actually differ:
struct proto {
const char *name; // "TCP", "UDP", etc.
/* Lifecycle */
int (*init)(struct sock *sk);
void (*destroy)(struct sock *sk);
/* Data transfer */
int (*sendmsg)(struct sock *sk, struct msghdr *msg, size_t size);
int (*recvmsg)(struct sock *sk, struct msghdr *msg, size_t size, int flags);
/* Memory pressure */
int (*memory_allocated)(struct sock *sk);
void (*enter_memory_pressure)(struct sock *sk);
/* Port binding */
int (*get_port)(struct sock *sk, unsigned short snum);
/* Object size */
unsigned int obj_size; // sizeof(struct tcp_sock), udp_sock, etc.
};
For TCP, sk_prot points to tcp_prot, so sk->sk_prot->sendmsg eventually resolves to tcp_sendmsg(). This is where the kernel starts building TCP segments, managing sequence numbers, handling congestion control, and doing all the things that make TCP... TCP.
Let's trace the complete path from user space to the transport layer:
write(fd, buf, len) [User Space]
↓ syscall
sys_write() → ksys_write() [Kernel: Syscall Handler]
↓ fd lookup → struct file
vfs_write() [VFS Layer]
↓ file->f_op->write_iter()
sock_write_iter() [Socket VFS Wrapper]
↓ file->private_data → struct socket
sock_sendmsg() [Socket Layer]
↓ security checks
↓ sock->ops->sendmsg() (proto_ops)
inet_sendmsg() [AF_INET Handler]
↓ sock->sk → struct sock
↓ sk->sk_prot->sendmsg() (struct proto)
tcp_sendmsg() [Transport Layer: TCP]
Each layer adds its own abstraction:
proto_ops)struct proto)Now that we have a clear understanding of how the data reaches the transport layer, let's dive into tcp_sendmsg() and explore how the kernel actually builds packets and hands them off to the IP layer.
But first — tcp_sendmsg() doesn't just copy bytes around. It creates packet structures that will travel through every layer of the stack, accumulating headers as they go. To understand what TCP does with your data, we need to meet the data structure that carries every packet through the kernel.
Every packet flowing through the Linux network stack — whether it's being sent or received — lives inside an sk_buff (socket buffer). This structure is carefully designed for performance: headers can be added or removed without copying data, and the same buffer can travel through multiple protocol layers.
Here's a simplified view:
struct sk_buff {
/* Buffer boundaries */
unsigned char *head; // Start of allocated buffer
unsigned char *data; // Start of actual packet data
unsigned char *tail; // End of actual packet data
unsigned char *end; // End of allocated buffer
/* Length tracking */
unsigned int len; // Length of data (tail - data)
unsigned int data_len; // Length of paged data (for scatter-gather)
/* Protocol headers (union of pointers) */
struct tcphdr *th; // TCP header
struct udphdr *uh; // UDP header
struct iphdr *iph; // IPv4 header
struct ipv6hdr *ipv6h; // IPv6 header
/* Ownership and queuing */
struct sock *sk; // Socket this buffer belongs to
struct sk_buff *next; // Next buffer in queue
struct sk_buff *prev; // Previous buffer in queue
/* Metadata */
ktime_t tstamp; // Timestamp
__u32 priority;
// ... many more fields
};
The relationship between head, data, tail, and end is crucial:
|<---------- allocated buffer ---------->|
head end
| |
v v
+------+============+------------------+
| | payload | |
+------+============+------------------+
^ ^
| |
data tail
|<---->| = headroom (space for headers to be added)
|<=========>| = actual data (len = tail - data)
|<--------------->| = tailroom (space to append)
When a packet travels down the stack, each layer uses skb_push() to prepend its header. But here's the key insight: no bytes are moved. Only the data pointer changes:
// skb_push() is essentially just pointer arithmetic:
static inline void *skb_push(struct sk_buff *skb, unsigned int len)
{
skb->data -= len; // Move pointer backward
skb->len += len; // Update length
return skb->data;
}
This is an O(1) operation — just a pointer subtraction. The headroom was pre-allocated precisely so headers could be written there without touching the payload bytes.
Similarly, skb_pull() on the receive path just moves data forward:
static inline void *skb_pull(struct sk_buff *skb, unsigned int len)
{
skb->data += len; // Move pointer forward (skip header)
skb->len -= len;
return skb->data;
}
So when TCP adds a 20-byte header, it calls skb_push(skb, 20), writes the header into those 20 bytes, and the payload remains untouched. Extremely efficient.
Lets look at an example
skb = alloc_skb(2048, GFP_KERNEL);
skb_reserve(skb, 128); // reserve headroom
Address →
0x1000 --------------------------------------------------- 0x1800
|<---------------- 2048 bytes ----------------->|
After skb_reserve(128):
head = 0x1000
data = 0x1080
tail = 0x1080
end = 0x1800
memcpy(skp->tail, user_buf, 100);
skb_put(skb, 100)
skb->tail += 100;
skb->len += 100;
Layout post copy operation
head = 0x1000
data = 0x1080 <-- payload start
tail = 0x10E4 <-- payload end (0x1080 + 100)
end = 0x1800
skb_push(skb, 20)
skb->data -= 20; // data: 0x1080 → 0x106C
skb->len += 20; // len: 100 → 120
After inserting the header
head = 0x1000
data = 0x106C <-- TCP header start
tail = 0x10E4
[ headroom ][ TCP hdr ][ payload ][ free space ]
For zero-copy scenarios(when transferring large data), the kernel needs to track which pages contain the data. This information lives in skb_shared_info, a structure located at the end of the sk_buff's data buffer (at skb->end):
struct skb_shared_info {
__u8 nr_frags; // Number of page fragments
skb_frag_t frags[MAX_SKB_FRAGS]; // Array of fragment descriptors
struct sk_buff *frag_list; // For chained sk_buffs
// ... other fields (TSO info, etc.)
};
struct skb_frag_t {
struct page *page; // Pointer to the memory page
__u32 page_offset; // Offset within the page
__u32 size; // Length of this fragment
};
So when using MSG_ZEROCOPY or a sendfile() packet might look like:
sk_buff linear area: [TCP hdr][IP hdr] (headers only)
|
v
skb_shared_info at skb->end:
nr_frags = 3
frags[0] = { page=0xffff8801, offset=0, size=1460 }
frags[1] = { page=0xffff8801, offset=1460, size=1460 }
frags[2] = { page=0xffff8802, offset=0, size=580 }
A macro is defined for every skb to note the skb_shared_info
#define skb_shinfo(skb) ((struct skb_shared_info *)(skb->end))
The NIC's scatter-gather DMA reads the linear headers first, then each fragment in order — assembling the complete frame on the wire without ever copying the file data.
tcp_sendmsg() is where user data gets transformed into TCP segments. It's surprisingly complex because it has to juggle connection state, memory limits, segmentation, and transmission decisions all at once.
Before doing anything, TCP needs to confirm the connection is usable:
if (sk->sk_state != TCP_ESTABLISHED) {
// For blocking sockets: wait for connection to complete
// For non-blocking: return -EAGAIN if connect() is in progress
}
TCP maintains a lot of internal state — sequence numbers, congestion windows, retransmit queues — and multiple code paths can touch this state simultaneously. Before modifying anything, tcp_sendmsg() grabs the socket lock:
lock_sock(sk);
Why is this necessary? Consider two scenarios where things go wrong without locking:
Race between two send() calls:
If two threads call send() on the same socket at the same time, they might both read the current sequence number before either updates it:
Thread A Thread B
──────── ────────
read write_seq = 1000 read write_seq = 1000 ← both see same value
create segment seq=1000 create segment seq=1000 ← duplicate segment!
write_seq = 1460 write_seq = 1460
The result? Two TCP segments with the same sequence number. The receiver won't know what to do with them, and the connection is effectively corrupted.
Race between send() and ACK processing:
While your application is sending data, the kernel might simultaneously be processing an incoming ACK in softirq(software interrupt) context:
Process context (tcp_sendmsg) Softirq context (tcp_ack)
───────────────────────────── ─────────────────────────
reading congestion window updating congestion window
adding skb to write_queue removing acked skb from queue
Both are touching the same queues and counters. Without synchronization, you get torn reads, corrupted linked lists, and eventually a kernel panic.
The socket lock (lock_sock/release_sock) is a sleeping lock — necessary because tcp_sendmsg() might need to wait for memory or copy data from user space, both of which can sleep. If a softirq tries to acquire the lock while a process holds it, the softirq work gets deferred to a backlog and processed later when release_sock() is called.
The kernel enforces per-socket memory limits via sk->sk_sndbuf. If the send buffer is full:
sk->sk_sleep, waiting for ACKs to free up space-EAGAIN immediatelyif (sk_stream_wspace(sk) <= 0) {
// Wait for space or return EAGAIN
err = sk_stream_wait_memory(sk, &timeo);
}
This is also where send buffer autotuning can kick in — the kernel may dynamically grow sk_sndbuf if memory is available.
User data rarely fits neatly into a single TCP segment. The kernel must respect the Maximum Segment Size (MSS):
MSS = MTU - IP header (20 bytes) - TCP header (20 bytes)
For a typical 1500-byte MTU:
MSS = 1500 - 20 - 20 = 1460 bytes
With TCP options (timestamps, SACK, etc.), the effective MSS is often smaller — around 1400-1420 bytes.
tcp_sendmsg() copies user data into sk_buffs in MSS-sized chunks:
// Simplified version of what happens inside:
int skb_add_data_nocache(struct sock *sk, struct sk_buff *skb,
struct iov_iter *from, int copy)
{
// Copy from userspace into skb's data area
if (skb_tailroom(skb) >= copy) {
// Fits in linear buffer
// Notice the "nocache" here. The kernel does a non-temporal write here i.e
// the write bypasses the cache and is directly written to the main memory.
// The reason why the kernel does this is the data is going straight to NIC. Putting it
// CPU cache would just evict other useful data, since the CPU won't read this payload again.
// Non-temporal store (MOVNTI/MOVNTDQ):
// CPU writes → Write-combining buffer → Directly to RAM
// [CPU] → [WC Buffer] → [RAM]
// ↑
// Bypasses cache entirely
// Note: Non-temporal stores have weaker ordering guarantees.
// The kernel uses sfence (store fence) when needed to ensure all NT stores complete before continuing.
copy_from_iter_nocache(skb->tail, copy, from);
skb->tail += copy;
skb->len += copy;
} else {
// Overflow to page fragments
// Allocates pages, adds to skb_shared_info->frags[]
}
// Update socket memory accounting
sk->sk_wmem_queued += copy;
return 0; // or -EFAULT if copy failed
}
int tcp_sendmsg(struct sock *sk, struct msghdr *msg, size_t size)
{
int copied = 0;
lock_sock(sk);
while (msg_data_left(msg)) {
// Get the last skb in the queue (if any)
skb = tcp_write_queue_tail(sk);
if (!skb || skb->len >= mss) {
// Current segment is full or doesn't exist — allocate a new one
// Allocates small linear buffer (~2KB) for headers + some payload.
// Large payloads overflow into the kernel page fragments (skb_shared_info->frags[]).
skb = sk_stream_alloc_skb(sk, ...);
// Add the new skb to the write queue
tcp_add_write_queue_tail(sk, skb);
}
// Copy data from user space into the skb
copy = min(mss - skb->len, remaining);
err = skb_add_data_nocache(sk, skb, &msg->msg_iter, copy);
if (err)
goto out; // copy_from_iter failed (e.g., -EFAULT for bad pointer)
copied += copy; // Track bytes successfully copied
// Update sequence numbers
tp->write_seq += copy;
// Check if we should mark this segment for push
if (forced_push(tp)) { // more on forced_push later
tcp_mark_push(tp, skb); // ← Sets PSH flag
}
}
tcp_push(sk, flags, mss_now, tp->nonagle, size_goal);
out:
release_sock(sk);
return copied; // ← Returns here, back to vfs_write → sys_write → userspace
}
Each segment gets queued onto sk->sk_write_queue, ready for transmission.
The actual copying happens via copy_from_iter_nocache() or similar functions. This is where the kernel validates that the user buffer is actually readable — a bad pointer here would cause -EFAULT.
For large transfers, the kernel might use page references instead of copying (zero-copy). When this happens:
skb_shared_info->frags[] (as described in section 7)This requires NIC support for scatter-gather.
Finally, tcp_sendmsg() calls tcp_push() to potentially transmit queued segments:
tcp_push(sk, flags, mss, tcp_sk(sk)->nonagle, ...);
This function checks congestion window limits, pacing, and other flow control state before calling tcp_write_xmit() to actually send packets. But transmission isn't guaranteed — if the congestion window is full, segments stay queued until ACKs arrive.
Not every write() triggers immediate transmission. TCP uses several mechanisms to batch small writes into larger, more efficient segments. Understanding these is key to understanding what tcp_push() actually checks.
TCP_NODELAY socket optionHow to use it in practice:
int one = 1, zero = 0;
// Cork the socket — hold all writes
setsockopt(fd, IPPROTO_TCP, TCP_CORK, &one, sizeof(one));
write(fd, http_headers, header_len); // Buffered, not sent
write(fd, http_body, body_len); // Also buffered
// Uncork — now send everything as efficiently packed segments
setsockopt(fd, IPPROTO_TCP, TCP_CORK, &zero, sizeof(zero));
This is commonly used by web servers (nginx, Apache) when sending HTTP responses. Without corking, the headers might go in one small packet and the body in another — wasting bandwidth. With corking, they get packed together optimally.
Safety valve: If you forget to uncork, the kernel automatically flushes after ~200ms. So corked data won't be stuck forever, but you'll get suboptimal latency.
Cork vs Nagle: They solve similar problems but differently:
TCP_NODELAY disables Nagle while Cork still worksWith that context, here's what tcp_push() does:
void tcp_push(struct sock *sk, int flags, int mss_now, int nonagle, int size_goal)
{
struct sk_buff *skb = tcp_write_queue_tail(sk);
if (!skb)
return;
// MSG_MORE acts like cork — batch with subsequent writes
if (flags & MSG_MORE)
nonagle = TCP_NAGLE_CORK;
__tcp_push_pending_frames(sk, mss_now, nonagle);
}
__tcp_push_pending_frames() then checks if Nagle/Cork allows sending:
if (tcp_nagle_test(tp, skb, mss_now, nonagle)) {
tcp_write_xmit(sk, ...); // Actually transmit
}
// else: data stays queued, waiting for ACK or more data
Once tcp_push() decides it's okay to send, the packet begins its journey down through the network stack to the NIC.
tcp_write_xmit() — Send segments within congestion windowThis function iterates over queued segments and transmits what the congestion window allows:
while ((skb = tcp_send_head(sk))) {
// Can we send this segment?
if (tcp_cwnd_test(tp, skb) && tcp_snd_wnd_test(tp, skb, mss)) { // congestion window and receiver window checks
// Build TCP header and send
tcp_transmit_skb(sk, skb, ...);
// On a successful send, update the snd_next
// to reflect we've sent this
tcp_event_new_data_sent(sk, skb);
// └── tp->snd_nxt = TCP_SKB_CB(skb)->end_seq;
} else {
break; // Congestion window full, stop here
}
}
tcp_transmit_skb() — Build the TCP headerThis is where the TCP header actually gets constructed:
// Push TCP header into the skb's headroom
th = (struct tcphdr *)skb_push(skb, tcp_header_size);
// Fill in header fields
th->source = inet->inet_sport;
th->dest = inet->inet_dport;
th->seq = htonl(tp->snd_nxt);
th->ack_seq = htonl(tp->rcv_nxt);
th->window = htons(tcp_select_window(sk));
// ... flags, options, etc.
// Calculate TCP checksum
tcp_v4_send_check(sk, skb);
// Hand to IP layer
err = ip_queue_xmit(sk, skb, ...);
ip_queue_xmit() — IP layer processingThe IP layer adds its header and handles routing:
// Route lookup (cached on socket for performance)
rt = ip_route_output(...);
// Push IP header
iph = (struct iphdr *)skb_push(skb, sizeof(struct iphdr));
iph->version = 4;
iph->ihl = 5;
iph->ttl = 64;
iph->protocol = IPPROTO_TCP;
iph->saddr = rt->src;
iph->daddr = rt->dst;
// ... checksum, etc.
// Pass through netfilter OUTPUT chain (iptables)
// This is where firewall rules get enforeced
// - Block all outgoing traffic to port 25 (SMTP)
// iptables -A OUTPUT -p tcp --dport 25 -j DROP
// - Log all outgoing SSH connections
// iptables -A OUTPUT -p tcp --dport 22 -j LOG
// If a rule says DROP, the packet is discarded here and never reaches the network.
// Then call ip_local_out() → dst_output() -> ip_output()
ip_local_out() — Locally generated packetsip_send_check().LOCAL_OUT (iptables OUTPUT/mangle/NAT) for locally created packets.dst_output(skb): usually ip_output() unless IPsec (xfrm) transforms apply.ip_outputNF_INET_POST_ROUTING (mangle POSTROUTING / nat POSTROUTING - SNAT)ip_output() → ip_finish_output() — Fragmentation checkIf the packet exceeds the path MTU, IP fragmentation happens here. For TCP this is rare since MSS already accounts for MTU.
if (skb->len > mtu) {
return ip_fragment(sk, skb, ...);
}
return ip_finish_output2(sk, skb);
ip_finish_output2() — Neighbor lookup (ARP)Before sending, we need the destination MAC address:
// Find neighbor entry (ARP cache)
neigh = dst_neigh_lookup(dst, &nexthop);
if (neigh) {
// Have MAC address — send directly
neigh_output(neigh, skb);
} else {
// Need ARP resolution — queue packet, send ARP request
neigh_resolve_output(neigh, skb);
}
dev_queue_xmit() — Queueing discipline (QDisc)Packets now enter the traffic control layer. Most systems use the default pfifo_fast qdisc:
// Enqueue to the qdisc
q->enqueue(skb, q);
// Try to dequeue and send
__qdisc_run(q);
→ dequeue skb
→ dev_hard_start_xmit(skb, dev)
The qdisc allows for traffic shaping, prioritization, and rate limiting (if configured).
dev_hard_start_xmit() — Call the NIC driverFinally, we reach the network driver:
// Call driver's transmit function
ops->ndo_start_xmit(skb, dev);
Each driver (e1000, ixgbe, mlx5, etc.) implements ndo_start_xmit. It typically:
At this point, the hardware does the rest:
The CPU is free to do other work. When transmission completes, the NIC raises an interrupt (or the driver polls in NAPI mode), and the skb memory is freed.
write(fd, buf, len) [User Space]
│
▼ syscall
tcp_sendmsg() [TCP Layer]
├── copy to sk_buffs
├── segment by MSS
├── queue to sk_write_queue
└── tcp_push()
│
▼
tcp_write_xmit()
│
▼
tcp_transmit_skb() [Build TCP Header]
│
▼
ip_queue_xmit() [IP Layer]
├── route lookup
├── build IP header
└── netfilter OUTPUT
│
▼
ip_output()
│
▼
ip_finish_output2() [Neighbor/ARP]
│
▼
dev_queue_xmit() [Qdisc]
│
├────────────────────────────────┐
│ │
▼ ▼
dev_hard_start_xmit() [Driver] AF_PACKET socket(if present gets a copy of skbuff)
│ │
▼ ▼
ndo_start_xmit() libpcap
├── DMA mapping │
├── write TX descriptors ▼
└── doorbell [Wireshark - receives a cpy of skbuff]
│ (if running)
▼
[NIC Hardware]
│
▼
[Wire] ~~~~~~→