shadow_rs/host/descriptor/socket/inet/
legacy_tcp.rs

1use std::ffi::CStr;
2use std::net::{Ipv4Addr, SocketAddrV4};
3use std::sync::Arc;
4
5use atomic_refcell::AtomicRefCell;
6use linux_api::errno::Errno;
7use linux_api::ioctls::IoctlRequest;
8use linux_api::socket::Shutdown;
9use nix::sys::socket::{MsgFlags, SockaddrIn};
10use shadow_shim_helper_rs::emulated_time::EmulatedTime;
11use shadow_shim_helper_rs::syscall_types::ForeignPtr;
12
13use crate::core::worker::Worker;
14use crate::cshadow as c;
15use crate::host::descriptor::listener::{StateListenHandle, StateListenerFilter};
16use crate::host::descriptor::socket::inet::{self, InetSocket};
17use crate::host::descriptor::socket::{RecvmsgArgs, RecvmsgReturn, SendmsgArgs, Socket};
18use crate::host::descriptor::{
19    CompatFile, File, FileMode, FileSignals, FileState, FileStatus, OpenFile, SyscallResult,
20};
21use crate::host::host::Host;
22use crate::host::memory_manager::MemoryManager;
23use crate::host::network::interface::FifoPacketPriority;
24use crate::host::network::namespace::NetworkNamespace;
25use crate::host::syscall::io::{IoVec, write_partial};
26use crate::host::syscall::types::{ForeignArrayPtr, SyscallError};
27use crate::host::thread::ThreadId;
28use crate::network::packet::PacketRc;
29use crate::utility::callback_queue::CallbackQueue;
30use crate::utility::sockaddr::SockaddrStorage;
31use crate::utility::{HostTreePointer, ObjectCounter};
32
33pub struct LegacyTcpSocket {
34    socket: HostTreePointer<c::TCP>,
35    // should only be used by `OpenFile` to make sure there is only ever one `OpenFile` instance for
36    // this file
37    has_open_file: bool,
38    /// Did the last connect() call block, and if so what thread?
39    thread_of_blocked_connect: Option<ThreadId>,
40    _counter: ObjectCounter,
41}
42
43impl LegacyTcpSocket {
44    pub fn new(status: FileStatus, host: &Host) -> Arc<AtomicRefCell<Self>> {
45        let recv_buf_size = host.params.init_sock_recv_buf_size.try_into().unwrap();
46        let send_buf_size = host.params.init_sock_send_buf_size.try_into().unwrap();
47
48        let tcp = unsafe { c::tcp_new(host, recv_buf_size, send_buf_size) };
49        let tcp = unsafe { Self::new_from_legacy(tcp) };
50
51        tcp.borrow_mut().set_status(status);
52
53        tcp
54    }
55
56    /// Takes ownership of the [`TCP`](c::TCP) reference.
57    ///
58    /// # Safety
59    ///
60    /// `legacy_tcp` must be safely dereferenceable, and not directly accessed again.
61    pub unsafe fn new_from_legacy(legacy_tcp: *mut c::TCP) -> Arc<AtomicRefCell<Self>> {
62        assert!(!legacy_tcp.is_null());
63
64        let socket = Self {
65            socket: HostTreePointer::new(legacy_tcp),
66            has_open_file: false,
67            thread_of_blocked_connect: None,
68            _counter: ObjectCounter::new("LegacyTcpSocket"),
69        };
70
71        let rv = Arc::new(AtomicRefCell::new(socket));
72
73        let inet_socket = InetSocket::LegacyTcp(rv.clone());
74        let inet_socket = Box::into_raw(Box::new(inet_socket.downgrade()));
75        unsafe { c::tcp_setRustSocket(legacy_tcp, inet_socket) };
76
77        rv
78    }
79
80    /// Get a canonical handle for this socket. We use the address of the `TCP` object so that the
81    /// rust socket and legacy socket have the same handle.
82    pub fn canonical_handle(&self) -> usize {
83        self.as_legacy_tcp() as usize
84    }
85
86    /// Get the [`c::TCP`] pointer.
87    pub fn as_legacy_tcp(&self) -> *mut c::TCP {
88        unsafe { self.socket.ptr() }
89    }
90
91    /// Get the [`c::TCP`] pointer as a [`c::LegacySocket`] pointer.
92    pub fn as_legacy_socket(&self) -> *mut c::LegacySocket {
93        self.as_legacy_tcp() as *mut c::LegacySocket
94    }
95
96    /// Get the [`c::TCP`] pointer as a [`c::LegacyFile`] pointer.
97    pub fn as_legacy_file(&self) -> *mut c::LegacyFile {
98        self.as_legacy_tcp() as *mut c::LegacyFile
99    }
100
101    pub fn status(&self) -> FileStatus {
102        let o_flags = unsafe { c::legacyfile_getFlags(self.as_legacy_file()) };
103        let o_flags =
104            linux_api::fcntl::OFlag::from_bits(o_flags).expect("Not a valid OFlag: {o_flags:?}");
105        let (status, extra_flags) = FileStatus::from_o_flags(o_flags);
106        assert!(
107            extra_flags.is_empty(),
108            "Rust wrapper doesn't support {extra_flags:?} flags",
109        );
110        status
111    }
112
113    pub fn set_status(&mut self, status: FileStatus) {
114        let o_flags = status.as_o_flags().bits();
115        unsafe { c::legacyfile_setFlags(self.as_legacy_file(), o_flags) };
116    }
117
118    pub fn mode(&self) -> FileMode {
119        FileMode::READ | FileMode::WRITE
120    }
121
122    pub fn has_open_file(&self) -> bool {
123        self.has_open_file
124    }
125
126    pub fn supports_sa_restart(&self) -> bool {
127        // TODO: false if a timeout has been set via setsockopt
128        true
129    }
130
131    pub fn set_has_open_file(&mut self, val: bool) {
132        self.has_open_file = val;
133    }
134
135    pub fn push_in_packet(
136        &mut self,
137        packet: PacketRc,
138        _cb_queue: &mut CallbackQueue,
139        _recv_time: EmulatedTime,
140    ) {
141        Worker::with_active_host(|host| {
142            // Here we drop the `PacketRc`, and we transfer our ref to the inner `Packet` to C.
143            unsafe {
144                c::legacysocket_pushInPacket(self.as_legacy_socket(), host, packet.into_raw())
145            };
146        })
147        .unwrap();
148    }
149
150    pub fn pull_out_packet(&mut self, _cb_queue: &mut CallbackQueue) -> Option<PacketRc> {
151        // If we get a `Packet`, a ref to it is transfered to us from the C code.
152        let packet = Worker::with_active_host(|host| unsafe {
153            c::legacysocket_pullOutPacket(self.as_legacy_socket(), host)
154        })
155        .unwrap();
156
157        if packet.is_null() {
158            return None;
159        }
160
161        // We own the ref to the `Packet` now, let the C code borrow it.
162        Worker::with_active_host(|host| unsafe {
163            c::tcp_networkInterfaceIsAboutToSendPacket(self.as_legacy_tcp(), host, packet);
164        })
165        .unwrap();
166
167        // We own the ref to the `Packet`, track it in a `PacketRc` and return it to the caller.
168        Some(PacketRc::from_raw(packet))
169    }
170
171    fn peek_packet(&self) -> Option<PacketRc> {
172        // If we get a `Packet`, a ref to it is transfered to us from the C code.
173        let packet = unsafe { c::legacysocket_peekNextOutPacket(self.as_legacy_socket()) };
174
175        if packet.is_null() {
176            return None;
177        }
178
179        // We own the ref to the `Packet`, track it in a `PacketRc` and return it to the caller.
180        Some(PacketRc::from_raw(packet))
181    }
182
183    pub fn peek_next_packet_priority(&self) -> Option<FifoPacketPriority> {
184        self.peek_packet().map(|p| p.priority())
185    }
186
187    pub fn has_data_to_send(&self) -> bool {
188        self.peek_packet().is_some()
189    }
190
191    pub fn getsockname(&self) -> Result<Option<SockaddrIn>, Errno> {
192        let mut ip: libc::in_addr_t = 0;
193        let mut port: libc::in_port_t = 0;
194
195        // should return ip and port in network byte order
196        let okay =
197            unsafe { c::legacysocket_getSocketName(self.as_legacy_socket(), &mut ip, &mut port) };
198        if okay != 1 {
199            return Ok(Some(SocketAddrV4::new(Ipv4Addr::UNSPECIFIED, 0).into()));
200        }
201
202        let ip = Ipv4Addr::from(u32::from_be(ip));
203        let port = u16::from_be(port);
204        let addr = SocketAddrV4::new(ip, port);
205
206        Ok(Some(addr.into()))
207    }
208
209    pub fn getpeername(&self) -> Result<Option<SockaddrIn>, Errno> {
210        let mut ip: libc::in_addr_t = 0;
211        let mut port: libc::in_port_t = 0;
212
213        // should return ip and port in network byte order
214        let okay =
215            unsafe { c::legacysocket_getPeerName(self.as_legacy_socket(), &mut ip, &mut port) };
216        if okay != 1 {
217            return Err(Errno::ENOTCONN);
218        }
219
220        let ip = Ipv4Addr::from(u32::from_be(ip));
221        let port = u16::from_be(port);
222        let addr = SocketAddrV4::new(ip, port);
223
224        Ok(Some(addr.into()))
225    }
226
227    pub fn address_family(&self) -> linux_api::socket::AddressFamily {
228        linux_api::socket::AddressFamily::AF_INET
229    }
230
231    pub fn close(&mut self, _cb_queue: &mut CallbackQueue) -> Result<(), SyscallError> {
232        Worker::with_active_host(|h| {
233            unsafe { c::legacyfile_close(self.as_legacy_file(), h) };
234        })
235        .unwrap();
236        Ok(())
237    }
238
239    pub fn bind(
240        socket: &Arc<AtomicRefCell<Self>>,
241        addr: Option<&SockaddrStorage>,
242        net_ns: &NetworkNamespace,
243        rng: impl rand::Rng,
244    ) -> Result<(), SyscallError> {
245        // if the address pointer was NULL
246        let Some(addr) = addr else {
247            return Err(Errno::EFAULT.into());
248        };
249
250        // if not an inet socket address
251        let Some(addr) = addr.as_inet() else {
252            return Err(Errno::EINVAL.into());
253        };
254
255        let addr: SocketAddrV4 = (*addr).into();
256
257        // if the socket is already bound
258        {
259            let socket = socket.borrow();
260            let socket = socket.as_legacy_socket();
261            if unsafe { c::legacysocket_isBound(socket) } == 1 {
262                return Err(Errno::EINVAL.into());
263            }
264        }
265
266        // make sure the socket doesn't have a peer
267        {
268            // Since we're not bound, we're not connected and have no peer. We may have a peer in
269            // the future if `connect()` is called on this socket.
270            let socket = socket.borrow();
271            let socket = socket.as_legacy_socket();
272            assert_eq!(0, unsafe {
273                c::legacysocket_getPeerName(socket, std::ptr::null_mut(), std::ptr::null_mut())
274            });
275        }
276
277        // this will allow us to receive packets from any peer
278        let peer_addr = SocketAddrV4::new(Ipv4Addr::UNSPECIFIED, 0);
279
280        // associate the socket
281        let (addr, handle) = inet::associate_socket(
282            InetSocket::LegacyTcp(Arc::clone(socket)),
283            addr,
284            peer_addr,
285            /* check_generic_peer= */ true,
286            net_ns,
287            rng,
288        )?;
289
290        // the handle normally disassociates the socket when dropped, but the C TCP code does it's
291        // own manual disassociation, so we'll just let it do its own thing
292        std::mem::forget(handle);
293
294        // update the socket's local address
295        let socket = socket.borrow_mut();
296        let socket = socket.as_legacy_socket();
297        unsafe {
298            c::legacysocket_setSocketName(
299                socket,
300                u32::from(*addr.ip()).to_be(),
301                addr.port().to_be(),
302            )
303        };
304
305        Ok(())
306    }
307
308    pub fn readv(
309        &mut self,
310        _iovs: &[IoVec],
311        _offset: Option<libc::off_t>,
312        _flags: libc::c_int,
313        _mem: &mut MemoryManager,
314        _cb_queue: &mut CallbackQueue,
315    ) -> Result<libc::ssize_t, SyscallError> {
316        // we could call LegacyTcpSocket::recvmsg() here, but for now we expect that there are no
317        // code paths that would call LegacyTcpSocket::readv() since the readv() syscall handler
318        // should have called LegacyTcpSocket::recvmsg() instead
319        panic!("Called LegacyTcpSocket::readv() on a TCP socket.");
320    }
321
322    pub fn writev(
323        &mut self,
324        _iovs: &[IoVec],
325        _offset: Option<libc::off_t>,
326        _flags: libc::c_int,
327        _mem: &mut MemoryManager,
328        _cb_queue: &mut CallbackQueue,
329    ) -> Result<libc::ssize_t, SyscallError> {
330        // we could call LegacyTcpSocket::sendmsg() here, but for now we expect that there are no
331        // code paths that would call LegacyTcpSocket::writev() since the writev() syscall handler
332        // should have called LegacyTcpSocket::sendmsg() instead
333        panic!("Called LegacyTcpSocket::writev() on a TCP socket");
334    }
335
336    pub fn sendmsg(
337        socket: &Arc<AtomicRefCell<Self>>,
338        args: SendmsgArgs,
339        mem: &mut MemoryManager,
340        _net_ns: &NetworkNamespace,
341        _rng: impl rand::Rng,
342        _cb_queue: &mut CallbackQueue,
343    ) -> Result<libc::ssize_t, SyscallError> {
344        let socket_ref = socket.borrow_mut();
345        let tcp = socket_ref.as_legacy_tcp();
346
347        if socket_ref.state().contains(FileState::CLOSED) {
348            // A file that is referenced in the descriptor table should never be a closed file. File
349            // handles (fds) are handles to open files, so if we have a file handle to a closed
350            // file, then there's an error somewhere in Shadow. Shadow's TCP sockets do close
351            // themselves even if there are still file handles (see `_tcp_endOfFileSignalled`), so
352            // we can't make this a panic.
353            log::warn!("Sending on a closed TCP socket");
354            return Err(Errno::EBADF.into());
355        }
356
357        let Some(mut flags) = MsgFlags::from_bits(args.flags) else {
358            log::warn!("Unrecognized send flags: {:#b}", args.flags);
359            return Err(Errno::EINVAL.into());
360        };
361
362        if socket_ref.status().contains(FileStatus::NONBLOCK) {
363            flags.insert(MsgFlags::MSG_DONTWAIT);
364        }
365
366        // run in a closure so that an early return doesn't skip checking if we should block
367        let result = (|| {
368            let mut bytes_sent = 0;
369
370            for iov in args.iovs {
371                let errcode = unsafe { c::tcp_getConnectionError(tcp) };
372
373                log::trace!("Connection error state is currently {errcode}");
374
375                #[allow(clippy::if_same_then_else)]
376                if errcode > 0 {
377                    // connect() was not called yet
378                    // TODO: Can they can piggy back a connect() on sendto() if they provide an
379                    // address for the connection?
380                    if bytes_sent == 0 {
381                        return Err(Errno::EPIPE);
382                    } else {
383                        break;
384                    }
385                } else if errcode == 0 {
386                    // They connected, but never read the success code with a second call to
387                    // connect(). That's OK, proceed to send as usual.
388                } else if errcode == -libc::EISCONN {
389                    // they are connected, and we can send now
390                } else if errcode == -libc::EALREADY {
391                    // connection in progress
392                    // TODO: should we wait, or just return -EALREADY?
393                    if bytes_sent == 0 {
394                        return Err(Errno::EWOULDBLOCK);
395                    } else {
396                        break;
397                    }
398                }
399
400                // SAFETY: We're passing an immutable pointer to the memory manager. We should not
401                // have any other mutable references to the memory manager at this point.
402                let rv = Worker::with_active_host(|host| unsafe {
403                    c::tcp_sendUserData(
404                        tcp,
405                        host,
406                        iov.base.cast::<()>(),
407                        iov.len.try_into().unwrap(),
408                        0,
409                        0,
410                        mem,
411                    )
412                })
413                .unwrap();
414
415                if rv < 0 {
416                    if bytes_sent == 0 {
417                        return Err(Errno::try_from(-rv).unwrap());
418                    } else {
419                        break;
420                    }
421                }
422
423                bytes_sent += rv;
424
425                if usize::try_from(rv).unwrap() < iov.len {
426                    // stop if we didn't write all of the data in the iov
427                    break;
428                }
429            }
430
431            Ok(bytes_sent)
432        })();
433
434        // if the syscall would block and we don't have the MSG_DONTWAIT flag
435        if result == Err(Errno::EWOULDBLOCK) && !flags.contains(MsgFlags::MSG_DONTWAIT) {
436            return Err(SyscallError::new_blocked_on_file(
437                File::Socket(Socket::Inet(InetSocket::LegacyTcp(socket.clone()))),
438                FileState::WRITABLE,
439                socket_ref.supports_sa_restart(),
440            ));
441        }
442
443        Ok(result?.try_into().unwrap())
444    }
445
446    pub fn recvmsg(
447        socket: &Arc<AtomicRefCell<Self>>,
448        mut args: RecvmsgArgs,
449        mem: &mut MemoryManager,
450        _cb_queue: &mut CallbackQueue,
451    ) -> Result<RecvmsgReturn, SyscallError> {
452        let socket_ref = socket.borrow_mut();
453        let tcp = socket_ref.as_legacy_tcp();
454
455        if socket_ref.state().contains(FileState::CLOSED) {
456            // A file that is referenced in the descriptor table should never be a closed file. File
457            // handles (fds) are handles to open files, so if we have a file handle to a closed
458            // file, then there's an error somewhere in Shadow. Shadow's TCP sockets do close
459            // themselves even if there are still file handles (see `_tcp_endOfFileSignalled`), so
460            // we can't make this a panic.
461            if unsafe { c::tcp_getConnectionError(tcp) != -libc::EISCONN } {
462                // connection error will be -ENOTCONN when reading is done
463                log::warn!("Receiving on a closed TCP socket");
464                return Err(Errno::EBADF.into());
465            }
466        }
467
468        let Some(mut flags) = MsgFlags::from_bits(args.flags) else {
469            log::warn!("Unrecognized recv flags: {:#b}", args.flags);
470            return Err(Errno::EINVAL.into());
471        };
472
473        if socket_ref.status().contains(FileStatus::NONBLOCK) {
474            flags.insert(MsgFlags::MSG_DONTWAIT);
475        }
476
477        // run in a closure so that an early return doesn't skip checking if we should block
478        let result = (|| {
479            let mut bytes_read = 0;
480
481            // want to make sure we run the loop at least once so that we can return any errors
482            if args.iovs.is_empty() {
483                const EMPTY_IOV: IoVec = IoVec {
484                    base: ForeignPtr::null(),
485                    len: 0,
486                };
487                args.iovs = std::slice::from_ref(&EMPTY_IOV);
488            }
489
490            for iov in args.iovs {
491                let errcode = unsafe { c::tcp_getConnectionError(tcp) };
492
493                if errcode > 0 {
494                    // connect() was not called yet
495                    if bytes_read == 0 {
496                        return Err(Errno::ENOTCONN);
497                    } else {
498                        break;
499                    }
500                } else if errcode == -libc::EALREADY {
501                    // Connection in progress
502                    if bytes_read == 0 {
503                        return Err(Errno::EWOULDBLOCK);
504                    } else {
505                        break;
506                    }
507                }
508
509                // SAFETY: We're passing a mutable pointer to the memory manager. We should not have
510                // any other mutable references to the memory manager at this point.
511                let rv = Worker::with_active_host(|host| unsafe {
512                    c::tcp_receiveUserData(
513                        tcp,
514                        host,
515                        iov.base.cast::<()>(),
516                        iov.len.try_into().unwrap(),
517                        std::ptr::null_mut(),
518                        std::ptr::null_mut(),
519                        mem,
520                    )
521                })
522                .unwrap();
523
524                if rv < 0 {
525                    if bytes_read == 0 {
526                        return Err(Errno::try_from(-rv).unwrap());
527                    } else {
528                        break;
529                    }
530                }
531
532                bytes_read += rv;
533
534                if usize::try_from(rv).unwrap() < iov.len {
535                    // stop if we didn't receive all of the data in the iov
536                    break;
537                }
538            }
539
540            Ok(RecvmsgReturn {
541                return_val: bytes_read.try_into().unwrap(),
542                addr: None,
543                msg_flags: 0,
544                control_len: 0,
545            })
546        })();
547
548        // if the syscall would block and we don't have the MSG_DONTWAIT flag
549        if result.as_ref().err() == Some(&Errno::EWOULDBLOCK)
550            && !flags.contains(MsgFlags::MSG_DONTWAIT)
551        {
552            return Err(SyscallError::new_blocked_on_file(
553                File::Socket(Socket::Inet(InetSocket::LegacyTcp(socket.clone()))),
554                FileState::READABLE,
555                socket_ref.supports_sa_restart(),
556            ));
557        }
558
559        Ok(result?)
560    }
561
562    pub fn ioctl(
563        &mut self,
564        request: IoctlRequest,
565        arg_ptr: ForeignPtr<()>,
566        memory_manager: &mut MemoryManager,
567    ) -> SyscallResult {
568        match request {
569            // equivalent to SIOCINQ
570            IoctlRequest::FIONREAD => {
571                let len = unsafe { c::tcp_getInputBufferLength(self.as_legacy_tcp()) }
572                    .try_into()
573                    .unwrap();
574
575                let arg_ptr = arg_ptr.cast::<libc::c_int>();
576                memory_manager.write(arg_ptr, &len)?;
577
578                Ok(0.into())
579            }
580            // equivalent to SIOCOUTQ
581            IoctlRequest::TIOCOUTQ => {
582                let len = unsafe { c::tcp_getOutputBufferLength(self.as_legacy_tcp()) }
583                    .try_into()
584                    .unwrap();
585
586                let arg_ptr = arg_ptr.cast::<libc::c_int>();
587                memory_manager.write(arg_ptr, &len)?;
588
589                Ok(0.into())
590            }
591            IoctlRequest::SIOCOUTQNSD => {
592                let len = unsafe { c::tcp_getNotSentBytes(self.as_legacy_tcp()) }
593                    .try_into()
594                    .unwrap();
595
596                let arg_ptr = arg_ptr.cast::<libc::c_int>();
597                memory_manager.write(arg_ptr, &len)?;
598
599                Ok(0.into())
600            }
601            // this isn't supported by tcp
602            IoctlRequest::SIOCGSTAMP => Err(Errno::ENOENT.into()),
603            IoctlRequest::FIONBIO => {
604                panic!("This should have been handled by the ioctl syscall handler");
605            }
606            IoctlRequest::TCGETS
607            | IoctlRequest::TCSETS
608            | IoctlRequest::TCSETSW
609            | IoctlRequest::TCSETSF
610            | IoctlRequest::TCGETA
611            | IoctlRequest::TCSETA
612            | IoctlRequest::TCSETAW
613            | IoctlRequest::TCSETAF
614            | IoctlRequest::TIOCGWINSZ
615            | IoctlRequest::TIOCSWINSZ => {
616                // not a terminal
617                Err(Errno::ENOTTY.into())
618            }
619            request => {
620                warn_once_then_debug!(
621                    "We do not yet handle ioctl request {request:?} on tcp sockets"
622                );
623                Err(Errno::EINVAL.into())
624            }
625        }
626    }
627
628    pub fn stat(&self) -> Result<linux_api::stat::stat, SyscallError> {
629        warn_once_then_debug!("We do not yet handle stat calls on tcp sockets");
630        Err(Errno::EINVAL.into())
631    }
632
633    pub fn listen(
634        socket: &Arc<AtomicRefCell<Self>>,
635        backlog: i32,
636        net_ns: &NetworkNamespace,
637        rng: impl rand::Rng,
638        _cb_queue: &mut CallbackQueue,
639    ) -> Result<(), Errno> {
640        let socket_ref = socket.borrow();
641
642        // only listen on the socket if it is not used for other functions
643        let is_listening_allowed =
644            unsafe { c::tcp_isListeningAllowed(socket_ref.as_legacy_tcp()) } == 1;
645        if !is_listening_allowed {
646            log::debug!("Cannot listen on previously used socket");
647            return Err(Errno::EOPNOTSUPP);
648        }
649
650        // if we are already listening, just update the backlog and return 0
651        let is_valid_listener = unsafe { c::tcp_isValidListener(socket_ref.as_legacy_tcp()) } == 1;
652        if is_valid_listener {
653            log::trace!("Socket already set up as a listener; updating backlog");
654            unsafe { c::tcp_updateServerBacklog(socket_ref.as_legacy_tcp(), backlog) };
655            return Ok(());
656        }
657
658        // a listening socket must be bound
659        let is_bound = unsafe { c::legacysocket_isBound(socket_ref.as_legacy_socket()) } == 1;
660        if !is_bound {
661            log::trace!("Implicitly binding listener socket");
662
663            // implicit bind: bind to all interfaces at an ephemeral port
664            let local_addr = SocketAddrV4::new(Ipv4Addr::UNSPECIFIED, 0);
665
666            // this will allow us to receive packets from any peer address
667            let peer_addr = SocketAddrV4::new(Ipv4Addr::UNSPECIFIED, 0);
668
669            // associate the socket
670            let (local_addr, handle) = super::associate_socket(
671                super::InetSocket::LegacyTcp(socket.clone()),
672                local_addr,
673                peer_addr,
674                /* check_generic_peer= */ true,
675                net_ns,
676                rng,
677            )?;
678
679            // the handle normally disassociates the socket when dropped, but the C TCP code does
680            // it's own manual disassociation, so we'll just let it do its own thing
681            std::mem::forget(handle);
682
683            unsafe {
684                c::legacysocket_setSocketName(
685                    socket_ref.as_legacy_socket(),
686                    u32::from(*local_addr.ip()).to_be(),
687                    local_addr.port().to_be(),
688                )
689            };
690        }
691
692        // we are allowed to listen but not already listening; start now
693        Worker::with_active_host(|host| {
694            unsafe {
695                c::tcp_enterServerMode(
696                    socket_ref.as_legacy_tcp(),
697                    host,
698                    Worker::active_process_id().unwrap().into(),
699                    backlog,
700                )
701            };
702        })
703        .unwrap();
704
705        Ok(())
706    }
707
708    pub fn connect(
709        socket: &Arc<AtomicRefCell<Self>>,
710        peer_addr: &SockaddrStorage,
711        net_ns: &NetworkNamespace,
712        rng: impl rand::Rng,
713        _cb_queue: &mut CallbackQueue,
714    ) -> Result<(), SyscallError> {
715        let mut socket_ref = socket.borrow_mut();
716
717        if let Some(tid) = socket_ref.thread_of_blocked_connect {
718            // check if there is already a blocking connect() call on another thread
719            if tid != Worker::active_thread_id().unwrap() {
720                // connect(2) says "Generally,  connection-based protocol sockets may successfully
721                // connect() only once", but the application is attempting to call connect() in two
722                // threads on a blocking socket at the same time. Let's just return an error and
723                // hope no one ever does this.
724                log::warn!("Two threads are attempting to connect() on a blocking socket");
725                return Err(Errno::EBADFD.into());
726            }
727        }
728
729        // if the socket is already listening, return EISCONN
730        let is_valid_listener = unsafe { c::tcp_isValidListener(socket_ref.as_legacy_tcp()) } == 1;
731        if is_valid_listener {
732            return Err(Errno::EISCONN.into());
733        }
734
735        let Some(peer_addr) = peer_addr.as_inet() else {
736            return Err(Errno::EINVAL.into());
737        };
738
739        let mut peer_addr: std::net::SocketAddrV4 = (*peer_addr).into();
740
741        // https://stackoverflow.com/a/22425796
742        if peer_addr.ip().is_unspecified() {
743            peer_addr.set_ip(std::net::Ipv4Addr::LOCALHOST);
744        }
745
746        let host_default_ip = net_ns.default_ip;
747
748        // NOTE: it would be nice to use `Ipv4Addr::is_loopback` in this code rather than comparing
749        // to `Ipv4Addr::LOCALHOST`, but the rest of Shadow probably can't handle other loopback
750        // addresses (ex: 127.0.0.2) and it's probably best not to change this behaviour
751
752        // make sure we will be able to route this later
753        // TODO: should we just send the SYN and let the connection fail normally?
754        if peer_addr.ip() != &std::net::Ipv4Addr::LOCALHOST {
755            let is_routable = Worker::is_routable(host_default_ip.into(), (*peer_addr.ip()).into());
756
757            if !is_routable {
758                // can't route it - there is no node with this address
759                log::warn!(
760                    "Attempting to connect to address '{peer_addr}' for which no host exists"
761                );
762                return Err(Errno::ECONNREFUSED.into());
763            }
764        }
765
766        // a connected tcp socket must be bound
767        let is_bound = unsafe { c::legacysocket_isBound(socket_ref.as_legacy_socket()) } == 1;
768        if !is_bound {
769            log::trace!("Implicitly binding listener socket");
770
771            // implicit bind: bind to an ephemeral port (use default interface unless the remote
772            // peer is on loopback)
773            let local_addr = if peer_addr.ip() == &std::net::Ipv4Addr::LOCALHOST {
774                SocketAddrV4::new(Ipv4Addr::LOCALHOST, 0)
775            } else {
776                SocketAddrV4::new(host_default_ip, 0)
777            };
778
779            // associate the socket
780            let (local_addr, handle) = super::associate_socket(
781                super::InetSocket::LegacyTcp(socket.clone()),
782                local_addr,
783                peer_addr,
784                /* check_generic_peer= */ true,
785                net_ns,
786                rng,
787            )?;
788
789            // the handle normally disassociates the socket when dropped, but the C TCP code does
790            // it's own manual disassociation, so we'll just let it do its own thing
791            std::mem::forget(handle);
792
793            unsafe {
794                c::legacysocket_setSocketName(
795                    socket_ref.as_legacy_socket(),
796                    u32::from(*local_addr.ip()).to_be(),
797                    local_addr.port().to_be(),
798                )
799            };
800        } else if let Some(bound_addr) = socket_ref.getsockname()? {
801            // make sure the new peer address is connectable from the bound interface
802            if !bound_addr.ip().is_unspecified() {
803                // assume that a socket bound to 0.0.0.0 can connect anywhere, so only check
804                // localhost
805                match (
806                    bound_addr.ip() == Ipv4Addr::LOCALHOST,
807                    peer_addr.ip() == &Ipv4Addr::LOCALHOST,
808                ) {
809                    // bound and peer on loopback interface
810                    (true, true) => {}
811                    // neither bound nor peer on loopback interface (shadow treats any
812                    // non-127.0.0.1 address as an "internet" address)
813                    (false, false) => {}
814                    _ => return Err(Errno::EINVAL.into()),
815                }
816            }
817        }
818
819        unsafe {
820            c::legacysocket_setPeerName(
821                socket_ref.as_legacy_socket(),
822                u32::from(*peer_addr.ip()).to_be(),
823                peer_addr.port().to_be(),
824            )
825        };
826
827        // now we are ready to connect
828        let errcode = Worker::with_active_host(|host| unsafe {
829            c::legacysocket_connectToPeer(
830                socket_ref.as_legacy_socket(),
831                host,
832                u32::from(*peer_addr.ip()).to_be(),
833                peer_addr.port().to_be(),
834                libc::AF_INET as u16,
835            )
836        })
837        .unwrap();
838
839        assert!(errcode <= 0);
840
841        let mut errcode = if errcode < 0 {
842            Err(Errno::try_from(-errcode).unwrap())
843        } else {
844            Ok(())
845        };
846
847        if !socket_ref.status().contains(FileStatus::NONBLOCK) {
848            // this is a blocking connect call
849            if errcode == Err(Errno::EINPROGRESS) {
850                // This is the first time we ever called connect, and so we need to wait for the
851                // 3-way handshake to complete. We will wait indefinitely for a success or failure.
852
853                let err = SyscallError::new_blocked_on_file(
854                    File::Socket(Socket::Inet(InetSocket::LegacyTcp(Arc::clone(socket)))),
855                    FileState::ACTIVE | FileState::WRITABLE,
856                    socket_ref.supports_sa_restart(),
857                );
858
859                // block the current thread
860                socket_ref.thread_of_blocked_connect = Some(Worker::active_thread_id().unwrap());
861                return Err(err);
862            }
863
864            // if we were previously blocked (we checked the thread ID above) and are now connected
865            if socket_ref.thread_of_blocked_connect.is_some() && errcode == Err(Errno::EISCONN) {
866                // it was EINPROGRESS, but is now a successful blocking connect
867                errcode = Ok(());
868            }
869        }
870
871        // make sure we return valid error codes for connect
872        if errcode == Err(Errno::ECONNRESET) || errcode == Err(Errno::ENOTCONN) {
873            errcode = Err(Errno::EISCONN);
874        }
875        // EALREADY is well defined in man page, but Linux returns EINPROGRESS
876        if errcode == Err(Errno::EALREADY) {
877            errcode = Err(Errno::EINPROGRESS);
878        }
879
880        socket_ref.thread_of_blocked_connect = None;
881        errcode.map_err(Into::into)
882    }
883
884    pub fn accept(
885        &mut self,
886        net_ns: &NetworkNamespace,
887        rng: impl rand::Rng,
888        _cb_queue: &mut CallbackQueue,
889    ) -> Result<OpenFile, SyscallError> {
890        let is_valid_listener = unsafe { c::tcp_isValidListener(self.as_legacy_tcp()) } == 1;
891
892        // we must be listening in order to accept
893        if !is_valid_listener {
894            log::debug!("Socket is not listening");
895            return Err(Errno::EINVAL.into());
896        }
897
898        let mut peer_addr: libc::sockaddr_in = shadow_pod::zeroed();
899        peer_addr.sin_family = libc::AF_INET as u16;
900        let mut accepted_fd = -1;
901
902        // now we can check if we have anything to accept
903        let errcode = Worker::with_active_host(|host| unsafe {
904            c::tcp_acceptServerPeer(
905                self.as_legacy_tcp(),
906                host,
907                &mut peer_addr.sin_addr.s_addr,
908                &mut peer_addr.sin_port,
909                &mut accepted_fd,
910            )
911        })
912        .unwrap();
913
914        assert!(errcode <= 0);
915
916        if errcode < 0 {
917            log::trace!("TCP error when accepting connection");
918            return Err(Errno::try_from(-errcode).unwrap().into());
919        }
920
921        // we accepted something!
922        assert!(accepted_fd >= 0);
923
924        // The rust socket syscall interface expects us to return the socket object so that it can
925        // add it to the descriptor table, but the TCP code has already added it to the descriptor
926        // table (see https://github.com/shadow/shadow/issues/1780). We'll remove the socket from
927        // the descriptor table, return it to the syscall handler, and let the syscall handler
928        // re-add it to the descriptor table. It may end up with a different fd handle, but that
929        // should be fine since nothing should be relying on the socket having a specific/fixed fd
930        // handle.
931
932        let new_descriptor = Worker::with_active_host(|host| {
933            Worker::with_active_thread(|thread| {
934                thread
935                    .descriptor_table_borrow_mut(host)
936                    .deregister_descriptor(accepted_fd.try_into().unwrap())
937                    .unwrap()
938            })
939        })
940        .unwrap()
941        .unwrap();
942
943        let CompatFile::New(open_file) = new_descriptor.into_file() else {
944            panic!(
945                "The TCP code should have added the TCP socket to the descriptor table as a rust socket"
946            );
947        };
948
949        // Associate the new socket with the local:peer address pair. In
950        // previous versions of shadow, the new socket was never registered this
951        // way. Instead the packets continued to be routed to the parent
952        // listening-socket, which was responsible for routing them to this new
953        // "child" socket.  But that led to bugs such as
954        // https://github.com/shadow/shadow/issues/3563.
955        {
956            let File::Socket(Socket::Inet(InetSocket::LegacyTcp(new_socket))) =
957                open_file.inner_file()
958            else {
959                panic!("Expected this to be a LegacyTcpSocket");
960            };
961
962            // get and validate child peer and local addresses.
963            let child_peer_addr;
964            let child_local_addr;
965            {
966                let new_socket_ref = new_socket.borrow();
967                // sanity check: make sure new socket peer address matches address returned from
968                // tcp_acceptServerPeer() above
969                {
970                    let mut ip: libc::in_addr_t = 0;
971                    let mut port: libc::in_port_t = 0;
972
973                    // should return ip and port in network byte order
974                    let okay = unsafe {
975                        c::legacysocket_getPeerName(
976                            new_socket_ref.as_legacy_socket(),
977                            &mut ip,
978                            &mut port,
979                        )
980                    };
981
982                    assert_eq!(okay, 1);
983                    assert_eq!(ip, peer_addr.sin_addr.s_addr);
984                    assert_eq!(port, peer_addr.sin_port);
985                }
986                child_peer_addr = new_socket_ref
987                    .getpeername()
988                    .expect("error finding child peer address")
989                    .expect("missing child peer address");
990                child_local_addr = new_socket_ref
991                    .getsockname()
992                    .expect("error finding local address")
993                    .expect("missing local address");
994                let parent_local_addr = self
995                    .getsockname()
996                    .expect("error finding parent local address")
997                    .expect("missing parent local address");
998                // port should be the same as the listening socket.
999                debug_assert_eq!(
1000                    child_local_addr.port(),
1001                    parent_local_addr.port(),
1002                    "local address of accept'ed socket doesn't match parent listening socket"
1003                );
1004                // address should be same as the parent address, unless it was unspecified.
1005                if !parent_local_addr.ip().is_unspecified() {
1006                    debug_assert_eq!(child_local_addr.ip(), parent_local_addr.ip());
1007                }
1008                // in any case, address of the child should *not* be unspecified.
1009                debug_assert!(!child_local_addr.ip().is_unspecified());
1010                debug_assert_ne!(!child_local_addr.port(), 0);
1011            }
1012
1013            let (_addr, handle) = inet::associate_socket(
1014                InetSocket::LegacyTcp(Arc::clone(new_socket)),
1015                SocketAddrV4::from(child_local_addr),
1016                SocketAddrV4::from(child_peer_addr),
1017                /* Allow the parent/listening socket to be bound to the same address,
1018                 * with a missing/generic peer. */
1019                /* check_generic_peer= */
1020                false,
1021                net_ns,
1022                rng,
1023            )?;
1024            // the handle normally disassociates the socket when dropped, but
1025            // the C TCP code does its own manual disassociation, so we'll just
1026            // let it do its own thing.
1027            std::mem::forget(handle);
1028        }
1029
1030        Ok(open_file)
1031    }
1032
1033    pub fn shutdown(
1034        &mut self,
1035        how: Shutdown,
1036        _cb_queue: &mut CallbackQueue,
1037    ) -> Result<(), SyscallError> {
1038        let how = match how {
1039            Shutdown::SHUT_RD => libc::SHUT_RD,
1040            Shutdown::SHUT_WR => libc::SHUT_WR,
1041            Shutdown::SHUT_RDWR => libc::SHUT_RDWR,
1042        };
1043
1044        let errcode = Worker::with_active_host(|host| unsafe {
1045            c::tcp_shutdown(self.as_legacy_tcp(), host, how)
1046        })
1047        .unwrap();
1048
1049        assert!(errcode <= 0);
1050
1051        if errcode < 0 {
1052            return Err(Errno::try_from(-errcode).unwrap().into());
1053        }
1054
1055        Ok(())
1056    }
1057
1058    pub fn getsockopt(
1059        &self,
1060        level: libc::c_int,
1061        optname: libc::c_int,
1062        optval_ptr: ForeignPtr<()>,
1063        optlen: libc::socklen_t,
1064        memory_manager: &mut MemoryManager,
1065        _cb_queue: &mut CallbackQueue,
1066    ) -> Result<libc::socklen_t, SyscallError> {
1067        match (level, optname) {
1068            (libc::SOL_TCP, libc::TCP_INFO) => {
1069                let mut info = shadow_pod::zeroed();
1070                unsafe { c::tcp_getInfo(self.as_legacy_tcp(), &mut info) };
1071
1072                let optval_ptr = optval_ptr.cast::<crate::cshadow::tcp_info>();
1073                let bytes_written =
1074                    write_partial(memory_manager, &info, optval_ptr, optlen as usize)?;
1075
1076                Ok(bytes_written as libc::socklen_t)
1077            }
1078            (libc::SOL_TCP, libc::TCP_NODELAY) => {
1079                // shadow doesn't support nagle's algorithm, so shadow always behaves as if
1080                // TCP_NODELAY is enabled
1081                let val = 1;
1082
1083                let optval_ptr = optval_ptr.cast::<libc::c_int>();
1084                let bytes_written =
1085                    write_partial(memory_manager, &val, optval_ptr, optlen as usize)?;
1086
1087                Ok(bytes_written as libc::socklen_t)
1088            }
1089            (libc::SOL_TCP, libc::TCP_CONGESTION) => {
1090                // the value of TCP_CA_NAME_MAX in linux
1091                const CONG_NAME_MAX: usize = 16;
1092
1093                if optval_ptr.is_null() {
1094                    return Err(Errno::EINVAL.into());
1095                }
1096
1097                let name: *const libc::c_char =
1098                    unsafe { c::tcpcong_nameStr(c::tcp_cong(self.as_legacy_tcp())) };
1099                assert!(!name.is_null(), "shadow's congestion type has no name");
1100                let name = unsafe { CStr::from_ptr(name) };
1101                let name = name.to_bytes_with_nul();
1102
1103                let bytes_to_copy = *[optlen as usize, CONG_NAME_MAX, name.len()]
1104                    .iter()
1105                    .min()
1106                    .unwrap();
1107
1108                let name = &name[..bytes_to_copy];
1109                let optval_ptr = optval_ptr.cast::<u8>();
1110                let optval_ptr = ForeignArrayPtr::new(optval_ptr, bytes_to_copy);
1111
1112                memory_manager.copy_to_ptr(optval_ptr, name)?;
1113
1114                // the len value returned by linux seems to be independent from the actual string length
1115                Ok(std::cmp::min(optlen as usize, CONG_NAME_MAX) as libc::socklen_t)
1116            }
1117            (libc::SOL_SOCKET, libc::SO_SNDBUF) => {
1118                let sndbuf_size: libc::c_int =
1119                    unsafe { c::legacysocket_getOutputBufferSize(self.as_legacy_socket()) }
1120                        .try_into()
1121                        .unwrap();
1122
1123                let optval_ptr = optval_ptr.cast::<libc::c_int>();
1124                let bytes_written =
1125                    write_partial(memory_manager, &sndbuf_size, optval_ptr, optlen as usize)?;
1126
1127                Ok(bytes_written as libc::socklen_t)
1128            }
1129            (libc::SOL_SOCKET, libc::SO_RCVBUF) => {
1130                let rcvbuf_size: libc::c_int =
1131                    unsafe { c::legacysocket_getInputBufferSize(self.as_legacy_socket()) }
1132                        .try_into()
1133                        .unwrap();
1134
1135                let optval_ptr = optval_ptr.cast::<libc::c_int>();
1136                let bytes_written =
1137                    write_partial(memory_manager, &rcvbuf_size, optval_ptr, optlen as usize)?;
1138
1139                Ok(bytes_written as libc::socklen_t)
1140            }
1141            (libc::SOL_SOCKET, libc::SO_ERROR) => {
1142                // return error for failed connect() attempts
1143                let conn_err = unsafe { c::tcp_getConnectionError(self.as_legacy_tcp()) };
1144
1145                let error = if conn_err == -libc::ECONNRESET || conn_err == -libc::ECONNREFUSED {
1146                    // result is a positive errcode
1147                    -conn_err
1148                } else {
1149                    0
1150                };
1151
1152                let optval_ptr = optval_ptr.cast::<libc::c_int>();
1153                let bytes_written =
1154                    write_partial(memory_manager, &error, optval_ptr, optlen as usize)?;
1155
1156                Ok(bytes_written as libc::socklen_t)
1157            }
1158            (libc::SOL_SOCKET, libc::SO_DOMAIN) => {
1159                let domain = libc::AF_INET;
1160
1161                let optval_ptr = optval_ptr.cast::<libc::c_int>();
1162                let bytes_written =
1163                    write_partial(memory_manager, &domain, optval_ptr, optlen as usize)?;
1164
1165                Ok(bytes_written as libc::socklen_t)
1166            }
1167            (libc::SOL_SOCKET, libc::SO_TYPE) => {
1168                let sock_type = libc::SOCK_STREAM;
1169
1170                let optval_ptr = optval_ptr.cast::<libc::c_int>();
1171                let bytes_written =
1172                    write_partial(memory_manager, &sock_type, optval_ptr, optlen as usize)?;
1173
1174                Ok(bytes_written as libc::socklen_t)
1175            }
1176            (libc::SOL_SOCKET, libc::SO_PROTOCOL) => {
1177                let protocol = libc::IPPROTO_TCP;
1178
1179                let optval_ptr = optval_ptr.cast::<libc::c_int>();
1180                let bytes_written =
1181                    write_partial(memory_manager, &protocol, optval_ptr, optlen as usize)?;
1182
1183                Ok(bytes_written as libc::socklen_t)
1184            }
1185            (libc::SOL_SOCKET, libc::SO_ACCEPTCONN) => {
1186                let is_listener = unsafe { c::tcp_isValidListener(self.as_legacy_tcp()) };
1187
1188                let optval_ptr = optval_ptr.cast::<libc::c_int>();
1189                let bytes_written =
1190                    write_partial(memory_manager, &is_listener, optval_ptr, optlen as usize)?;
1191
1192                Ok(bytes_written as libc::socklen_t)
1193            }
1194            (libc::SOL_SOCKET, libc::SO_BROADCAST) => {
1195                let optval_ptr = optval_ptr.cast::<libc::c_int>();
1196                // we don't support broadcast sockets, so just just return the default 0
1197                let bytes_written = write_partial(memory_manager, &0, optval_ptr, optlen as usize)?;
1198
1199                Ok(bytes_written as libc::socklen_t)
1200            }
1201            _ => {
1202                log_once_per_value_at_level!(
1203                    (level, optname),
1204                    (i32, i32),
1205                    log::Level::Warn,
1206                    log::Level::Debug,
1207                    "getsockopt called with unsupported level {level} and opt {optname}"
1208                );
1209                Err(Errno::ENOPROTOOPT.into())
1210            }
1211        }
1212    }
1213
1214    pub fn setsockopt(
1215        &mut self,
1216        level: libc::c_int,
1217        optname: libc::c_int,
1218        optval_ptr: ForeignPtr<()>,
1219        optlen: libc::socklen_t,
1220        memory_manager: &MemoryManager,
1221    ) -> Result<(), SyscallError> {
1222        match (level, optname) {
1223            (libc::SOL_TCP, libc::TCP_NODELAY) => {
1224                // Shadow doesn't support nagle's algorithm, so Shadow always behaves as if
1225                // TCP_NODELAY is enabled. Some programs will fail if `setsockopt(fd, SOL_TCP,
1226                // TCP_NODELAY, &1, sizeof(int))` returns an error, so we treat this as a no-op for
1227                // compatibility.
1228
1229                type OptType = libc::c_int;
1230
1231                if usize::try_from(optlen).unwrap() < std::mem::size_of::<OptType>() {
1232                    return Err(Errno::EINVAL.into());
1233                }
1234
1235                let optval_ptr = optval_ptr.cast::<OptType>();
1236                let enable = memory_manager.read(optval_ptr)?;
1237
1238                if enable != 0 {
1239                    // wants to enable TCP_NODELAY
1240                    log::debug!("Ignoring TCP_NODELAY");
1241                } else {
1242                    // wants to disable TCP_NODELAY
1243                    log::warn!(
1244                        "Cannot disable TCP_NODELAY since shadow does not implement Nagle's algorithm."
1245                    );
1246                    return Err(Errno::ENOPROTOOPT.into());
1247                }
1248            }
1249            (libc::SOL_TCP, libc::TCP_CONGESTION) => {
1250                // the value of TCP_CA_NAME_MAX in linux
1251                const CONG_NAME_MAX: usize = 16;
1252
1253                let mut name = [0u8; CONG_NAME_MAX];
1254
1255                let optlen = std::cmp::min(optlen as usize, CONG_NAME_MAX);
1256                let name = &mut name[..optlen];
1257
1258                let optval_ptr = optval_ptr.cast::<u8>();
1259                let optval_ptr = ForeignArrayPtr::new(optval_ptr, optlen);
1260                memory_manager.copy_from_ptr(name, optval_ptr)?;
1261
1262                // truncate the name at the first NUL character if there is one, but don't include
1263                // the NUL since in linux the strings don't need a NUL
1264                let name = name
1265                    .iter()
1266                    .position(|x| *x == 0)
1267                    .map(|x| &name[..x])
1268                    .unwrap_or(name);
1269
1270                let reno = unsafe { CStr::from_ptr(c::TCP_CONG_RENO_NAME) }.to_bytes();
1271
1272                if name != reno {
1273                    log::warn!("Shadow sockets only support '{reno:?}' for TCP_CONGESTION");
1274                    return Err(Errno::ENOENT.into());
1275                }
1276
1277                // shadow doesn't support other congestion types, so do nothing
1278            }
1279            (libc::SOL_SOCKET, libc::SO_SNDBUF) => {
1280                type OptType = libc::c_int;
1281
1282                if usize::try_from(optlen).unwrap() < std::mem::size_of::<OptType>() {
1283                    return Err(Errno::EINVAL.into());
1284                }
1285
1286                let optval_ptr = optval_ptr.cast::<OptType>();
1287                let val: u64 = memory_manager
1288                    .read(optval_ptr)?
1289                    .try_into()
1290                    .or(Err(Errno::EINVAL))?;
1291
1292                // linux kernel doubles this value upon setting
1293                let val = val * 2;
1294
1295                // Linux also has limits SOCK_MIN_SNDBUF (slightly greater than 4096) and the sysctl
1296                // max limit. We choose a reasonable lower limit for Shadow. The minimum limit in
1297                // man 7 socket is incorrect.
1298                let val = std::cmp::max(val, 4096);
1299
1300                // This upper limit was added as an arbitrarily high number so that we don't change
1301                // Shadow's behaviour, but also prevents an application from setting this to
1302                // something unnecessarily large like INT_MAX.
1303                let val = std::cmp::min(val, 268435456); // 2^28 = 256 MiB
1304
1305                unsafe { c::legacysocket_setOutputBufferSize(self.as_legacy_socket(), val) };
1306                unsafe { c::tcp_disableSendBufferAutotuning(self.as_legacy_tcp()) };
1307            }
1308            (libc::SOL_SOCKET, libc::SO_RCVBUF) => {
1309                type OptType = libc::c_int;
1310
1311                if usize::try_from(optlen).unwrap() < std::mem::size_of::<OptType>() {
1312                    return Err(Errno::EINVAL.into());
1313                }
1314
1315                let optval_ptr = optval_ptr.cast::<OptType>();
1316                let val: u64 = memory_manager
1317                    .read(optval_ptr)?
1318                    .try_into()
1319                    .or(Err(Errno::EINVAL))?;
1320
1321                // linux kernel doubles this value upon setting
1322                let val = val * 2;
1323
1324                // Linux also has limits SOCK_MIN_RCVBUF (slightly greater than 2048) and the sysctl
1325                // max limit. We choose a reasonable lower limit for Shadow. The minimum limit in
1326                // man 7 socket is incorrect.
1327                let val = std::cmp::max(val, 2048);
1328
1329                // This upper limit was added as an arbitrarily high number so that we don't change
1330                // Shadow's behaviour, but also prevents an application from setting this to
1331                // something unnecessarily large like INT_MAX.
1332                let val = std::cmp::min(val, 268435456); // 2^28 = 256 MiB
1333
1334                unsafe { c::legacysocket_setInputBufferSize(self.as_legacy_socket(), val) };
1335                unsafe { c::tcp_disableReceiveBufferAutotuning(self.as_legacy_tcp()) };
1336            }
1337            (libc::SOL_SOCKET, libc::SO_REUSEADDR) => {
1338                // TODO: implement this, tor and tgen use it
1339                log::trace!("setsockopt SO_REUSEADDR not yet implemented");
1340            }
1341            (libc::SOL_SOCKET, libc::SO_REUSEPORT) => {
1342                // TODO: implement this, tgen uses it
1343                log::trace!("setsockopt SO_REUSEPORT not yet implemented");
1344            }
1345            (libc::SOL_SOCKET, libc::SO_KEEPALIVE) => {
1346                // TODO: implement this, libevent uses it in
1347                // evconnlistener_new_bind()
1348                log::trace!("setsockopt SO_KEEPALIVE not yet implemented");
1349            }
1350            (libc::SOL_SOCKET, libc::SO_BROADCAST) => {
1351                type OptType = libc::c_int;
1352
1353                if usize::try_from(optlen).unwrap() < std::mem::size_of::<OptType>() {
1354                    return Err(Errno::EINVAL.into());
1355                }
1356
1357                let optval_ptr = optval_ptr.cast::<OptType>();
1358                let val = memory_manager.read(optval_ptr)?;
1359
1360                if val == 0 {
1361                    // we don't support broadcast sockets, so an attempt to disable is okay
1362                } else {
1363                    // TODO: implement this, pkg.go.dev/net uses it
1364                    warn_once_then_debug!(
1365                        "setsockopt SO_BROADCAST not yet implemented for tcp; ignoring and returning 0"
1366                    );
1367                }
1368            }
1369            _ => {
1370                log_once_per_value_at_level!(
1371                    (level, optname),
1372                    (i32, i32),
1373                    log::Level::Warn,
1374                    log::Level::Debug,
1375                    "setsockopt called with unsupported level {level} and opt {optname}"
1376                );
1377                return Err(Errno::ENOPROTOOPT.into());
1378            }
1379        }
1380
1381        Ok(())
1382    }
1383
1384    pub fn add_listener(
1385        &mut self,
1386        monitoring_state: FileState,
1387        monitoring_signals: FileSignals,
1388        filter: StateListenerFilter,
1389        notify_fn: impl Fn(FileState, FileState, FileSignals, &mut CallbackQueue)
1390        + Send
1391        + Sync
1392        + 'static,
1393    ) -> StateListenHandle {
1394        let event_source = unsafe { c::legacyfile_getEventSource(self.as_legacy_file()) };
1395        let event_source = unsafe { event_source.as_ref() }.unwrap();
1396
1397        Worker::with_active_host(|host| {
1398            let mut event_source = event_source.borrow_mut(host.root());
1399            event_source.add_listener(monitoring_state, monitoring_signals, filter, notify_fn)
1400        })
1401        .unwrap()
1402    }
1403
1404    pub fn add_legacy_listener(&mut self, ptr: HostTreePointer<c::StatusListener>) {
1405        unsafe { c::legacyfile_addListener(self.as_legacy_file(), ptr.ptr()) };
1406    }
1407
1408    pub fn remove_legacy_listener(&mut self, ptr: *mut c::StatusListener) {
1409        unsafe { c::legacyfile_removeListener(self.as_legacy_file(), ptr) };
1410    }
1411
1412    pub fn state(&self) -> FileState {
1413        unsafe { c::legacyfile_getStatus(self.as_legacy_file()) }
1414    }
1415}
1416
1417impl std::ops::Drop for LegacyTcpSocket {
1418    fn drop(&mut self) {
1419        unsafe { c::legacyfile_unref(self.socket.ptr() as *mut libc::c_void) };
1420    }
1421}