summaryrefslogtreecommitdiff
path: root/ext/io
diff options
context:
space:
mode:
Diffstat (limited to 'ext/io')
-rw-r--r--ext/io/Cargo.toml3
-rw-r--r--ext/io/fs.rs330
-rw-r--r--ext/io/lib.rs623
3 files changed, 676 insertions, 280 deletions
diff --git a/ext/io/Cargo.toml b/ext/io/Cargo.toml
index 55b2ccab6..fc9de711f 100644
--- a/ext/io/Cargo.toml
+++ b/ext/io/Cargo.toml
@@ -14,7 +14,10 @@ description = "IO promitives for Deno extensions"
path = "lib.rs"
[dependencies]
+async-trait.workspace = true
deno_core.workspace = true
+filetime.workspace = true
+fs3.workspace = true
once_cell.workspace = true
tokio.workspace = true
diff --git a/ext/io/fs.rs b/ext/io/fs.rs
new file mode 100644
index 000000000..bb6bdec4f
--- /dev/null
+++ b/ext/io/fs.rs
@@ -0,0 +1,330 @@
+// Copyright 2018-2023 the Deno authors. All rights reserved. MIT license.
+
+use std::borrow::Cow;
+use std::io;
+use std::rc::Rc;
+use std::time::SystemTime;
+use std::time::UNIX_EPOCH;
+
+use deno_core::error::not_supported;
+use deno_core::error::resource_unavailable;
+use deno_core::error::AnyError;
+use deno_core::BufMutView;
+use deno_core::BufView;
+use deno_core::OpState;
+use deno_core::ResourceId;
+use tokio::task::JoinError;
+
+pub enum FsError {
+ Io(io::Error),
+ FileBusy,
+ NotSupported,
+}
+
+impl From<io::Error> for FsError {
+ fn from(err: io::Error) -> Self {
+ Self::Io(err)
+ }
+}
+
+impl From<FsError> for AnyError {
+ fn from(err: FsError) -> Self {
+ match err {
+ FsError::Io(err) => AnyError::from(err),
+ FsError::FileBusy => resource_unavailable(),
+ FsError::NotSupported => not_supported(),
+ }
+ }
+}
+
+impl From<JoinError> for FsError {
+ fn from(err: JoinError) -> Self {
+ if err.is_cancelled() {
+ todo!("async tasks must not be cancelled")
+ }
+ if err.is_panic() {
+ std::panic::resume_unwind(err.into_panic()); // resume the panic on the main thread
+ }
+ unreachable!()
+ }
+}
+
+pub type FsResult<T> = Result<T, FsError>;
+
+pub struct FsStat {
+ pub is_file: bool,
+ pub is_directory: bool,
+ pub is_symlink: bool,
+ pub size: u64,
+
+ pub mtime: Option<u64>,
+ pub atime: Option<u64>,
+ pub birthtime: Option<u64>,
+
+ pub dev: u64,
+ pub ino: u64,
+ pub mode: u32,
+ pub nlink: u64,
+ pub uid: u32,
+ pub gid: u32,
+ pub rdev: u64,
+ pub blksize: u64,
+ pub blocks: u64,
+}
+
+impl FsStat {
+ pub fn from_std(metadata: std::fs::Metadata) -> Self {
+ macro_rules! unix_or_zero {
+ ($member:ident) => {{
+ #[cfg(unix)]
+ {
+ use std::os::unix::fs::MetadataExt;
+ metadata.$member()
+ }
+ #[cfg(not(unix))]
+ {
+ 0
+ }
+ }};
+ }
+
+ #[inline(always)]
+ fn to_msec(maybe_time: Result<SystemTime, io::Error>) -> Option<u64> {
+ match maybe_time {
+ Ok(time) => Some(
+ time
+ .duration_since(UNIX_EPOCH)
+ .map(|t| t.as_millis() as u64)
+ .unwrap_or_else(|err| err.duration().as_millis() as u64),
+ ),
+ Err(_) => None,
+ }
+ }
+
+ Self {
+ is_file: metadata.is_file(),
+ is_directory: metadata.is_dir(),
+ is_symlink: metadata.file_type().is_symlink(),
+ size: metadata.len(),
+
+ mtime: to_msec(metadata.modified()),
+ atime: to_msec(metadata.accessed()),
+ birthtime: to_msec(metadata.created()),
+
+ dev: unix_or_zero!(dev),
+ ino: unix_or_zero!(ino),
+ mode: unix_or_zero!(mode),
+ nlink: unix_or_zero!(nlink),
+ uid: unix_or_zero!(uid),
+ gid: unix_or_zero!(gid),
+ rdev: unix_or_zero!(rdev),
+ blksize: unix_or_zero!(blksize),
+ blocks: unix_or_zero!(blocks),
+ }
+ }
+}
+
+#[async_trait::async_trait(?Send)]
+pub trait File {
+ fn read_sync(self: Rc<Self>, buf: &mut [u8]) -> FsResult<usize>;
+ async fn read(self: Rc<Self>, limit: usize) -> FsResult<BufView> {
+ let vec = vec![0; limit];
+ let buf = BufMutView::from(vec);
+ let (nread, buf) = self.read_byob(buf).await?;
+ let mut vec = buf.unwrap_vec();
+ if vec.len() != nread {
+ vec.truncate(nread);
+ }
+ Ok(BufView::from(vec))
+ }
+ async fn read_byob(
+ self: Rc<Self>,
+ buf: BufMutView,
+ ) -> FsResult<(usize, BufMutView)>;
+
+ fn write_sync(self: Rc<Self>, buf: &[u8]) -> FsResult<usize>;
+ async fn write(
+ self: Rc<Self>,
+ buf: BufView,
+ ) -> FsResult<deno_core::WriteOutcome>;
+
+ fn write_all_sync(self: Rc<Self>, buf: &[u8]) -> FsResult<()>;
+ async fn write_all(self: Rc<Self>, buf: BufView) -> FsResult<()>;
+
+ fn read_all_sync(self: Rc<Self>) -> FsResult<Vec<u8>>;
+ async fn read_all_async(self: Rc<Self>) -> FsResult<Vec<u8>>;
+
+ fn chmod_sync(self: Rc<Self>, pathmode: u32) -> FsResult<()>;
+ async fn chmod_async(self: Rc<Self>, mode: u32) -> FsResult<()>;
+
+ fn seek_sync(self: Rc<Self>, pos: io::SeekFrom) -> FsResult<u64>;
+ async fn seek_async(self: Rc<Self>, pos: io::SeekFrom) -> FsResult<u64>;
+
+ fn datasync_sync(self: Rc<Self>) -> FsResult<()>;
+ async fn datasync_async(self: Rc<Self>) -> FsResult<()>;
+
+ fn sync_sync(self: Rc<Self>) -> FsResult<()>;
+ async fn sync_async(self: Rc<Self>) -> FsResult<()>;
+
+ fn stat_sync(self: Rc<Self>) -> FsResult<FsStat>;
+ async fn stat_async(self: Rc<Self>) -> FsResult<FsStat>;
+
+ fn lock_sync(self: Rc<Self>, exclusive: bool) -> FsResult<()>;
+ async fn lock_async(self: Rc<Self>, exclusive: bool) -> FsResult<()>;
+
+ fn unlock_sync(self: Rc<Self>) -> FsResult<()>;
+ async fn unlock_async(self: Rc<Self>) -> FsResult<()>;
+
+ fn truncate_sync(self: Rc<Self>, len: u64) -> FsResult<()>;
+ async fn truncate_async(self: Rc<Self>, len: u64) -> FsResult<()>;
+
+ fn utime_sync(
+ self: Rc<Self>,
+ atime_secs: i64,
+ atime_nanos: u32,
+ mtime_secs: i64,
+ mtime_nanos: u32,
+ ) -> FsResult<()>;
+ async fn utime_async(
+ self: Rc<Self>,
+ atime_secs: i64,
+ atime_nanos: u32,
+ mtime_secs: i64,
+ mtime_nanos: u32,
+ ) -> FsResult<()>;
+
+ // lower level functionality
+ fn as_stdio(self: Rc<Self>) -> FsResult<std::process::Stdio>;
+ #[cfg(unix)]
+ fn backing_fd(self: Rc<Self>) -> Option<std::os::unix::prelude::RawFd>;
+ #[cfg(windows)]
+ fn backing_fd(self: Rc<Self>) -> Option<std::os::windows::io::RawHandle>;
+ fn try_clone_inner(self: Rc<Self>) -> FsResult<Rc<dyn File>>;
+}
+
+pub struct FileResource {
+ name: String,
+ file: Rc<dyn File>,
+}
+
+impl FileResource {
+ pub fn new(file: Rc<dyn File>, name: String) -> Self {
+ Self { name, file }
+ }
+
+ pub fn with_resource<F, R>(
+ state: &OpState,
+ rid: ResourceId,
+ f: F,
+ ) -> Result<R, AnyError>
+ where
+ F: FnOnce(Rc<FileResource>) -> Result<R, AnyError>,
+ {
+ let resource = state.resource_table.get::<FileResource>(rid)?;
+ f(resource)
+ }
+
+ pub fn get_file(
+ state: &OpState,
+ rid: ResourceId,
+ ) -> Result<Rc<dyn File>, AnyError> {
+ let resource = state.resource_table.get::<FileResource>(rid)?;
+ Ok(resource.file())
+ }
+
+ pub fn with_file<F, R>(
+ state: &OpState,
+ rid: ResourceId,
+ f: F,
+ ) -> Result<R, AnyError>
+ where
+ F: FnOnce(Rc<dyn File>) -> Result<R, AnyError>,
+ {
+ Self::with_resource(state, rid, |r| f(r.file.clone()))
+ }
+
+ pub fn file(&self) -> Rc<dyn File> {
+ self.file.clone()
+ }
+}
+
+impl deno_core::Resource for FileResource {
+ fn name(&self) -> Cow<str> {
+ Cow::Borrowed(&self.name)
+ }
+
+ fn read(
+ self: Rc<Self>,
+ limit: usize,
+ ) -> deno_core::AsyncResult<deno_core::BufView> {
+ Box::pin(async move {
+ self
+ .file
+ .clone()
+ .read(limit)
+ .await
+ .map_err(|err| err.into())
+ })
+ }
+
+ fn read_byob(
+ self: Rc<Self>,
+ buf: deno_core::BufMutView,
+ ) -> deno_core::AsyncResult<(usize, deno_core::BufMutView)> {
+ Box::pin(async move {
+ self
+ .file
+ .clone()
+ .read_byob(buf)
+ .await
+ .map_err(|err| err.into())
+ })
+ }
+
+ fn write(
+ self: Rc<Self>,
+ buf: deno_core::BufView,
+ ) -> deno_core::AsyncResult<deno_core::WriteOutcome> {
+ Box::pin(async move {
+ self.file.clone().write(buf).await.map_err(|err| err.into())
+ })
+ }
+
+ fn write_all(
+ self: Rc<Self>,
+ buf: deno_core::BufView,
+ ) -> deno_core::AsyncResult<()> {
+ Box::pin(async move {
+ self
+ .file
+ .clone()
+ .write_all(buf)
+ .await
+ .map_err(|err| err.into())
+ })
+ }
+
+ fn read_byob_sync(
+ self: Rc<Self>,
+ data: &mut [u8],
+ ) -> Result<usize, deno_core::anyhow::Error> {
+ self.file.clone().read_sync(data).map_err(|err| err.into())
+ }
+
+ fn write_sync(
+ self: Rc<Self>,
+ data: &[u8],
+ ) -> Result<usize, deno_core::anyhow::Error> {
+ self.file.clone().write_sync(data).map_err(|err| err.into())
+ }
+
+ #[cfg(unix)]
+ fn backing_fd(self: Rc<Self>) -> Option<std::os::unix::prelude::RawFd> {
+ self.file.clone().backing_fd()
+ }
+
+ #[cfg(windows)]
+ fn backing_fd(self: Rc<Self>) -> Option<std::os::windows::io::RawHandle> {
+ self.file.clone().backing_fd()
+ }
+}
diff --git a/ext/io/lib.rs b/ext/io/lib.rs
index 73ce72578..49e4ab714 100644
--- a/ext/io/lib.rs
+++ b/ext/io/lib.rs
@@ -1,6 +1,5 @@
// Copyright 2018-2023 the Deno authors. All rights reserved. MIT license.
-use deno_core::error::resource_unavailable;
use deno_core::error::AnyError;
use deno_core::op;
use deno_core::AsyncMutFuture;
@@ -13,8 +12,12 @@ use deno_core::CancelTryFuture;
use deno_core::OpState;
use deno_core::RcRef;
use deno_core::Resource;
-use deno_core::ResourceId;
use deno_core::TaskQueue;
+use fs::FileResource;
+use fs::FsError;
+use fs::FsResult;
+use fs::FsStat;
+use fs3::FileExt;
use once_cell::sync::Lazy;
use std::borrow::Cow;
use std::cell::RefCell;
@@ -22,6 +25,7 @@ use std::fs::File as StdFile;
use std::io;
use std::io::ErrorKind;
use std::io::Read;
+use std::io::Seek;
use std::io::Write;
use std::rc::Rc;
use tokio::io::AsyncRead;
@@ -40,6 +44,8 @@ use winapi::um::processenv::GetStdHandle;
#[cfg(windows)]
use winapi::um::winbase;
+pub mod fs;
+
// Store the stdio fd/handles in global statics in order to keep them
// alive for the duration of the application since the last handle/fd
// being dropped will close the corresponding pipe.
@@ -89,39 +95,39 @@ deno_core::extension!(deno_io,
if let Some(stdio) = options.stdio {
let t = &mut state.resource_table;
- let rid = t.add(StdFileResource::stdio(
- match stdio.stdin {
- StdioPipe::Inherit => StdFileResourceInner {
- kind: StdFileResourceKind::Stdin,
- file: STDIN_HANDLE.try_clone().unwrap(),
- },
+ let rid = t.add(fs::FileResource::new(
+ Rc::new(match stdio.stdin {
+ StdioPipe::Inherit => StdFileResourceInner::new(
+ StdFileResourceKind::Stdin,
+ STDIN_HANDLE.try_clone().unwrap(),
+ ),
StdioPipe::File(pipe) => StdFileResourceInner::file(pipe),
- },
- "stdin",
+ }),
+ "stdin".to_string(),
));
assert_eq!(rid, 0, "stdin must have ResourceId 0");
- let rid = t.add(StdFileResource::stdio(
- match stdio.stdout {
- StdioPipe::Inherit => StdFileResourceInner {
- kind: StdFileResourceKind::Stdout,
- file: STDOUT_HANDLE.try_clone().unwrap(),
- },
+ let rid = t.add(FileResource::new(
+ Rc::new(match stdio.stdout {
+ StdioPipe::Inherit => StdFileResourceInner::new(
+ StdFileResourceKind::Stdout,
+ STDOUT_HANDLE.try_clone().unwrap(),
+ ),
StdioPipe::File(pipe) => StdFileResourceInner::file(pipe),
- },
- "stdout",
+ }),
+ "stdout".to_string(),
));
assert_eq!(rid, 1, "stdout must have ResourceId 1");
- let rid = t.add(StdFileResource::stdio(
- match stdio.stderr {
- StdioPipe::Inherit => StdFileResourceInner {
- kind: StdFileResourceKind::Stderr,
- file: STDERR_HANDLE.try_clone().unwrap(),
- },
+ let rid = t.add(FileResource::new(
+ Rc::new(match stdio.stderr {
+ StdioPipe::Inherit => StdFileResourceInner::new(
+ StdFileResourceKind::Stderr,
+ STDERR_HANDLE.try_clone().unwrap(),
+ ),
StdioPipe::File(pipe) => StdFileResourceInner::file(pipe),
- },
- "stderr",
+ }),
+ "stderr".to_string(),
));
assert_eq!(rid, 2, "stderr must have ResourceId 2");
}
@@ -291,34 +297,88 @@ enum StdFileResourceKind {
Stderr,
}
-struct StdFileResourceInner {
+pub struct StdFileResourceInner {
kind: StdFileResourceKind,
- file: StdFile,
+ // We can't use an AsyncRefCell here because we need to allow
+ // access to the resource synchronously at any time and
+ // asynchronously one at a time in order
+ cell: RefCell<Option<StdFile>>,
+ // Used to keep async actions in order and only allow one
+ // to occur at a time
+ cell_async_task_queue: TaskQueue,
}
impl StdFileResourceInner {
pub fn file(fs_file: StdFile) -> Self {
+ StdFileResourceInner::new(StdFileResourceKind::File, fs_file)
+ }
+
+ fn new(kind: StdFileResourceKind, fs_file: StdFile) -> Self {
StdFileResourceInner {
- kind: StdFileResourceKind::File,
- file: fs_file,
+ kind,
+ cell: RefCell::new(Some(fs_file)),
+ cell_async_task_queue: Default::default(),
}
}
- pub fn with_file<R>(&mut self, f: impl FnOnce(&mut StdFile) -> R) -> R {
- f(&mut self.file)
+ fn with_sync<F, R>(&self, action: F) -> FsResult<R>
+ where
+ F: FnOnce(&mut StdFile) -> FsResult<R>,
+ {
+ match self.cell.try_borrow_mut() {
+ Ok(mut cell) if cell.is_some() => action(cell.as_mut().unwrap()),
+ _ => Err(fs::FsError::FileBusy),
+ }
}
- pub fn try_clone(&self) -> Result<Self, std::io::Error> {
- Ok(Self {
- kind: self.kind,
- file: self.file.try_clone()?,
+ async fn with_inner_blocking_task<F, R: 'static + Send>(&self, action: F) -> R
+ where
+ F: FnOnce(&mut StdFile) -> R + Send + 'static,
+ {
+ // we want to restrict this to one async action at a time
+ let _permit = self.cell_async_task_queue.acquire().await;
+ // we take the value out of the cell, use it on a blocking task,
+ // then put it back into the cell when we're done
+ let mut did_take = false;
+ let mut cell_value = {
+ let mut cell = self.cell.borrow_mut();
+ match cell.as_mut().unwrap().try_clone().ok() {
+ Some(value) => value,
+ None => {
+ did_take = true;
+ cell.take().unwrap()
+ }
+ }
+ };
+ let (cell_value, result) = tokio::task::spawn_blocking(move || {
+ let result = action(&mut cell_value);
+ (cell_value, result)
})
+ .await
+ .unwrap();
+
+ if did_take {
+ // put it back
+ self.cell.borrow_mut().replace(cell_value);
+ }
+
+ result
}
- pub fn write_and_maybe_flush(
- &mut self,
- buf: &[u8],
- ) -> Result<usize, AnyError> {
+ async fn with_blocking_task<F, R: 'static + Send>(&self, action: F) -> R
+ where
+ F: FnOnce() -> R + Send + 'static,
+ {
+ // we want to restrict this to one async action at a time
+ let _permit = self.cell_async_task_queue.acquire().await;
+
+ tokio::task::spawn_blocking(action).await.unwrap()
+ }
+}
+
+#[async_trait::async_trait(?Send)]
+impl crate::fs::File for StdFileResourceInner {
+ fn write_sync(self: Rc<Self>, buf: &[u8]) -> FsResult<usize> {
// Rust will line buffer and we don't want that behavior
// (see https://github.com/denoland/deno/issues/948), so flush stdout and stderr.
// Although an alternative solution could be to bypass Rust's std by
@@ -326,7 +386,7 @@ impl StdFileResourceInner {
// that we get solved for free by using Rust's stdio wrappers (see
// std/src/sys/windows/stdio.rs in Rust's source code).
match self.kind {
- StdFileResourceKind::File => Ok(self.file.write(buf)?),
+ StdFileResourceKind::File => self.with_sync(|file| Ok(file.write(buf)?)),
StdFileResourceKind::Stdin => {
Err(Into::<std::io::Error>::into(ErrorKind::Unsupported).into())
}
@@ -347,14 +407,22 @@ impl StdFileResourceInner {
}
}
- pub fn write_all_and_maybe_flush(
- &mut self,
- buf: &[u8],
- ) -> Result<(), AnyError> {
- // this method exists instead of using a `Write` implementation
- // so that we can acquire the locks once and do both actions
+ fn read_sync(self: Rc<Self>, buf: &mut [u8]) -> FsResult<usize> {
+ match self.kind {
+ StdFileResourceKind::File | StdFileResourceKind::Stdin => {
+ self.with_sync(|file| Ok(file.read(buf)?))
+ }
+ StdFileResourceKind::Stdout | StdFileResourceKind::Stderr => {
+ Err(FsError::NotSupported)
+ }
+ }
+ }
+
+ fn write_all_sync(self: Rc<Self>, buf: &[u8]) -> FsResult<()> {
match self.kind {
- StdFileResourceKind::File => Ok(self.file.write_all(buf)?),
+ StdFileResourceKind::File => {
+ self.with_sync(|file| Ok(file.write_all(buf)?))
+ }
StdFileResourceKind::Stdin => {
Err(Into::<std::io::Error>::into(ErrorKind::Unsupported).into())
}
@@ -374,292 +442,292 @@ impl StdFileResourceInner {
}
}
}
-}
-
-impl Read for StdFileResourceInner {
- fn read(&mut self, buf: &mut [u8]) -> std::io::Result<usize> {
+ async fn write_all(self: Rc<Self>, buf: BufView) -> FsResult<()> {
match self.kind {
- StdFileResourceKind::File | StdFileResourceKind::Stdin => {
- self.file.read(buf)
+ StdFileResourceKind::File => {
+ self
+ .with_inner_blocking_task(move |file| Ok(file.write_all(&buf)?))
+ .await
}
- StdFileResourceKind::Stdout | StdFileResourceKind::Stderr => {
- Err(ErrorKind::Unsupported.into())
+ StdFileResourceKind::Stdin => {
+ Err(Into::<std::io::Error>::into(ErrorKind::Unsupported).into())
+ }
+ StdFileResourceKind::Stdout => {
+ self
+ .with_blocking_task(move || {
+ // bypass the file and use std::io::stdout()
+ let mut stdout = std::io::stdout().lock();
+ stdout.write_all(&buf)?;
+ stdout.flush()?;
+ Ok(())
+ })
+ .await
+ }
+ StdFileResourceKind::Stderr => {
+ self
+ .with_blocking_task(move || {
+ // bypass the file and use std::io::stderr()
+ let mut stderr = std::io::stderr().lock();
+ stderr.write_all(&buf)?;
+ stderr.flush()?;
+ Ok(())
+ })
+ .await
}
}
}
-}
-pub struct StdFileResource {
- name: String,
- // We can't use an AsyncRefCell here because we need to allow
- // access to the resource synchronously at any time and
- // asynchronously one at a time in order
- cell: RefCell<Option<StdFileResourceInner>>,
- // Used to keep async actions in order and only allow one
- // to occur at a time
- cell_async_task_queue: TaskQueue,
-}
-
-impl StdFileResource {
- fn stdio(inner: StdFileResourceInner, name: &str) -> Self {
- Self {
- cell: RefCell::new(Some(inner)),
- cell_async_task_queue: Default::default(),
- name: name.to_string(),
+ async fn write(
+ self: Rc<Self>,
+ view: BufView,
+ ) -> FsResult<deno_core::WriteOutcome> {
+ match self.kind {
+ StdFileResourceKind::File => {
+ self
+ .with_inner_blocking_task(|file| {
+ let nwritten = file.write(&view)?;
+ Ok(deno_core::WriteOutcome::Partial { nwritten, view })
+ })
+ .await
+ }
+ StdFileResourceKind::Stdin => {
+ Err(Into::<std::io::Error>::into(ErrorKind::Unsupported).into())
+ }
+ StdFileResourceKind::Stdout => {
+ self
+ .with_blocking_task(|| {
+ // bypass the file and use std::io::stdout()
+ let mut stdout = std::io::stdout().lock();
+ let nwritten = stdout.write(&view)?;
+ stdout.flush()?;
+ Ok(deno_core::WriteOutcome::Partial { nwritten, view })
+ })
+ .await
+ }
+ StdFileResourceKind::Stderr => {
+ self
+ .with_blocking_task(|| {
+ // bypass the file and use std::io::stderr()
+ let mut stderr = std::io::stderr().lock();
+ let nwritten = stderr.write(&view)?;
+ stderr.flush()?;
+ Ok(deno_core::WriteOutcome::Partial { nwritten, view })
+ })
+ .await
+ }
}
}
- pub fn fs_file(fs_file: StdFile) -> Self {
- Self {
- cell: RefCell::new(Some(StdFileResourceInner::file(fs_file))),
- cell_async_task_queue: Default::default(),
- name: "fsFile".to_string(),
+ fn read_all_sync(self: Rc<Self>) -> FsResult<Vec<u8>> {
+ match self.kind {
+ StdFileResourceKind::File | StdFileResourceKind::Stdin => {
+ let mut buf = Vec::new();
+ self.with_sync(|file| Ok(file.read_to_end(&mut buf)?))?;
+ Ok(buf)
+ }
+ StdFileResourceKind::Stdout | StdFileResourceKind::Stderr => {
+ Err(FsError::NotSupported)
+ }
}
}
-
- fn with_inner<TResult, E>(
- &self,
- action: impl FnOnce(&mut StdFileResourceInner) -> Result<TResult, E>,
- ) -> Option<Result<TResult, E>> {
- match self.cell.try_borrow_mut() {
- Ok(mut cell) if cell.is_some() => {
- let mut file = cell.take().unwrap();
- let result = action(&mut file);
- cell.replace(file);
- Some(result)
+ async fn read_all_async(self: Rc<Self>) -> FsResult<Vec<u8>> {
+ match self.kind {
+ StdFileResourceKind::File | StdFileResourceKind::Stdin => {
+ self
+ .with_inner_blocking_task(|file| {
+ let mut buf = Vec::new();
+ file.read_to_end(&mut buf)?;
+ Ok(buf)
+ })
+ .await
+ }
+ StdFileResourceKind::Stdout | StdFileResourceKind::Stderr => {
+ Err(FsError::NotSupported)
}
- _ => None,
}
}
- async fn with_inner_blocking_task<F, R: Send + 'static>(&self, action: F) -> R
- where
- F: FnOnce(&mut StdFileResourceInner) -> R + Send + 'static,
- {
- // we want to restrict this to one async action at a time
- let _permit = self.cell_async_task_queue.acquire().await;
- // we take the value out of the cell, use it on a blocking task,
- // then put it back into the cell when we're done
- let mut did_take = false;
- let mut cell_value = {
- let mut cell = self.cell.borrow_mut();
- match cell.as_mut().unwrap().try_clone() {
- Ok(value) => value,
- Err(_) => {
- did_take = true;
- cell.take().unwrap()
- }
- }
- };
- let (cell_value, result) = tokio::task::spawn_blocking(move || {
- let result = action(&mut cell_value);
- (cell_value, result)
- })
- .await
- .unwrap();
-
- if did_take {
- // put it back
- self.cell.borrow_mut().replace(cell_value);
+ fn chmod_sync(self: Rc<Self>, _mode: u32) -> FsResult<()> {
+ #[cfg(unix)]
+ {
+ use std::os::unix::prelude::PermissionsExt;
+ self.with_sync(|file| {
+ Ok(file.set_permissions(std::fs::Permissions::from_mode(_mode))?)
+ })
}
-
- result
+ #[cfg(not(unix))]
+ Err(FsError::NotSupported)
+ }
+ async fn chmod_async(self: Rc<Self>, _mode: u32) -> FsResult<()> {
+ #[cfg(unix)]
+ {
+ use std::os::unix::prelude::PermissionsExt;
+ self
+ .with_inner_blocking_task(move |file| {
+ Ok(file.set_permissions(std::fs::Permissions::from_mode(_mode))?)
+ })
+ .await
+ }
+ #[cfg(not(unix))]
+ Err(FsError::NotSupported)
}
- async fn read_byob(
- self: Rc<Self>,
- mut buf: BufMutView,
- ) -> Result<(usize, BufMutView), AnyError> {
+ fn seek_sync(self: Rc<Self>, pos: io::SeekFrom) -> FsResult<u64> {
+ self.with_sync(|file| Ok(file.seek(pos)?))
+ }
+ async fn seek_async(self: Rc<Self>, pos: io::SeekFrom) -> FsResult<u64> {
self
- .with_inner_blocking_task(move |inner| {
- let nread = inner.read(&mut buf)?;
- Ok((nread, buf))
- })
+ .with_inner_blocking_task(move |file| Ok(file.seek(pos)?))
.await
}
- async fn write(
- self: Rc<Self>,
- view: BufView,
- ) -> Result<deno_core::WriteOutcome, AnyError> {
+ fn datasync_sync(self: Rc<Self>) -> FsResult<()> {
+ self.with_sync(|file| Ok(file.sync_data()?))
+ }
+ async fn datasync_async(self: Rc<Self>) -> FsResult<()> {
self
- .with_inner_blocking_task(move |inner| {
- let nwritten = inner.write_and_maybe_flush(&view)?;
- Ok(deno_core::WriteOutcome::Partial { nwritten, view })
- })
+ .with_inner_blocking_task(|file| Ok(file.sync_data()?))
.await
}
- async fn write_all(self: Rc<Self>, view: BufView) -> Result<(), AnyError> {
+ fn sync_sync(self: Rc<Self>) -> FsResult<()> {
+ self.with_sync(|file| Ok(file.sync_all()?))
+ }
+ async fn sync_async(self: Rc<Self>) -> FsResult<()> {
self
- .with_inner_blocking_task(move |inner| {
- inner.write_all_and_maybe_flush(&view)
- })
+ .with_inner_blocking_task(|file| Ok(file.sync_all()?))
.await
}
- fn read_byob_sync(self: Rc<Self>, buf: &mut [u8]) -> Result<usize, AnyError> {
- self
- .with_inner(|inner| inner.read(buf))
- .ok_or_else(resource_unavailable)?
- .map_err(Into::into)
+ fn stat_sync(self: Rc<Self>) -> FsResult<FsStat> {
+ self.with_sync(|file| Ok(file.metadata().map(FsStat::from_std)?))
}
-
- fn write_sync(self: Rc<Self>, data: &[u8]) -> Result<usize, AnyError> {
+ async fn stat_async(self: Rc<Self>) -> FsResult<FsStat> {
self
- .with_inner(|inner| inner.write_and_maybe_flush(data))
- .ok_or_else(resource_unavailable)?
- }
-
- fn with_resource<F, R>(
- state: &mut OpState,
- rid: ResourceId,
- f: F,
- ) -> Result<R, AnyError>
- where
- F: FnOnce(Rc<StdFileResource>) -> Result<R, AnyError>,
- {
- let resource = state.resource_table.get::<StdFileResource>(rid)?;
- f(resource)
+ .with_inner_blocking_task(|file| {
+ Ok(file.metadata().map(FsStat::from_std)?)
+ })
+ .await
}
- pub fn with_file<F, R>(
- state: &mut OpState,
- rid: ResourceId,
- f: F,
- ) -> Result<R, AnyError>
- where
- F: FnOnce(&mut StdFile) -> Result<R, AnyError>,
- {
- Self::with_resource(state, rid, move |resource| {
- resource
- .with_inner(move |inner| inner.with_file(f))
- .ok_or_else(resource_unavailable)?
+ fn lock_sync(self: Rc<Self>, exclusive: bool) -> FsResult<()> {
+ self.with_sync(|file| {
+ if exclusive {
+ file.lock_exclusive()?;
+ } else {
+ file.lock_shared()?;
+ }
+ Ok(())
})
}
-
- pub fn with_file2<F, R>(self: Rc<Self>, f: F) -> Option<Result<R, io::Error>>
- where
- F: FnOnce(&mut StdFile) -> Result<R, io::Error>,
- {
- self.with_inner(move |inner| inner.with_file(f))
- }
-
- pub async fn with_file_blocking_task<F, R: Send + 'static>(
- state: Rc<RefCell<OpState>>,
- rid: ResourceId,
- f: F,
- ) -> Result<R, AnyError>
- where
- F: (FnOnce(&mut StdFile) -> Result<R, AnyError>) + Send + 'static,
- {
- let resource = state
- .borrow_mut()
- .resource_table
- .get::<StdFileResource>(rid)?;
-
- resource
- .with_inner_blocking_task(move |inner| inner.with_file(f))
+ async fn lock_async(self: Rc<Self>, exclusive: bool) -> FsResult<()> {
+ self
+ .with_inner_blocking_task(move |file| {
+ if exclusive {
+ file.lock_exclusive()?;
+ } else {
+ file.lock_shared()?;
+ }
+ Ok(())
+ })
.await
}
- pub async fn with_file_blocking_task2<F, R: Send + 'static>(
- self: Rc<Self>,
- f: F,
- ) -> Result<R, io::Error>
- where
- F: (FnOnce(&mut StdFile) -> Result<R, io::Error>) + Send + 'static,
- {
+ fn unlock_sync(self: Rc<Self>) -> FsResult<()> {
+ self.with_sync(|file| Ok(file.unlock()?))
+ }
+ async fn unlock_async(self: Rc<Self>) -> FsResult<()> {
self
- .with_inner_blocking_task(move |inner| inner.with_file(f))
+ .with_inner_blocking_task(|file| Ok(file.unlock()?))
.await
}
- pub fn clone_file(
- state: &mut OpState,
- rid: ResourceId,
- ) -> Result<StdFile, AnyError> {
- Self::with_file(state, rid, move |std_file| {
- std_file.try_clone().map_err(AnyError::from)
- })
+ fn truncate_sync(self: Rc<Self>, len: u64) -> FsResult<()> {
+ self.with_sync(|file| Ok(file.set_len(len)?))
}
-
- pub fn as_stdio(
- state: &mut OpState,
- rid: u32,
- ) -> Result<std::process::Stdio, AnyError> {
- Self::with_resource(state, rid, |resource| {
- resource
- .with_inner(|inner| match inner.kind {
- StdFileResourceKind::File => {
- let file = inner.file.try_clone()?;
- Ok(file.into())
- }
- _ => Ok(std::process::Stdio::inherit()),
- })
- .ok_or_else(resource_unavailable)?
- })
- }
-}
-
-impl Resource for StdFileResource {
- fn name(&self) -> Cow<str> {
- self.name.as_str().into()
+ async fn truncate_async(self: Rc<Self>, len: u64) -> FsResult<()> {
+ self
+ .with_inner_blocking_task(move |file| Ok(file.set_len(len)?))
+ .await
}
- fn read(self: Rc<Self>, limit: usize) -> AsyncResult<deno_core::BufView> {
- Box::pin(async move {
- let vec = vec![0; limit];
- let buf = BufMutView::from(vec);
- let (nread, buf) = StdFileResource::read_byob(self, buf).await?;
- let mut vec = buf.unwrap_vec();
- if vec.len() != nread {
- vec.truncate(nread);
- }
- Ok(BufView::from(vec))
+ fn utime_sync(
+ self: Rc<Self>,
+ atime_secs: i64,
+ atime_nanos: u32,
+ mtime_secs: i64,
+ mtime_nanos: u32,
+ ) -> FsResult<()> {
+ let atime = filetime::FileTime::from_unix_time(atime_secs, atime_nanos);
+ let mtime = filetime::FileTime::from_unix_time(mtime_secs, mtime_nanos);
+
+ self.with_sync(|file| {
+ filetime::set_file_handle_times(file, Some(atime), Some(mtime))?;
+ Ok(())
})
}
-
- fn read_byob(
+ async fn utime_async(
self: Rc<Self>,
- buf: deno_core::BufMutView,
- ) -> AsyncResult<(usize, deno_core::BufMutView)> {
- Box::pin(StdFileResource::read_byob(self, buf))
- }
+ atime_secs: i64,
+ atime_nanos: u32,
+ mtime_secs: i64,
+ mtime_nanos: u32,
+ ) -> FsResult<()> {
+ let atime = filetime::FileTime::from_unix_time(atime_secs, atime_nanos);
+ let mtime = filetime::FileTime::from_unix_time(mtime_secs, mtime_nanos);
- fn write(
- self: Rc<Self>,
- view: deno_core::BufView,
- ) -> AsyncResult<deno_core::WriteOutcome> {
- Box::pin(StdFileResource::write(self, view))
+ self
+ .with_inner_blocking_task(move |file| {
+ filetime::set_file_handle_times(file, Some(atime), Some(mtime))?;
+ Ok(())
+ })
+ .await
}
- fn write_all(self: Rc<Self>, view: deno_core::BufView) -> AsyncResult<()> {
- Box::pin(StdFileResource::write_all(self, view))
+ async fn read_byob(
+ self: Rc<Self>,
+ mut buf: BufMutView,
+ ) -> FsResult<(usize, BufMutView)> {
+ self
+ .with_inner_blocking_task(|file| {
+ let nread = file.read(&mut buf)?;
+ Ok((nread, buf))
+ })
+ .await
}
- fn write_sync(
- self: Rc<Self>,
- data: &[u8],
- ) -> Result<usize, deno_core::anyhow::Error> {
- StdFileResource::write_sync(self, data)
+ fn try_clone_inner(self: Rc<Self>) -> FsResult<Rc<dyn fs::File>> {
+ let inner: &Option<_> = &self.cell.borrow();
+ match inner {
+ Some(inner) => Ok(Rc::new(StdFileResourceInner {
+ kind: self.kind,
+ cell: RefCell::new(Some(inner.try_clone()?)),
+ cell_async_task_queue: Default::default(),
+ })),
+ None => Err(FsError::FileBusy),
+ }
}
- fn read_byob_sync(
- self: Rc<Self>,
- data: &mut [u8],
- ) -> Result<usize, deno_core::anyhow::Error> {
- StdFileResource::read_byob_sync(self, data)
+ fn as_stdio(self: Rc<Self>) -> FsResult<std::process::Stdio> {
+ match self.kind {
+ StdFileResourceKind::File => self.with_sync(|file| {
+ let file = file.try_clone()?;
+ Ok(file.into())
+ }),
+ _ => Ok(std::process::Stdio::inherit()),
+ }
}
#[cfg(unix)]
fn backing_fd(self: Rc<Self>) -> Option<std::os::unix::prelude::RawFd> {
use std::os::unix::io::AsRawFd;
- self
- .with_inner(move |std_file| {
- Ok::<_, ()>(std_file.with_file(|f| f.as_raw_fd()))
- })?
- .ok()
+ self.with_sync(|file| Ok(file.as_raw_fd())).ok()
+ }
+
+ #[cfg(windows)]
+ fn backing_fd(self: Rc<Self>) -> Option<std::os::windows::io::RawHandle> {
+ use std::os::windows::prelude::AsRawHandle;
+ self.with_sync(|file| Ok(file.as_raw_handle())).ok()
}
}
@@ -671,12 +739,7 @@ pub fn op_print(
is_err: bool,
) -> Result<(), AnyError> {
let rid = if is_err { 2 } else { 1 };
- StdFileResource::with_resource(state, rid, move |resource| {
- resource
- .with_inner(|inner| {
- inner.write_all_and_maybe_flush(msg.as_bytes())?;
- Ok(())
- })
- .ok_or_else(resource_unavailable)?
+ FileResource::with_file(state, rid, move |file| {
+ Ok(file.write_all_sync(msg.as_bytes())?)
})
}