Support timerfd-related syscall
This commit is contained in:
		
							parent
							
								
									1e7d258989
								
							
						
					
					
						commit
						eb07b01ac0
					
				| @ -147,6 +147,10 @@ enclave { | ||||
|             [out] struct timespec* rem | ||||
|         ) propagate_errno; | ||||
| 
 | ||||
|         int occlum_ocall_timerfd_create(clockid_t clockid, int flags) propagate_errno; | ||||
|         int occlum_ocall_timerfd_settime(int fd, int flags, [in] const struct itimerspec* new_value, [out] struct itimerspec* old_value) propagate_errno; | ||||
|         int occlum_ocall_timerfd_gettime(int fd, [out] struct itimerspec* curr_value) propagate_errno; | ||||
| 
 | ||||
|         void occlum_ocall_sync(void); | ||||
| 
 | ||||
|         void* occlum_ocall_posix_memalign(size_t alignment, size_t size); | ||||
|  | ||||
| @ -1,8 +1,11 @@ | ||||
| #ifndef __OCCLUM_EDL_TYPES_H__ | ||||
| #define __OCCLUM_EDL_TYPES_H__ | ||||
| 
 | ||||
| #include <sys/struct_timespec.h> | ||||
| 
 | ||||
| typedef long                time_t; | ||||
| typedef long                suseconds_t; | ||||
| typedef long                syscall_slong_t; | ||||
| typedef int                 clockid_t; | ||||
| 
 | ||||
