Support timerfd-related syscall
This commit is contained in:
parent
1e7d258989
commit
eb07b01ac0
@ -147,6 +147,10 @@ enclave {
|
|||||||
[out] struct timespec* rem
|
[out] struct timespec* rem
|
||||||
) propagate_errno;
|
) 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_sync(void);
|
||||||
|
|
||||||
void* occlum_ocall_posix_memalign(size_t alignment, size_t size);
|
void* occlum_ocall_posix_memalign(size_t alignment, size_t size);
|
||||||
|
@ -1,8 +1,11 @@
|
|||||||
#ifndef __OCCLUM_EDL_TYPES_H__
|
#ifndef __OCCLUM_EDL_TYPES_H__
|
||||||
#define __OCCLUM_EDL_TYPES_H__
|
#define __OCCLUM_EDL_TYPES_H__
|
||||||
|
|
||||||
|
#include <sys/struct_timespec.h>
|
||||||
|
|
||||||
typedef long time_t;
|
typedef long time_t;
|
||||||
typedef long suseconds_t;
|
typedef long suseconds_t;
|
||||||
|
typedef long syscall_slong_t;
|
||||||
typedef int clockid_t;
|
typedef int clockid_t;
|
||||||
|
|
||||||
struct timeval {
|
struct timeval {
|
||||||
@ -16,6 +19,16 @@ struct occlum_stdio_fds {
|
|||||||
int stderr_fd;
|
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
|
#define FD_SETSIZE 1024
|
||||||
typedef struct {
|
typedef struct {
|
||||||
unsigned long fds_bits[FD_SETSIZE / 8 / sizeof(long)];
|
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::rootfs::ROOT_INODE;
|
||||||
pub use self::stdio::{HostStdioFds, StdinFile, StdoutFile};
|
pub use self::stdio::{HostStdioFds, StdinFile, StdoutFile};
|
||||||
pub use self::syscalls::*;
|
pub use self::syscalls::*;
|
||||||
|
pub use self::timer_file::{AsTimer, TimerCreationFlags, TimerFile};
|
||||||
|
|
||||||
pub mod channel;
|
pub mod channel;
|
||||||
mod dev_fs;
|
mod dev_fs;
|
||||||
@ -50,6 +51,7 @@ mod rootfs;
|
|||||||
mod sefs;
|
mod sefs;
|
||||||
mod stdio;
|
mod stdio;
|
||||||
mod syscalls;
|
mod syscalls;
|
||||||
|
mod timer_file;
|
||||||
|
|
||||||
/// Split a `path` str to `(base_path, file_name)`
|
/// Split a `path` str to `(base_path, file_name)`
|
||||||
fn split_path(path: &str) -> (&str, &str) {
|
fn split_path(path: &str) -> (&str, &str) {
|
||||||
|
@ -5,6 +5,8 @@ use super::file_ops::{
|
|||||||
StatFlags, UnlinkFlags, AT_FDCWD,
|
StatFlags, UnlinkFlags, AT_FDCWD,
|
||||||
};
|
};
|
||||||
use super::fs_ops;
|
use super::fs_ops;
|
||||||
|
use super::time::{clockid_t, itimerspec_t, ClockID};
|
||||||
|
use super::timer_file::{TimerCreationFlags, TimerSetFlags};
|
||||||
use super::*;
|
use super::*;
|
||||||
use util::mem_util::from_user;
|
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)
|
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> {
|
pub fn do_open(path: *const i8, flags: u32, mode: u32) -> Result<isize> {
|
||||||
self::do_openat(AT_FDCWD, path, flags, mode)
|
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::poll_new::{do_poll_new, PollFd};
|
||||||
pub use self::select::{do_select, FdSetExt};
|
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::any::Any;
|
||||||
use std::convert::TryFrom;
|
use std::convert::TryFrom;
|
||||||
use std::fmt;
|
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;
|
let fd = eventfd.host_fd() as FileDesc;
|
||||||
index_host_pollfds.push(i);
|
index_host_pollfds.push(i);
|
||||||
host_pollfds.push(PollEvent::new(fd, pollfd.events()));
|
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 {
|
} else {
|
||||||
return_errno!(EBADF, "not a supported file type");
|
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::io::{Read, Seek, SeekFrom, Write};
|
||||||
use std::mem::MaybeUninit;
|
use std::mem::MaybeUninit;
|
||||||
use std::ptr;
|
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::log::{self, LevelFilter};
|
||||||
use util::mem_util::from_user::*;
|
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_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_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_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_sendfile, do_stat, do_statfs, do_symlink, do_symlinkat, do_sync, do_timerfd_create,
|
||||||
do_unlinkat, do_write, do_writev, iovec_t, File, FileDesc, FileRef, HostStdioFds, Stat, Statfs,
|
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::interrupt::{do_handle_interrupt, sgx_interrupt_info_t};
|
||||||
use crate::misc::{resource_t, rlimit_t, sysinfo_t, utsname_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(),
|
(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),
|
(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(),
|
(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),
|
(Eventfd = 284) => do_eventfd(init_val: u32),
|
||||||
(Fallocate = 285) => do_fallocate(fd: FileDesc, mode: u32, offset: off_t, len: off_t),
|
(Fallocate = 285) => do_fallocate(fd: FileDesc, mode: u32, offset: off_t, len: off_t),
|
||||||
(TimerfdSettime = 286) => handle_unsupported(),
|
(TimerfdSettime = 286) => do_timerfd_settime(fd: FileDesc, flags: i32, new_value: *const itimerspec_t, old_value: *mut itimerspec_t),
|
||||||
(TimerfdGettime = 287) => handle_unsupported(),
|
(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),
|
(Accept4 = 288) => do_accept4(fd: c_int, addr: *mut libc::sockaddr, addr_len: *mut libc::socklen_t, flags: c_int),
|
||||||
(Signalfd4 = 289) => handle_unsupported(),
|
(Signalfd4 = 289) => handle_unsupported(),
|
||||||
(Eventfd2 = 290) => do_eventfd2(init_val: u32, flags: i32),
|
(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 <pthread.h>
|
||||||
#include <sys/time.h>
|
#include <sys/time.h>
|
||||||
|
#include <sys/timerfd.h>
|
||||||
#include <sys/prctl.h>
|
#include <sys/prctl.h>
|
||||||
#include "ocalls.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);
|
int nanoseconds = prctl(PR_GET_TIMERSLACK, 0, 0, 0, 0);
|
||||||
*timer_slack = nanoseconds;
|
*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
|
# Dependencies: need to be compiled but not to run by any Makefile target
|
||||||
TEST_DEPS := client data_sink naughty_child
|
TEST_DEPS := client data_sink naughty_child
|
||||||
# Tests: need to be compiled and run by test-% target
|
# 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 \
|
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 \
|
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 \
|
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