| struct timeval { | ||||
| @ -16,6 +19,16 @@ struct occlum_stdio_fds { | ||||
|     int stderr_fd; | ||||
| }; | ||||
| 
 | ||||
| typedef struct _timespec{ | ||||
|     time_t tv_sec; | ||||
|     syscall_slong_t tv_nsec; | ||||
| }; | ||||
| 
 | ||||
| typedef struct itimerspec{ | ||||
|     struct _timespec it_interval; | ||||
|     struct _timespec it_value; | ||||
| } itimerspec_t; | ||||
| 
 | ||||
| #define FD_SETSIZE 1024 | ||||
| typedef struct { | ||||
|     unsigned long fds_bits[FD_SETSIZE / 8 / sizeof(long)]; | ||||
|  | ||||
| @ -31,6 +31,7 @@ pub use self::pipe::PipeType; | ||||
| pub use self::rootfs::ROOT_INODE; | ||||
| pub use self::stdio::{HostStdioFds, StdinFile, StdoutFile}; | ||||
| pub use self::syscalls::*; | ||||
| pub use self::timer_file::{AsTimer, TimerCreationFlags, TimerFile}; | ||||
| 
 | ||||
| pub mod channel; | ||||
| mod dev_fs; | ||||
| @ -50,6 +51,7 @@ mod rootfs; | ||||
| mod sefs; | ||||
| mod stdio; | ||||
| mod syscalls; | ||||
| mod timer_file; | ||||
| 
 | ||||
| /// Split a `path` str to `(base_path, file_name)`
 | ||||
| fn split_path(path: &str) -> (&str, &str) { | ||||
|  | ||||
| @ -5,6 +5,8 @@ use super::file_ops::{ | ||||
|     StatFlags, UnlinkFlags, AT_FDCWD, | ||||
| }; | ||||
| use super::fs_ops; | ||||
| use super::time::{clockid_t, itimerspec_t, ClockID}; | ||||
| use super::timer_file::{TimerCreationFlags, TimerSetFlags}; | ||||
| use super::*; | ||||
| use util::mem_util::from_user; | ||||
| 
 | ||||
| @ -35,6 +37,66 @@ pub fn do_eventfd2(init_val: u32, flags: i32) -> Result<isize> { | ||||
|     Ok(fd as isize) | ||||
| } | ||||
| 
 | ||||
| pub fn do_timerfd_create(clockid: clockid_t, flags: i32) -> Result<isize> { | ||||
|     debug!("timerfd: clockid {}, flags {} ", clockid, flags); | ||||
| 
 | ||||
|     let clockid = ClockID::from_raw(clockid)?; | ||||
|     match clockid { | ||||
|         ClockID::CLOCK_REALTIME | ClockID::CLOCK_MONOTONIC => {} | ||||
|         _ => { | ||||
|             return_errno!(EINVAL, "invalid clockid"); | ||||
|         } | ||||
|     } | ||||
|     let timer_create_flags = | ||||
|         TimerCreationFlags::from_bits(flags).ok_or_else(|| errno!(EINVAL, "invalid flags"))?; | ||||
|     let file_ref: Arc<dyn File> = { | ||||
|         let timer = TimerFile::new(clockid, timer_create_flags)?; | ||||
|         Arc::new(timer) | ||||
|     }; | ||||
| 
 | ||||
|     let fd = current!().add_file( | ||||
|         file_ref, | ||||
|         timer_create_flags.contains(TimerCreationFlags::TFD_CLOEXEC), | ||||
|     ); | ||||
|     Ok(fd as isize) | ||||
| } | ||||
| 
 | ||||
| pub fn do_timerfd_settime( | ||||
|     fd: FileDesc, | ||||
|     flags: i32, | ||||
|     new_value_ptr: *const itimerspec_t, | ||||
|     old_value_ptr: *mut itimerspec_t, | ||||
| ) -> Result<isize> { | ||||
|     from_user::check_ptr(new_value_ptr)?; | ||||
|     let new_value = itimerspec_t::from_raw_ptr(new_value_ptr)?; | ||||
|     let timer_set_flags = | ||||
|         TimerSetFlags::from_bits(flags).ok_or_else(|| errno!(EINVAL, "invalid flags"))?; | ||||
| 
 | ||||
|     let current = current!(); | ||||
|     let file = current.file(fd)?; | ||||
|     let timerfile = file.as_timer()?; | ||||
|     let old_value = timerfile.set_time(timer_set_flags, &new_value)?; | ||||
|     if !old_value_ptr.is_null() { | ||||
|         from_user::check_mut_ptr(old_value_ptr)?; | ||||
|         unsafe { | ||||
|             old_value_ptr.write(old_value); | ||||
|         } | ||||
|     } | ||||
|     Ok(0) | ||||
| } | ||||
| 
 | ||||
| pub fn do_timerfd_gettime(fd: FileDesc, curr_value_ptr: *mut itimerspec_t) -> Result<isize> { | ||||
|     from_user::check_mut_ptr(curr_value_ptr)?; | ||||
|     let current = current!(); | ||||
|     let file = current.file(fd)?; | ||||
|     let timerfile = file.as_timer()?; | ||||
|     let curr_value = timerfile.time()?; | ||||
|     unsafe { | ||||
|         curr_value_ptr.write(curr_value); | ||||
|     } | ||||
|     Ok(0) | ||||
| } | ||||
| 
 | ||||
| pub fn do_open(path: *const i8, flags: u32, mode: u32) -> Result<isize> { | ||||
|     self::do_openat(AT_FDCWD, path, flags, mode) | ||||
| } | ||||
|  | ||||
							
								
								
									
										182
									
								
								src/libos/src/fs/timer_file.rs
									
									
									
									
									
										Normal file
									
								
							
							
								
								
								
								
								
									
									
								
							
						
						
									
										182
									
								
								src/libos/src/fs/timer_file.rs
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,182 @@ | ||||
| use super::*; | ||||
| 
 | ||||
| use crate::time::{clockid_t, itimerspec_t, timespec_t, ClockID}; | ||||
| use atomic::{Atomic, Ordering}; | ||||
| use std::time::Duration; | ||||
| 
 | ||||
| /// Native Linux timerfd
 | ||||
| #[derive(Debug)] | ||||
| pub struct TimerFile { | ||||
|     host_fd: HostFd, | ||||
|     host_events: Atomic<IoEvents>, | ||||
|     notifier: IoNotifier, | ||||
| } | ||||
| 
 | ||||
| impl TimerFile { | ||||
|     pub fn new(clockid: ClockID, flags: TimerCreationFlags) -> Result<Self> { | ||||
|         let raw_host_fd = try_libc!({ | ||||
|             let mut ret: i32 = 0; | ||||
|             let status = occlum_ocall_timerfd_create(&mut ret, clockid as clockid_t, flags.bits()); | ||||
|             assert!(status == sgx_status_t::SGX_SUCCESS); | ||||
|             ret | ||||
|         }) as FileDesc; | ||||
|         let host_fd = HostFd::new(raw_host_fd); | ||||
|         let host_events = Atomic::new(IoEvents::empty()); | ||||
|         let notifier = IoNotifier::new(); | ||||
|         Ok(Self { | ||||
|             host_fd, | ||||
|             host_events, | ||||
|             notifier, | ||||
|         }) | ||||
|     } | ||||
| 
 | ||||
|     pub fn host_fd(&self) -> c_int { | ||||
|         self.host_fd.to_raw() as c_int | ||||
|     } | ||||
| 
 | ||||
|     pub fn set_time(&self, flags: TimerSetFlags, new_value: &itimerspec_t) -> Result<itimerspec_t> { | ||||
|         let mut ret = 0; | ||||
|         let mut old_value: itimerspec_t = Default::default(); | ||||
|         let fd = self.host_fd() as FileDesc; | ||||
|         let sgx_status = unsafe { | ||||
|             occlum_ocall_timerfd_settime(&mut ret, fd, flags.bits(), new_value, &mut old_value) | ||||
|         }; | ||||
|         assert!(sgx_status == sgx_status_t::SGX_SUCCESS); | ||||
|         assert!(ret == 0 || libc::errno() == Errno::EINTR as i32); | ||||
|         if ret != 0 { | ||||
|             return_errno!(EINTR, "settime interrupted"); | ||||
|         } | ||||
|         Ok(old_value) | ||||
|     } | ||||
| 
 | ||||
|     pub fn time(&self) -> Result<itimerspec_t> { | ||||
|         let mut ret = 0; | ||||
|         let mut curr_value: itimerspec_t = Default::default(); | ||||
|         let fd = self.host_fd() as FileDesc; | ||||
|         let sgx_status = unsafe { occlum_ocall_timerfd_gettime(&mut ret, fd, &mut curr_value) }; | ||||
|         assert!(sgx_status == sgx_status_t::SGX_SUCCESS); | ||||
|         assert!(ret == 0 || libc::errno() == Errno::EINTR as i32); | ||||
|         if ret != 0 { | ||||
|             return_errno!(EINTR, "gettime interrupted"); | ||||
|         } | ||||
|         Ok(curr_value) | ||||
|     } | ||||
| } | ||||
| 
 | ||||
| bitflags! { | ||||
|     pub struct TimerCreationFlags: i32 { | ||||
|         /// Provides semaphore-like semantics for reads from the new file descriptor
 | ||||
|         /// Non-blocking
 | ||||
|         const TFD_NONBLOCK  = 1 << 11; | ||||
|         /// Close on exec
 | ||||
|         const TFD_CLOEXEC   = 1 << 19; | ||||
|     } | ||||
| } | ||||
| 
 | ||||
| bitflags! { | ||||
|     pub struct TimerSetFlags: i32 { | ||||
|         const TFD_TIMER_ABSTIME = 1 << 0; | ||||
|         const TFD_TIMER_CANCEL_ON_SET = 1 << 1; | ||||
|     } | ||||
| } | ||||
| 
 | ||||
| extern "C" { | ||||
|     fn occlum_ocall_timerfd_create(ret: *mut i32, clockid: i32, flags: i32) -> sgx_status_t; | ||||
| } | ||||
| 
 | ||||
| extern "C" { | ||||
|     fn occlum_ocall_timerfd_settime( | ||||
|         ret: *mut i32, | ||||
|         fd: u32, | ||||
|         flags: i32, | ||||
|         new_value: *const itimerspec_t, | ||||
|         old_value: *mut itimerspec_t, | ||||
|     ) -> sgx_status_t; | ||||
| } | ||||
| 
 | ||||
| extern "C" { | ||||
|     fn occlum_ocall_timerfd_gettime( | ||||
|         ret: *mut i32, | ||||
|         fd: u32, | ||||
|         curr_value: *mut itimerspec_t, | ||||
|     ) -> sgx_status_t; | ||||
| } | ||||
| 
 | ||||
| impl File for TimerFile { | ||||
|     fn read(&self, buf: &mut [u8]) -> Result<usize> { | ||||
|         let (buf_ptr, buf_len) = buf.as_mut().as_mut_ptr_and_len(); | ||||
| 
 | ||||
|         let ret = try_libc!(libc::ocall::read( | ||||
|             self.host_fd.to_raw() as i32, | ||||
|             buf_ptr as *mut c_void, | ||||
|             buf_len | ||||
|         )) as usize; | ||||
|         assert!(ret <= buf.len()); | ||||
|         Ok(ret) | ||||
|     } | ||||
| 
 | ||||
|     // TODO: implement ioctl
 | ||||
|     // fn ioctl(&self, cmd: &mut IoctlCmd) -> Result<i32> {
 | ||||
|     //     self.ioctl_impl(cmd)
 | ||||
|     // }
 | ||||
| 
 | ||||
|     fn access_mode(&self) -> Result<AccessMode> { | ||||
|         Ok(AccessMode::O_RDWR) | ||||
|     } | ||||
| 
 | ||||
|     fn status_flags(&self) -> Result<StatusFlags> { | ||||
|         let ret = try_libc!(libc::ocall::fcntl_arg0(self.host_fd(), libc::F_GETFL)); | ||||
|         Ok(StatusFlags::from_bits_truncate(ret as u32)) | ||||
|     } | ||||
| 
 | ||||
|     fn set_status_flags(&self, new_status_flags: StatusFlags) -> Result<()> { | ||||
|         let valid_flags_mask = StatusFlags::O_APPEND | ||||
|             | StatusFlags::O_ASYNC | ||||
|             | StatusFlags::O_DIRECT | ||||
|             | StatusFlags::O_NOATIME | ||||
|             | StatusFlags::O_NONBLOCK; | ||||
|         let raw_status_flags = (new_status_flags & valid_flags_mask).bits(); | ||||
|         try_libc!(libc::ocall::fcntl_arg1( | ||||
|             self.host_fd(), | ||||
|             libc::F_SETFL, | ||||
|             raw_status_flags as c_int | ||||
|         )); | ||||
|         Ok(()) | ||||
|     } | ||||
| 
 | ||||
|     fn poll_new(&self) -> IoEvents { | ||||
|         self.host_events.load(Ordering::Acquire) | ||||
|     } | ||||
| 
 | ||||
|     fn notifier(&self) -> Option<&IoNotifier> { | ||||
|         Some(&self.notifier) | ||||
|     } | ||||
| 
 | ||||
|     fn host_fd(&self) -> Option<&HostFd> { | ||||
|         Some(&self.host_fd) | ||||
|     } | ||||
| 
 | ||||
|     fn update_host_events(&self, ready: &IoEvents, mask: &IoEvents, trigger_notifier: bool) { | ||||
|         self.host_events.update(ready, mask, Ordering::Release); | ||||
| 
 | ||||
|         if trigger_notifier { | ||||
|             self.notifier.broadcast(ready); | ||||
|         } | ||||
|     } | ||||
| 
 | ||||
|     fn as_any(&self) -> &dyn Any { | ||||
|         self | ||||
|     } | ||||
| } | ||||
| 
 | ||||
| pub trait AsTimer { | ||||
|     fn as_timer(&self) -> Result<&TimerFile>; | ||||
| } | ||||
| 
 | ||||
| impl AsTimer for FileRef { | ||||
|     fn as_timer(&self) -> Result<&TimerFile> { | ||||
|         self.as_any() | ||||
|             .downcast_ref::<TimerFile>() | ||||
|             .ok_or_else(|| errno!(EBADF, "not an timer file")) | ||||
|     } | ||||
| } | ||||
| @ -15,7 +15,7 @@ pub use self::poll::{do_poll, PollEvent, PollEventFlags}; | ||||
| pub use self::poll_new::{do_poll_new, PollFd}; | ||||
| pub use self::select::{do_select, FdSetExt}; | ||||
| 
 | ||||
| use fs::{AsEvent, AsINodeFile, CreationFlags, File, FileDesc, FileRef, HostFd, PipeType}; | ||||
| use fs::{AsEvent, AsINodeFile, AsTimer, CreationFlags, File, FileDesc, FileRef, HostFd, PipeType}; | ||||
| use std::any::Any; | ||||
| use std::convert::TryFrom; | ||||
| use std::fmt; | ||||
|  | ||||
| @ -119,6 +119,10 @@ pub fn do_poll(pollfds: &mut [PollEvent], timeout: *mut timeval_t) -> Result<usi | ||||
|             let fd = eventfd.host_fd() as FileDesc; | ||||
|             index_host_pollfds.push(i); | ||||
|             host_pollfds.push(PollEvent::new(fd, pollfd.events())); | ||||
|         } else if let Ok(timerfd) = file_ref.as_timer() { | ||||
|             let fd = timerfd.host_fd() as FileDesc; | ||||
|             index_host_pollfds.push(i); | ||||
|             host_pollfds.push(PollEvent::new(fd, pollfd.events())); | ||||
|         } else { | ||||
|             return_errno!(EBADF, "not a supported file type"); | ||||
|         } | ||||
|  | ||||
| @ -16,7 +16,7 @@ use std::ffi::{CStr, CString}; | ||||
| use std::io::{Read, Seek, SeekFrom, Write}; | ||||
| use std::mem::MaybeUninit; | ||||
| use std::ptr; | ||||
| use time::{clockid_t, timespec_t, timeval_t}; | ||||
| use time::{clockid_t, itimerspec_t, timespec_t, timeval_t}; | ||||
| use util::log::{self, LevelFilter}; | ||||
| use util::mem_util::from_user::*; | ||||
| 
 | ||||
| @ -28,8 +28,9 @@ use crate::fs::{ | ||||
|     do_getdents, do_getdents64, do_ioctl, do_lchown, do_link, do_linkat, do_lseek, do_lstat, | ||||
|     do_mkdir, do_mkdirat, do_mount_rootfs, do_open, do_openat, do_pipe, do_pipe2, do_pread, | ||||
|     do_pwrite, do_read, do_readlink, do_readlinkat, do_readv, do_rename, do_renameat, do_rmdir, | ||||
|     do_sendfile, do_stat, do_statfs, do_symlink, do_symlinkat, do_sync, do_truncate, do_unlink, | ||||
|     do_unlinkat, do_write, do_writev, iovec_t, File, FileDesc, FileRef, HostStdioFds, Stat, Statfs, | ||||
|     do_sendfile, do_stat, do_statfs, do_symlink, do_symlinkat, do_sync, do_timerfd_create, | ||||
|     do_timerfd_gettime, do_timerfd_settime, do_truncate, do_unlink, do_unlinkat, do_write, | ||||
|     do_writev, iovec_t, AsTimer, File, FileDesc, FileRef, HostStdioFds, Stat, Statfs, | ||||
| }; | ||||
| use crate::interrupt::{do_handle_interrupt, sgx_interrupt_info_t}; | ||||
| use crate::misc::{resource_t, rlimit_t, sysinfo_t, utsname_t}; | ||||
| @ -368,11 +369,11 @@ macro_rules! process_syscall_table_with_callback { | ||||
|             (Utimensat = 280) => handle_unsupported(), | ||||
|             (EpollPwait = 281) => do_epoll_pwait(epfd: c_int, events: *mut libc::epoll_event, maxevents: c_int, timeout: c_int, sigmask: *const usize), | ||||
|             (Signalfd = 282) => handle_unsupported(), | ||||
|             (TimerfdCreate = 283) => handle_unsupported(), | ||||
|             (TimerfdCreate = 283) => do_timerfd_create(clockid: clockid_t, flags: i32 ), | ||||
|             (Eventfd = 284) => do_eventfd(init_val: u32), | ||||
|             (Fallocate = 285) => do_fallocate(fd: FileDesc, mode: u32, offset: off_t, len: off_t), | ||||
|             (TimerfdSettime = 286) => handle_unsupported(), | ||||
|             (TimerfdGettime = 287) => handle_unsupported(), | ||||
|             (TimerfdSettime = 286) => do_timerfd_settime(fd: FileDesc, flags: i32, new_value: *const itimerspec_t, old_value: *mut itimerspec_t), | ||||
|             (TimerfdGettime = 287) => do_timerfd_gettime(fd: FileDesc, curr_value: *mut itimerspec_t), | ||||
|             (Accept4 = 288) => do_accept4(fd: c_int, addr: *mut libc::sockaddr, addr_len: *mut libc::socklen_t, flags: c_int), | ||||
|             (Signalfd4 = 289) => handle_unsupported(), | ||||
|             (Eventfd2 = 290) => do_eventfd2(init_val: u32, flags: i32), | ||||
|  | ||||
| @ -252,3 +252,25 @@ impl TimeProvider for OcclumTimeProvider { | ||||
|         } | ||||
|     } | ||||
| } | ||||
| 
 | ||||
| // For Timerfd
 | ||||
| #[repr(C)] | ||||
| #[derive(Debug, Default, Copy, Clone)] | ||||
| #[allow(non_camel_case_types)] | ||||
| pub struct itimerspec_t { | ||||
|     it_interval: timespec_t, | ||||
|     it_value: timespec_t, | ||||
| } | ||||
| 
 | ||||
| impl itimerspec_t { | ||||
|     pub fn from_raw_ptr(ptr: *const itimerspec_t) -> Result<itimerspec_t> { | ||||
|         let its = unsafe { *ptr }; | ||||
|         its.validate()?; | ||||
|         Ok(its) | ||||
|     } | ||||
|     pub fn validate(&self) -> Result<()> { | ||||
|         self.it_interval.validate()?; | ||||
|         self.it_value.validate()?; | ||||
|         Ok(()) | ||||
|     } | ||||
| } | ||||
|  | ||||
| @ -1,5 +1,6 @@ | ||||
| #include <pthread.h> | ||||
| #include <sys/time.h> | ||||
| #include <sys/timerfd.h> | ||||
| #include <sys/prctl.h> | ||||
| #include "ocalls.h" | ||||
| 
 | ||||
| @ -41,3 +42,16 @@ void occlum_ocall_get_timerslack(int *timer_slack) { | ||||
|     int nanoseconds = prctl(PR_GET_TIMERSLACK, 0, 0, 0, 0); | ||||
|     *timer_slack = nanoseconds; | ||||
| } | ||||
| 
 | ||||
| int occlum_ocall_timerfd_create(int clockid, int flags) { | ||||
|     return timerfd_create(clockid, flags); | ||||
| } | ||||
| 
 | ||||
| int occlum_ocall_timerfd_settime(int fd, int flags, const struct itimerspec *new_value, | ||||
|                                  struct itimerspec *old_value) { | ||||
|     return timerfd_settime(fd, flags, new_value, old_value); | ||||
| } | ||||
| 
 | ||||
| int occlum_ocall_timerfd_gettime(int fd, struct itimerspec *curr_value) { | ||||
|     return timerfd_gettime(fd, curr_value); | ||||
| } | ||||
| @ -15,7 +15,7 @@ FAIL_LOG = $(BUILD_DIR)/test/.fail | ||||
| # Dependencies: need to be compiled but not to run by any Makefile target
 | ||||
| TEST_DEPS := client data_sink naughty_child | ||||
| # Tests: need to be compiled and run by test-% target
 | ||||
| TESTS ?= env empty hello_world malloc mmap file fs_perms getpid spawn sched pipe time \
 | ||||
| TESTS ?= env empty hello_world malloc mmap file fs_perms getpid spawn sched pipe time timerfd \
 | ||||
| 	truncate readdir mkdir open stat link symlink chmod chown tls pthread system_info rlimit \
 | ||||
| 	server server_epoll unix_socket cout hostfs cpuid rdtsc device sleep exit_group \
 | ||||
| 	ioctl fcntl eventfd emulate_syscall access signal sysinfo prctl rename procfs wait \
 | ||||
|  | ||||
							
								
								
									
										5
									
								
								test/timerfd/Makefile
									
									
									
									
									
										Normal file
									
								
							
							
								
								
								
								
								
									
									
								
							
						
						
									
										5
									
								
								test/timerfd/Makefile
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,5 @@ | ||||
| include ../test_common.mk | ||||
| 
 | ||||
| EXTRA_C_FLAGS := | ||||
| EXTRA_LINK_FLAGS := | ||||
| BIN_ARGS := | ||||
							
								
								
									
										95
									
								
								test/timerfd/main.c
									
									
									
									
									
										Normal file
									
								
							
							
								
								
								
								
								
									
									
								
							
						
						
									
										95
									
								
								test/timerfd/main.c
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,95 @@ | ||||
| #include <stdio.h> | ||||
| #include <sys/timerfd.h> | ||||
| #include <sys/select.h> | ||||
| #include <time.h> | ||||
| #include "test.h" | ||||
| 
 | ||||
| // ============================================================================
 | ||||
| // Test cases for timerfd full process
 | ||||
| // ============================================================================
 | ||||
| int test_timerfd() { | ||||
|     int tfd = timerfd_create(CLOCK_REALTIME,  0); | ||||
| 
 | ||||
|     printf("Starting at (%d)...\n", (int)time(NULL)); | ||||
|     if (tfd <= 0) { | ||||
|         THROW_ERROR("timerfd_create(CLOCK_REALTIME, ...) failed"); | ||||
|     } | ||||
|     char dummybuf[8]; | ||||
|     struct itimerspec spec = { | ||||
|         { 1, 0 }, // Set to {0, 0} if you need a one-shot timer
 | ||||
|         { 2, 0 } | ||||
|     }; | ||||
|     struct itimerspec curr = { | ||||
|         { 0, 0 }, // Set to {0, 0} if you need a one-shot timer
 | ||||
|         { 0, 0 } | ||||
|     }; | ||||
|     if (timerfd_settime(tfd, 0, &spec, NULL)) { | ||||
|         THROW_ERROR("timerfd_settime(...) failed"); | ||||
|     } | ||||
|     if (timerfd_gettime(tfd, &curr)) { | ||||
|         THROW_ERROR("timerfd_gettime(...) failed"); | ||||
|     } | ||||
|     /* Wait */ | ||||
|     fd_set rfds; | ||||
|     int retval; | ||||
| 
 | ||||
|     /* Watch timefd file descriptor */ | ||||
|     FD_ZERO(&rfds); | ||||
|     FD_SET(0, &rfds); | ||||
|     FD_SET(tfd, &rfds); | ||||
| 
 | ||||
|     /* Let's wait for initial timer expiration */ | ||||
|     retval = select(tfd + 1, &rfds, NULL, NULL, | ||||
|                     NULL); /* Last parameter = NULL --> wait forever */ | ||||
|     printf("Expired at %d! (%d) (%ld)\n", (int)time(NULL), retval, read(tfd, dummybuf, 8) ); | ||||
| 
 | ||||
|     /* Let's wait for initial timer expiration */ | ||||
|     retval = select(tfd + 1, &rfds, NULL, NULL, NULL); | ||||
|     printf("Expired at %d! (%d) (%ld)\n", (int)time(NULL), retval, read(tfd, dummybuf, 8) ); | ||||
| 
 | ||||
|     retval = select(tfd + 1, &rfds, NULL, NULL, NULL); | ||||
|     printf("Expired at %d! (%d) (%ld)\n", (int)time(NULL), retval, read(tfd, dummybuf, 8) ); | ||||
|     return 0; | ||||
| } | ||||
| 
 | ||||
| int test_invalid_argument() { | ||||
|     int tfd = timerfd_create(CLOCK_REALTIME,  0); | ||||
|     if (tfd <= 0) { | ||||
|         THROW_ERROR("timerfd_create(CLOCK_REALTIME, ...) failed"); | ||||
|     } | ||||
|     int invalid_clockid = 6; | ||||
|     int invalid_create_flags = 11; | ||||
|     int invalid_settime_flags = 5; | ||||
|     struct itimerspec spec = { | ||||
|         { 1, 0 }, // Set to {0, 0} if you need a one-shot timer
 | ||||
|         { 2, 0 } | ||||
|     }; | ||||
|     /* Test invalid argument */ | ||||
|     int ret = timerfd_create(CLOCK_REALTIME, invalid_create_flags); | ||||
|     if (ret >= 0 || errno != EINVAL ) { | ||||
|         THROW_ERROR("failed to check timerfd_create with invalid flags"); | ||||
|     } | ||||
|     ret = timerfd_create(invalid_clockid,  0); | ||||
|     if (ret >= 0 || errno != EINVAL ) { | ||||
|         THROW_ERROR("failed to check timerfd_create with invalid clockid"); | ||||
|     } | ||||
|     ret = timerfd_settime(tfd, invalid_settime_flags, &spec, NULL); | ||||
|     if (ret >= 0 || errno != EINVAL ) { | ||||
|         THROW_ERROR("failed to check timerfd_settime with invalid flags"); | ||||
|     } | ||||
|     return 0; | ||||
| } | ||||
| 
 | ||||
| // ============================================================================
 | ||||
| // Test suite
 | ||||
| // ============================================================================
 | ||||
| 
 | ||||
| static test_case_t test_cases[] = { | ||||
|     TEST_CASE(test_timerfd), | ||||
|     TEST_CASE(test_invalid_argument), | ||||
| }; | ||||
| 
 | ||||
| int main() { | ||||
|     return test_suite_run(test_cases, ARRAY_SIZE(test_cases)); | ||||
| } | ||||
| 
 | ||||
		Loading…
	
		Reference in New Issue
	
	Block a user