From 73b96dc18a135938414ba4b187cd4b19cc135b69 Mon Sep 17 00:00:00 2001 From: Stefan Lankes Date: Sun, 6 Oct 2024 22:23:34 +0000 Subject: [PATCH] add initial draft of containerd-runh-shim This is a fork of containerd-shim-runc-v2-rs and adatapted for runh --- Cargo.toml | 79 +++ README.md | 11 + build.rs | 39 ++ crates/runh/Cargo.toml | 39 ++ crates/runh/README.md | 46 ++ crates/runh/src/container.rs | 86 +++ crates/runh/src/error.rs | 124 ++++ crates/runh/src/events.rs | 181 ++++++ crates/runh/src/io.rs | 426 ++++++++++++++ crates/runh/src/lib.rs | 1032 ++++++++++++++++++++++++++++++++++ crates/runh/src/monitor.rs | 180 ++++++ crates/runh/src/options.rs | 632 +++++++++++++++++++++ crates/runh/src/utils.rs | 133 +++++ src/cgroup_memory.rs | 202 +++++++ src/common.rs | 250 ++++++++ src/console.rs | 74 +++ src/container.rs | 248 ++++++++ src/io.rs | 38 ++ src/main.rs | 56 ++ src/processes.rs | 201 +++++++ src/runh.rs | 756 +++++++++++++++++++++++++ src/service.rs | 240 ++++++++ src/task.rs | 423 ++++++++++++++ 23 files changed, 5496 insertions(+) create mode 100644 Cargo.toml create mode 100644 README.md create mode 100644 build.rs create mode 100644 crates/runh/Cargo.toml create mode 100644 crates/runh/README.md create mode 100644 crates/runh/src/container.rs create mode 100644 crates/runh/src/error.rs create mode 100644 crates/runh/src/events.rs create mode 100644 crates/runh/src/io.rs create mode 100644 crates/runh/src/lib.rs create mode 100644 crates/runh/src/monitor.rs create mode 100644 crates/runh/src/options.rs create mode 100644 crates/runh/src/utils.rs create mode 100644 src/cgroup_memory.rs create mode 100644 src/common.rs create mode 100644 src/console.rs create mode 100644 src/container.rs create mode 100644 src/io.rs create mode 100644 src/main.rs create mode 100644 src/processes.rs create mode 100644 src/runh.rs create mode 100644 src/service.rs create mode 100644 src/task.rs diff --git a/Cargo.toml b/Cargo.toml new file mode 100644 index 0000000..1944e22 --- /dev/null +++ b/Cargo.toml @@ -0,0 +1,79 @@ +[package] +name = "containerd-runh-shim" +version = "0.1.1" +authors = [ + "Stefan Lankes ", + "Shaobao Feng ", + "Tianyang Zhang ", + "The containerd Authors", +] +description = "Rust implementation of containerd's runh v2 shim runtime" +keywords = ["containerd", "shim", "containers"] +categories = ["api-bindings", "asynchronous"] +edition = "2021" +license = "Apache-2.0" + +[workspace] +members = [ + "crates/runh", +] + +[workspace.package] +license = "Apache-2.0" +repository = "https://github.com/hermit-os/containerd-runh-shim" +homepage = "https://hermit-os.org" +edition = "2021" + +# Common dependencies for all crates +[workspace.dependencies] +async-trait = "0.1.52" +cgroups-rs = "0.3.4" +crossbeam = "0.8.1" +futures = "0.3.19" +libc = "0.2.112" +log = {version = "0.4.2", features=["kv_unstable"]} +nix = "0.29" +oci-spec = "0.6" +os_pipe = "1.1" +prctl = "1.0.0" +prost = "0.13" +prost-build = "0.13" +prost-types = "0.13" +serde = { version = "1.0", features = ["derive"] } +serde_json = "1.0" +simple_logger = { version = "5.0", default-features = false } +tempfile = "3.6" +thiserror = "1.0" +time = { version = "0.3.29", features = ["serde", "std", "formatting"] } +tokio = "1.26" +tonic = "0.12" +tonic-build = "0.12" +tower = "0.4" +uuid = { version = "1.0", features = ["v4"] } + +[[bin]] +name = "containerd-shim-runh-v2" +path = "src/main.rs" +doc = false + +[dependencies] +containerd-shim = { version = "0.7.4", features = ["async"] } +libc.workspace = true +log.workspace = true +nix = { workspace = true, features = ["socket", "uio", "term"] } +oci-spec.workspace = true +prctl.workspace = true +runh = { path = "crates/runh", version = "0.2.0", features = ["async"] } +serde.workspace = true +serde_json.workspace = true +time.workspace = true +uuid.workspace = true + +# Async dependencies +async-trait.workspace = true +tokio = { workspace = true, features = ["full"] } + +[target.'cfg(target_os = "linux")'.dependencies] +cgroups-rs.workspace = true +nix = { workspace = true, features = ["event"] } + diff --git a/README.md b/README.md new file mode 100644 index 0000000..b3e5711 --- /dev/null +++ b/README.md @@ -0,0 +1,11 @@ +# Rust containerd shim v2 for runh container + +This shim is a fork of [io.containerd.runc.v2-rs](https://github.com/containerd/rust-extensions/blob/main/crates/runc-shim) and adapted for [runh](https://github.com/hermit-os/runh). +By default [containerd](https://github.com/containerd/containerd) relies on runtime shim to launch containers. + +## Usage + +To build binary, run: +```shell +cargo build --release +``` diff --git a/build.rs b/build.rs new file mode 100644 index 0000000..5184c60 --- /dev/null +++ b/build.rs @@ -0,0 +1,39 @@ +/* + Copyright The containerd Authors. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +*/ + +use std::{process::Command, str::from_utf8}; + +fn main() { + let output = match Command::new("git").arg("rev-parse").arg("HEAD").output() { + Ok(output) => output, + Err(_) => { + return; + } + }; + let mut hash = from_utf8(&output.stdout).unwrap().trim().to_string(); + + let output_dirty = match Command::new("git").arg("diff").arg("--exit-code").output() { + Ok(output) => output, + Err(_) => { + return; + } + }; + + if !output_dirty.status.success() { + hash.push_str(".m"); + } + println!("cargo:rustc-env=CARGO_GIT_HASH={}", hash); +} diff --git a/crates/runh/Cargo.toml b/crates/runh/Cargo.toml new file mode 100644 index 0000000..71ed674 --- /dev/null +++ b/crates/runh/Cargo.toml @@ -0,0 +1,39 @@ +[package] +name = "runh" +version = "0.2.0" +authors = ["Stefan Lankes ", "Yuna Tomida ", "The containerd Authors"] +description = "A crate for consuming the runh binary in your Rust applications" +keywords = ["containerd", "containers", "runh"] +categories = ["api-bindings", "asynchronous"] + +edition.workspace = true +license.workspace = true +repository.workspace = true +homepage.workspace = true + +[features] +async = ["tokio", "async-trait", "tokio-pipe"] +docs = [] + +[dependencies] +libc.workspace = true +log.workspace = true +nix = { workspace = true, features = ["user", "fs"] } +oci-spec.workspace = true +os_pipe.workspace = true +path-absolutize = "3.0.11" +prctl.workspace = true +serde.workspace = true +serde_json.workspace = true +tempfile.workspace = true +thiserror.workspace = true +time.workspace = true +uuid.workspace = true + +# Async dependencies +async-trait = { workspace = true, optional = true } +tokio = { workspace = true, features = ["full"], optional = true } +tokio-pipe = { version = "0.2.10", optional = true } + +[package.metadata.docs.rs] +features = ["docs"] diff --git a/crates/runh/README.md b/crates/runh/README.md new file mode 100644 index 0000000..54b0292 --- /dev/null +++ b/crates/runh/README.md @@ -0,0 +1,46 @@ +# Rust bindings for runc CLI + +[![Crates.io](https://img.shields.io/crates/v/runc)](https://crates.io/crates/runc) +[![docs.rs](https://img.shields.io/docsrs/runc)](https://docs.rs/runc/latest/runc/) +[![Crates.io](https://img.shields.io/crates/l/containerd-shim)](https://github.com/containerd/rust-extensions/blob/main/LICENSE) +[![CI](https://github.com/containerd/rust-extensions/actions/workflows/ci.yml/badge.svg?branch=main)](https://github.com/containerd/rust-extensions/actions/workflows/ci.yml) + +A crate for consuming the runc binary in your Rust applications, similar to [go-runc](https://github.com/containerd/go-runc) for Go. +This crate is based on archived [rust-runc](https://github.com/pwFoo/rust-runc). + +## Usage +Both sync/async version is available. +You can build runc client with `RuncConfig` in method chaining style. +Call `build()` or `build_async()` to get client. +Note that async client depends on [tokio](https://github.com/tokio-rs/tokio), then please use it on tokio runtime. + +```rust,ignore +#[tokio::main] +async fn main() { + let config = runc::GlobalOpts::new() + .root("./new_root") + .debug(false) + .log("/path/to/logfile.json") + .log_format(runc::LogFormat::Json) + .rootless(true); + + let client = config.build_async().unwrap(); + + let opts = runc::options::CreateOpts::new() + .pid_file("/path/to/pid/file") + .no_pivot(true); + + client.create("container-id", "path/to/bundle", Some(&opts)).unwrap(); +} +``` + +## Limitations +- Supported commands are only: + - create + - start + - state + - kill + - delete +- Exec is **not** available in `RuncAsyncClient` now. +- Console utilites are **not** available + - see [Go version](https://github.com/containerd/go-runc/blob/main/console.go) diff --git a/crates/runh/src/container.rs b/crates/runh/src/container.rs new file mode 100644 index 0000000..0f846c7 --- /dev/null +++ b/crates/runh/src/container.rs @@ -0,0 +1,86 @@ +/* + Copyright The containerd Authors. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +*/ + +// Forked from https://github.com/pwFoo/rust-runc/blob/313e6ae5a79b54455b0a242a795c69adf035141a/src/lib.rs + +/* + * Copyright 2020 fsyncd, Berlin, Germany. + * Additional material, copyright of the containerd authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +use std::collections::HashMap; + +use serde::{Deserialize, Serialize}; +use time::{serde::timestamp, OffsetDateTime}; + +/// Information for runc container +#[derive(Debug, Serialize, Deserialize)] +pub struct Container { + pub id: String, + pub pid: usize, + pub status: String, + pub bundle: String, + pub rootfs: String, + #[serde(with = "timestamp")] + pub created: OffsetDateTime, + pub annotations: HashMap, +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn serde_test() { + let j = r#" + { + "id": "fake", + "pid": 1000, + "status": "RUNNING", + "bundle": "/path/to/bundle", + "rootfs": "/path/to/rootfs", + "created": 1431684000, + "annotations": { + "foo": "bar" + } + }"#; + + let c: Container = serde_json::from_str(j).unwrap(); + assert_eq!(c.id, "fake"); + assert_eq!(c.pid, 1000); + assert_eq!(c.status, "RUNNING"); + assert_eq!(c.bundle, "/path/to/bundle"); + assert_eq!(c.rootfs, "/path/to/rootfs"); + assert_eq!( + c.created, + OffsetDateTime::from_unix_timestamp(1431684000).unwrap() + ); + assert_eq!(c.annotations.get("foo"), Some(&"bar".to_string())); + assert_eq!(c.annotations.get("bar"), None); + } +} diff --git a/crates/runh/src/error.rs b/crates/runh/src/error.rs new file mode 100644 index 0000000..ac437ad --- /dev/null +++ b/crates/runh/src/error.rs @@ -0,0 +1,124 @@ +/* + Copyright The containerd Authors. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +*/ + +// Forked from https://github.com/pwFoo/rust-runc/blob/313e6ae5a79b54455b0a242a795c69adf035141a/src/lib.rs + +/* + * Copyright 2020 fsyncd, Berlin, Germany. + * Additional material, copyright of the containerd authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +use std::{env, io, process::ExitStatus}; + +use thiserror::Error; + +#[derive(Error, Debug)] +pub enum Error { + #[error("Unable to extract test files: {0}")] + BundleExtractFailed(io::Error), + + #[error("Invalid path: {0}")] + InvalidPath(io::Error), + + #[error(transparent)] + JsonDeserializationFailed(#[from] serde_json::error::Error), + + #[error("Missing container statistics")] + MissingContainerStats, + + #[error(transparent)] + ProcessSpawnFailed(io::Error), + + #[error("Error occured in runc: {0}")] + InvalidCommand(io::Error), + + #[error("Runh command failed: status={status}, stdout=\"{stdout}\", stderr=\"{stderr}\"")] + CommandFailed { + status: ExitStatus, + stdout: String, + stderr: String, + }, + + #[error("Runh IO unavailable: {0}")] + UnavailableIO(io::Error), + + #[cfg(feature = "async")] + #[error("Runh command timed out: {0}")] + CommandTimeout(tokio::time::error::Elapsed), + + #[error("Unable to parse runc version")] + InvalidVersion, + + #[error("Unable to locate the runc")] + NotFound, + + #[error("Error occurs with fs: {0}")] + FileSystemError(io::Error), + + #[error("Failed to spec file: {0}")] + SpecFileCreationFailed(io::Error), + + #[error(transparent)] + SpecFileCleanupFailed(io::Error), + + #[error("Failed to find valid path for spec file")] + SpecFileNotFound, + + #[error("Top command is missing a pid header")] + TopMissingPidHeader, + + #[error("Top command returned an empty response")] + TopShortResponseError, + + #[error("Unix socket connection error: {0}")] + UnixSocketConnectionFailed(io::Error), + + #[error("Unable to bind to unix socket: {0}")] + UnixSocketBindFailed(io::Error), + + #[error("Unix socket failed to receive pty")] + UnixSocketReceiveMessageFailed, + + #[error("Unix socket unexpectedly closed")] + UnixSocketClosed, + + #[error("Failed to handle environment variable: {0}")] + EnvError(env::VarError), + + #[error("Sorry, this part of api is not implemented: {0}")] + Unimplemented(String), + + #[error("Error occured in runc client: {0}")] + Other(Box), + + #[error("Failed to set cmd io: {0}")] + IoSet(String), + + #[error("Failed to create dir: {0}")] + CreateDir(nix::Error), +} diff --git a/crates/runh/src/events.rs b/crates/runh/src/events.rs new file mode 100644 index 0000000..9bf7125 --- /dev/null +++ b/crates/runh/src/events.rs @@ -0,0 +1,181 @@ +/* + Copyright The containerd Authors. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +*/ + +// Forked from https://github.com/pwFoo/rust-runc/blob/313e6ae5a79b54455b0a242a795c69adf035141a/src/events.rs + +/* + * Copyright 2020 fsyncd, Berlin, Germany. + * Additional material, copyright of the containerd authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +use std::collections::HashMap; + +use serde::{Deserialize, Serialize}; + +/// Event type generated by runc +#[derive(Debug, Clone, Serialize, Deserialize)] +#[serde(rename_all(serialize = "lowercase", deserialize = "lowercase"))] +pub enum EventType { + /// Statistics + Stats, + /// Out of memory + Oom, +} + +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct Event { + #[serde(rename = "type")] + pub event_type: EventType, + pub id: String, + #[serde(rename = "data")] + pub stats: Option, +} + +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct Stats { + pub cpu: Cpu, + pub memory: Memory, + pub pids: Pids, + #[serde(rename = "blkio")] + pub block_io: BlkIO, + #[serde(rename = "hugetlb")] + pub huge_tlb: HugeTLB, +} + +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct HugeTLB { + pub usage: Option, + pub max: Option, + #[serde(rename = "failcnt")] + pub fail_count: u64, +} + +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct BlkIOEntry { + pub major: Option, + pub minor: Option, + pub op: Option, + pub value: Option, +} + +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct BlkIO { + /// Number of bytes transferred to and from the disk + #[serde(rename = "ioServiceBytesRecursive")] + pub io_service_bytes_recursive: Option>, + /// Number of io requests issued to the disk + #[serde(rename = "ioServicedRecursive")] + pub io_serviced_recursive: Option>, + /// Number of queued disk io requests + #[serde(rename = "ioQueueRecursive")] + pub io_queued_recursive: Option>, + /// Amount of time io requests took to service + #[serde(rename = "ioServiceTimeRecursive")] + pub io_service_time_recursive: Option>, + /// Amount of time io requests spent waiting in the queue + #[serde(rename = "ioWaitTimeRecursive")] + pub io_wait_time_recursive: Option>, + /// Number of merged io requests + #[serde(rename = "ioMergedRecursive")] + pub io_merged_recursive: Option>, + /// Disk time allocated the device + #[serde(rename = "ioTimeRecursive")] + pub io_time_recursive: Option>, + /// Number of sectors transferred to and from the io device + #[serde(rename = "sectorsRecursive")] + pub sectors_recursive: Option>, +} + +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct Pids { + /// Number of pids in the cgroup + pub current: Option, + /// Active pids hard limit + pub limit: Option, +} + +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct Throttling { + /// Number of periods with throttling active + pub periods: Option, + #[serde(rename = "throttledPeriods")] + /// Number of periods when the container hit its throttling limit + pub throtted_periods: Option, + /// Aggregate time the container was throttled for in nanoseconds + #[serde(rename = "throttledTime")] + pub throtted_time: Option, +} + +/// Each members represents time in nanoseconds +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct CpuUsage { + /// Total CPU time consumed + pub total: Option, + /// Total CPU time consumed per core + pub per_cpu: Option>, + /// Total CPU time consumed in kernel mode + pub kernel: u64, + /// Total CPU time consumed in user mode + pub user: u64, +} + +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct Cpu { + pub usage: Option, + pub throttling: Option, +} + +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct MemoryEntry { + /// Memory limit in bytes + pub limit: u64, + /// Usage in bytes + pub usage: Option, + /// Maximum usage in bytes + pub max: Option, + /// Count of memory allocation failures + #[serde(rename = "failcnt")] + pub fail_count: u64, +} + +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct Memory { + /// Memory usage for cache + pub cache: Option, + /// Overall memory usage, excluding swap + pub usage: Option, + /// Overall memory usage, including swap + pub swap: Option, + /// Kernel usage of memory + pub kernel: Option, + /// Kernel TCP of memory + #[serde(rename = "kernelTCP")] + pub kernel_tcp: Option, + /// Raw stats of memory + pub raw: Option>, +} diff --git a/crates/runh/src/io.rs b/crates/runh/src/io.rs new file mode 100644 index 0000000..9758ec3 --- /dev/null +++ b/crates/runh/src/io.rs @@ -0,0 +1,426 @@ +/* + Copyright The containerd Authors. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +*/ +#[cfg(not(feature = "async"))] +use std::io::{Read, Write}; +use std::{ + fmt::Debug, + fs::{File, OpenOptions}, + io::Result, + os::unix::{fs::OpenOptionsExt, io::AsRawFd}, + process::Stdio, + sync::Mutex, +}; + +use log::debug; +use nix::unistd::{Gid, Uid}; +use os_pipe::{PipeReader, PipeWriter}; +#[cfg(feature = "async")] +use tokio::io::{AsyncRead, AsyncWrite}; + +use crate::Command; + +pub trait Io: Debug + Send + Sync { + /// Return write side of stdin + #[cfg(not(feature = "async"))] + fn stdin(&self) -> Option> { + None + } + + /// Return read side of stdout + #[cfg(not(feature = "async"))] + fn stdout(&self) -> Option> { + None + } + + /// Return read side of stderr + #[cfg(not(feature = "async"))] + fn stderr(&self) -> Option> { + None + } + + /// Return write side of stdin + #[cfg(feature = "async")] + fn stdin(&self) -> Option> { + None + } + + /// Return read side of stdout + #[cfg(feature = "async")] + fn stdout(&self) -> Option> { + None + } + + /// Return read side of stderr + #[cfg(feature = "async")] + fn stderr(&self) -> Option> { + None + } + + /// Set IO for passed command. + /// Read side of stdin, write side of stdout and write side of stderr should be provided to command. + fn set(&self, cmd: &mut Command) -> Result<()>; + + /// Only close write side (should be stdout/err "from" runc process) + fn close_after_start(&self); +} + +#[derive(Debug, Clone)] +pub struct IOOption { + pub open_stdin: bool, + pub open_stdout: bool, + pub open_stderr: bool, +} + +impl Default for IOOption { + fn default() -> Self { + Self { + open_stdin: true, + open_stdout: true, + open_stderr: true, + } + } +} + +/// Struct to represent a pipe that can be used to transfer stdio inputs and outputs. +/// +/// With this Io driver, methods of [crate::Runh] may capture the output/error messages. +/// When one side of the pipe is closed, the state will be represented with [`None`]. +#[derive(Debug)] +pub struct Pipe { + rd: PipeReader, + wr: PipeWriter, +} + +#[derive(Debug)] +pub struct PipedIo { + stdin: Option, + stdout: Option, + stderr: Option, +} + +impl Pipe { + fn new() -> std::io::Result { + let (rd, wr) = os_pipe::pipe()?; + Ok(Self { rd, wr }) + } +} + +impl PipedIo { + pub fn new(uid: u32, gid: u32, opts: &IOOption) -> std::io::Result { + Ok(Self { + stdin: Self::create_pipe(uid, gid, opts.open_stdin, true)?, + stdout: Self::create_pipe(uid, gid, opts.open_stdout, false)?, + stderr: Self::create_pipe(uid, gid, opts.open_stderr, false)?, + }) + } + + fn create_pipe( + uid: u32, + gid: u32, + enabled: bool, + stdin: bool, + ) -> std::io::Result> { + if !enabled { + return Ok(None); + } + + let pipe = Pipe::new()?; + let uid = Some(Uid::from_raw(uid)); + let gid = Some(Gid::from_raw(gid)); + if stdin { + let rd = pipe.rd.try_clone()?; + nix::unistd::fchown(rd.as_raw_fd(), uid, gid)?; + } else { + let wr = pipe.wr.try_clone()?; + nix::unistd::fchown(wr.as_raw_fd(), uid, gid)?; + } + Ok(Some(pipe)) + } +} + +impl Io for PipedIo { + #[cfg(not(feature = "async"))] + fn stdin(&self) -> Option> { + self.stdin.as_ref().and_then(|pipe| { + pipe.wr + .try_clone() + .map(|x| Box::new(x) as Box) + .ok() + }) + } + + #[cfg(feature = "async")] + fn stdin(&self) -> Option> { + self.stdin.as_ref().and_then(|pipe| { + let fd = pipe.wr.as_raw_fd(); + tokio_pipe::PipeWrite::from_raw_fd_checked(fd) + .map(|x| Box::new(x) as Box) + .ok() + }) + } + + #[cfg(not(feature = "async"))] + fn stdout(&self) -> Option> { + self.stdout.as_ref().and_then(|pipe| { + pipe.rd + .try_clone() + .map(|x| Box::new(x) as Box) + .ok() + }) + } + + #[cfg(feature = "async")] + fn stdout(&self) -> Option> { + self.stdout.as_ref().and_then(|pipe| { + let fd = pipe.rd.as_raw_fd(); + tokio_pipe::PipeRead::from_raw_fd_checked(fd) + .map(|x| Box::new(x) as Box) + .ok() + }) + } + + #[cfg(not(feature = "async"))] + fn stderr(&self) -> Option> { + self.stderr.as_ref().and_then(|pipe| { + pipe.rd + .try_clone() + .map(|x| Box::new(x) as Box) + .ok() + }) + } + + #[cfg(feature = "async")] + fn stderr(&self) -> Option> { + self.stderr.as_ref().and_then(|pipe| { + let fd = pipe.rd.as_raw_fd(); + tokio_pipe::PipeRead::from_raw_fd_checked(fd) + .map(|x| Box::new(x) as Box) + .ok() + }) + } + + // Note that this internally use [`std::fs::File`]'s `try_clone()`. + // Thus, the files passed to commands will be not closed after command exit. + fn set(&self, cmd: &mut Command) -> std::io::Result<()> { + if let Some(p) = self.stdin.as_ref() { + let pr = p.rd.try_clone()?; + cmd.stdin(pr); + } + + if let Some(p) = self.stdout.as_ref() { + let pw = p.wr.try_clone()?; + cmd.stdout(pw); + } + + if let Some(p) = self.stderr.as_ref() { + let pw = p.wr.try_clone()?; + cmd.stdout(pw); + } + + Ok(()) + } + + fn close_after_start(&self) { + if let Some(p) = self.stdout.as_ref() { + nix::unistd::close(p.wr.as_raw_fd()).unwrap_or_else(|e| debug!("close stdout: {}", e)); + } + + if let Some(p) = self.stderr.as_ref() { + nix::unistd::close(p.wr.as_raw_fd()).unwrap_or_else(|e| debug!("close stderr: {}", e)); + } + } +} + +/// IO driver to direct output/error messages to /dev/null. +/// +/// With this Io driver, all methods of [crate::Runh] can't capture the output/error messages. +#[derive(Debug)] +pub struct NullIo { + dev_null: Mutex>, +} + +impl NullIo { + pub fn new() -> std::io::Result { + let f = OpenOptions::new().read(true).open("/dev/null")?; + let dev_null = Mutex::new(Some(f)); + Ok(Self { dev_null }) + } +} + +impl Io for NullIo { + fn set(&self, cmd: &mut Command) -> std::io::Result<()> { + if let Some(null) = self.dev_null.lock().unwrap().as_ref() { + cmd.stdout(null.try_clone()?); + cmd.stderr(null.try_clone()?); + } + Ok(()) + } + + fn close_after_start(&self) { + let mut m = self.dev_null.lock().unwrap(); + let _ = m.take(); + } +} + +/// Io driver based on Stdio::inherited(), to direct outputs/errors to stdio. +/// +/// With this Io driver, all methods of [crate::Runh] can't capture the output/error messages. +#[derive(Debug)] +pub struct InheritedStdIo {} + +impl InheritedStdIo { + pub fn new() -> std::io::Result { + Ok(InheritedStdIo {}) + } +} + +impl Io for InheritedStdIo { + fn set(&self, cmd: &mut Command) -> std::io::Result<()> { + cmd.stdin(Stdio::null()) + .stdout(Stdio::inherit()) + .stderr(Stdio::inherit()); + Ok(()) + } + + fn close_after_start(&self) {} +} + +/// Io driver based on Stdio::piped(), to capture outputs/errors from runC. +/// +/// With this Io driver, methods of [crate::Runh] may capture the output/error messages. +#[derive(Debug)] +pub struct PipedStdIo {} + +impl PipedStdIo { + pub fn new() -> std::io::Result { + Ok(PipedStdIo {}) + } +} + +impl Io for PipedStdIo { + fn set(&self, cmd: &mut Command) -> std::io::Result<()> { + cmd.stdin(Stdio::null()) + .stdout(Stdio::piped()) + .stderr(Stdio::piped()); + Ok(()) + } + + fn close_after_start(&self) {} +} + +/// FIFO for the scenario that set FIFO for command Io. +#[derive(Debug)] +pub struct FIFO { + pub stdin: Option, + pub stdout: Option, + pub stderr: Option, +} + +impl Io for FIFO { + fn set(&self, cmd: &mut Command) -> Result<()> { + if let Some(path) = self.stdin.as_ref() { + let stdin = OpenOptions::new() + .read(true) + .custom_flags(libc::O_NONBLOCK) + .open(path)?; + cmd.stdin(stdin); + } + + if let Some(path) = self.stdout.as_ref() { + let stdout = OpenOptions::new().write(true).open(path)?; + cmd.stdout(stdout); + } + + if let Some(path) = self.stderr.as_ref() { + let stderr = OpenOptions::new().write(true).open(path)?; + cmd.stderr(stderr); + } + + Ok(()) + } + + fn close_after_start(&self) {} +} + +#[cfg(test)] +mod tests { + use super::*; + + #[cfg(not(target_os = "macos"))] + #[test] + fn test_io_option() { + let opts = IOOption { + open_stdin: false, + open_stdout: false, + open_stderr: false, + }; + let io = PipedIo::new(1000, 1000, &opts).unwrap(); + + assert!(io.stdin().is_none()); + assert!(io.stdout().is_none()); + assert!(io.stderr().is_none()); + } + + #[cfg(target_os = "linux")] + #[cfg(not(feature = "async"))] + #[test] + fn test_create_piped_io() { + let opts = IOOption::default(); + let uid = nix::unistd::getuid(); + let gid = nix::unistd::getgid(); + let io = PipedIo::new(uid.as_raw(), gid.as_raw(), &opts).unwrap(); + let mut buf = [0xfau8]; + + let mut stdin = io.stdin().unwrap(); + stdin.write_all(&buf).unwrap(); + buf[0] = 0x0; + + io.stdin + .as_ref() + .map(|v| v.rd.try_clone().unwrap().read(&mut buf).unwrap()); + assert_eq!(&buf, &[0xfau8]); + + let mut stdout = io.stdout().unwrap(); + buf[0] = 0xce; + io.stdout + .as_ref() + .map(|v| v.wr.try_clone().unwrap().write(&buf).unwrap()); + buf[0] = 0x0; + stdout.read_exact(&mut buf).unwrap(); + assert_eq!(&buf, &[0xceu8]); + + let mut stderr = io.stderr().unwrap(); + buf[0] = 0xa5; + io.stderr + .as_ref() + .map(|v| v.wr.try_clone().unwrap().write(&buf).unwrap()); + buf[0] = 0x0; + stderr.read_exact(&mut buf).unwrap(); + assert_eq!(&buf, &[0xa5u8]); + + io.close_after_start(); + stdout.read_exact(&mut buf).unwrap_err(); + stderr.read_exact(&mut buf).unwrap_err(); + } + + #[test] + fn test_null_io() { + let io = NullIo::new().unwrap(); + assert!(io.stdin().is_none()); + assert!(io.stdout().is_none()); + assert!(io.stderr().is_none()); + } +} diff --git a/crates/runh/src/lib.rs b/crates/runh/src/lib.rs new file mode 100644 index 0000000..730ba8e --- /dev/null +++ b/crates/runh/src/lib.rs @@ -0,0 +1,1032 @@ +/* + Copyright The containerd Authors. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +*/ + +// Forked from https://github.com/pwFoo/rust-runc/blob/313e6ae5a79b54455b0a242a795c69adf035141a/src/lib.rs + +/* + * Copyright 2020 fsyncd, Berlin, Germany. + * Additional material, copyright of the containerd authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#![cfg_attr(feature = "docs", doc = include_str!("../README.md"))] + +//! A crate for consuming the runc binary in your Rust applications, similar to +//! [go-runc](https://github.com/containerd/go-runc) for Go. +use std::{ + fmt::{self, Debug, Display}, + path::{Path, PathBuf}, + process::{ExitStatus, Stdio}, + sync::Arc, +}; + +#[cfg(feature = "async")] +use async_trait::async_trait; +#[cfg(feature = "async")] +use log::debug; +use oci_spec::runtime::{LinuxResources, Process}; + +use crate::{container::Container, error::Error, options::*, utils::write_value_to_temp_file}; + +pub mod container; +pub mod error; +pub mod events; +pub mod io; +#[cfg(feature = "async")] +pub mod monitor; +pub mod options; +pub mod utils; + +pub type Result = std::result::Result; + +/// Response is for (pid, exit status, outputs). +#[derive(Debug, Clone)] +pub struct Response { + pub pid: u32, + pub status: ExitStatus, + pub output: String, +} + +#[derive(Debug, Clone)] +pub struct Version { + pub runc_version: Option, + pub spec_version: Option, + pub commit: Option, +} + +#[derive(Debug, Clone, Default)] +pub enum LogFormat { + Json, + #[default] + Text, +} + +impl Display for LogFormat { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + LogFormat::Json => write!(f, "{}", JSON), + LogFormat::Text => write!(f, "{}", TEXT), + } + } +} + +#[cfg(not(feature = "async"))] +pub type Command = std::process::Command; + +#[cfg(feature = "async")] +pub type Command = tokio::process::Command; + +#[derive(Debug, Clone)] +pub struct Runh { + command: PathBuf, + args: Vec, + spawner: Arc, +} + +impl Runh { + fn command(&self, args: &[String]) -> Result { + let args = [&self.args, args].concat(); + let mut cmd = Command::new(&self.command); + + // Default to piped stdio, and they may be override by command options. + cmd.stdin(Stdio::null()) + .stdout(Stdio::piped()) + .stderr(Stdio::piped()); + + // NOTIFY_SOCKET introduces a special behavior in runc but should only be set if invoked from systemd + cmd.args(&args).env_remove("NOTIFY_SOCKET"); + + Ok(cmd) + } +} + +#[cfg(not(feature = "async"))] +impl Runh { + fn launch(&self, cmd: Command, combined_output: bool) -> Result { + let (status, pid, stdout, stderr) = self.spawner.execute(cmd)?; + if status.success() { + let output = if combined_output { + stdout + stderr.as_str() + } else { + stdout + }; + Ok(Response { + pid, + status, + output, + }) + } else { + Err(Error::CommandFailed { + status, + stdout, + stderr, + }) + } + } + + /// Create a new container + pub fn create

(&self, id: &str, bundle: P, opts: Option<&CreateOpts>) -> Result + where + P: AsRef, + { + let mut args = vec![ + "create".to_string(), + "--bundle".to_string(), + utils::abs_string(bundle)?, + ]; + if let Some(opts) = opts { + args.append(&mut opts.args()?); + } + args.push(id.to_string()); + let mut cmd = self.command(&args)?; + match opts { + Some(CreateOpts { io: Some(io), .. }) => { + io.set(&mut cmd).map_err(|e| Error::IoSet(e.to_string()))?; + let res = self.launch(cmd, true)?; + io.close_after_start(); + Ok(res) + } + _ => self.launch(cmd, true), + } + } + + /// Delete a container + pub fn delete(&self, id: &str, opts: Option<&DeleteOpts>) -> Result<()> { + let mut args = vec!["delete".to_string()]; + if let Some(opts) = opts { + args.append(&mut opts.args()); + } + args.push(id.to_string()); + self.launch(self.command(&args)?, true)?; + Ok(()) + } + + /// Execute an additional process inside the container + pub fn exec(&self, id: &str, spec: &Process, opts: Option<&ExecOpts>) -> Result<()> { + let (_temp_file, filename) = write_value_to_temp_file(spec)?; + let mut args = vec!["exec".to_string(), "--process".to_string(), filename]; + if let Some(opts) = opts { + args.append(&mut opts.args()?); + } + args.push(id.to_string()); + let mut cmd = self.command(&args)?; + match opts { + Some(ExecOpts { io: Some(io), .. }) => { + io.set(&mut cmd).map_err(|e| Error::IoSet(e.to_string()))?; + self.launch(cmd, true)?; + io.close_after_start(); + } + _ => { + self.launch(cmd, true)?; + } + } + Ok(()) + } + + /// Send the specified signal to processes inside the container + pub fn kill(&self, id: &str, sig: u32, opts: Option<&KillOpts>) -> Result<()> { + let mut args = vec!["kill".to_string()]; + if let Some(opts) = opts { + args.append(&mut opts.args()); + } + args.push(id.to_string()); + args.push(sig.to_string()); + let _ = self.launch(self.command(&args)?, true)?; + Ok(()) + } + + /// List all containers associated with this runh instance + pub fn list(&self) -> Result> { + let args = ["list".to_string(), "--format=json".to_string()]; + let res = self.launch(self.command(&args)?, true)?; + let output = res.output.trim(); + + // Ugly hack to work around golang + Ok(if output == "null" { + Vec::new() + } else { + serde_json::from_str(output).map_err(Error::JsonDeserializationFailed)? + }) + } + + /// Pause a container + pub fn pause(&self, id: &str) -> Result<()> { + let args = ["pause".to_string(), id.to_string()]; + let _ = self.launch(self.command(&args)?, true)?; + Ok(()) + } + + /// Resume a container + pub fn resume(&self, id: &str) -> Result<()> { + let args = ["resume".to_string(), id.to_string()]; + let _ = self.launch(self.command(&args)?, true)?; + Ok(()) + } + + pub fn checkpoint(&self) -> Result<()> { + Err(Error::Unimplemented("checkpoint".to_string())) + } + + pub fn restore(&self) -> Result<()> { + Err(Error::Unimplemented("restore".to_string())) + } + + /// List all the processes inside the container, returning their pids + pub fn ps(&self, id: &str) -> Result> { + let args = [ + "ps".to_string(), + "--format=json".to_string(), + id.to_string(), + ]; + let res = self.launch(self.command(&args)?, false)?; + let output = res.output.trim(); + + // Ugly hack to work around golang + Ok(if output == "null" { + Vec::new() + } else { + serde_json::from_str(output).map_err(Error::JsonDeserializationFailed)? + }) + } + + /// Run the create, start, delete lifecycle of the container and return its exit status + pub fn run

(&self, id: &str, bundle: P, opts: Option<&CreateOpts>) -> Result + where + P: AsRef, + { + let mut args = vec![ + "run".to_string(), + "--bundle".to_string(), + utils::abs_string(bundle)?, + ]; + if let Some(opts) = opts { + args.append(&mut opts.args()?); + } + args.push(id.to_string()); + let mut cmd = self.command(&args)?; + if let Some(CreateOpts { io: Some(io), .. }) = opts { + io.set(&mut cmd).map_err(|e| Error::IoSet(e.to_string()))?; + }; + self.launch(cmd, true) + } + + /// Start an already created container + pub fn start(&self, id: &str) -> Result { + let args = ["start".to_string(), id.to_string()]; + self.launch(self.command(&args)?, true) + } + + /// Return the state of a container + pub fn state(&self, id: &str) -> Result { + let args = ["state".to_string(), id.to_string()]; + let res = self.launch(self.command(&args)?, true)?; + serde_json::from_str(&res.output).map_err(Error::JsonDeserializationFailed) + } + + /// Return the latest statistics for a container + pub fn stats(&self, id: &str) -> Result { + let args = vec!["events".to_string(), "--stats".to_string(), id.to_string()]; + let res = self.launch(self.command(&args)?, true)?; + let event: events::Event = + serde_json::from_str(&res.output).map_err(Error::JsonDeserializationFailed)?; + if let Some(stats) = event.stats { + Ok(stats) + } else { + Err(Error::MissingContainerStats) + } + } + + /// Update a container with the provided resource spec + pub fn update(&self, id: &str, resources: &LinuxResources) -> Result<()> { + let (_temp_file, filename) = write_value_to_temp_file(resources)?; + let args = [ + "update".to_string(), + "--resources".to_string(), + filename, + id.to_string(), + ]; + self.launch(self.command(&args)?, true)?; + Ok(()) + } +} + +// a macro tool to cleanup the file with name $filename, +// there is no async drop in async rust, so we have to call remove_file everytime +// after a temp file created, before return of a function. +// with this macro we don't have to write the match case codes everytime. +#[cfg(feature = "async")] +macro_rules! tc { + ($b:expr, $filename: expr) => { + match $b { + Ok(r) => r, + Err(e) => { + let _ = tokio::fs::remove_file($filename).await; + return Err(e); + } + } + }; +} + +#[cfg(not(feature = "async"))] +pub trait Spawner: Debug { + fn execute(&self, cmd: Command) -> Result<(ExitStatus, u32, String, String)>; +} + +#[cfg(feature = "async")] +#[async_trait] +pub trait Spawner: Debug { + async fn execute(&self, cmd: Command) -> Result<(ExitStatus, u32, String, String)>; +} + +/// Async implementation for [Runh]. +/// +/// Note that you MUST use this client on tokio runtime, as this client internally use [`tokio::process::Command`] +/// and some other utilities. +#[cfg(feature = "async")] +impl Runh { + async fn launch(&self, mut cmd: Command, combined_output: bool) -> Result { + debug!("Execute command {:?}", cmd); + unsafe { + cmd.pre_exec(move || { + #[cfg(target_os = "linux")] + if let Ok(thp) = std::env::var("THP_DISABLED") { + if let Ok(thp_disabled) = thp.parse::() { + if let Err(e) = prctl::set_thp_disable(thp_disabled) { + debug!("set_thp_disable err: {}", e); + }; + } + } + Ok(()) + }); + } + + let (status, pid, stdout, stderr) = self.spawner.execute(cmd).await?; + if status.success() { + let output = if combined_output { + stdout + stderr.as_str() + } else { + stdout + }; + Ok(Response { + pid, + status, + output, + }) + } else { + Err(Error::CommandFailed { + status, + stdout, + stderr, + }) + } + } + + /// Create a new container + pub async fn create

( + &self, + id: &str, + bundle: P, + opts: Option<&CreateOpts>, + ) -> Result + where + P: AsRef, + { + let mut args = vec![ + "create".to_string(), + "--bundle".to_string(), + utils::abs_string(bundle)?, + ]; + if let Some(opts) = opts { + args.append(&mut opts.args()?); + } + args.push(id.to_string()); + let mut cmd = self.command(&args)?; + match opts { + Some(CreateOpts { io: Some(io), .. }) => { + io.set(&mut cmd).map_err(Error::UnavailableIO)?; + let res = self.launch(cmd, true).await?; + io.close_after_start(); + Ok(res) + } + _ => self.launch(cmd, true).await, + } + } + + /// Delete a container + pub async fn delete(&self, id: &str, opts: Option<&DeleteOpts>) -> Result<()> { + let mut args = vec!["delete".to_string()]; + if let Some(opts) = opts { + args.append(&mut opts.args()); + } + args.push(id.to_string()); + let _ = self.launch(self.command(&args)?, true).await?; + Ok(()) + } + + /// Return an event stream of container notifications + pub async fn events(&self, _id: &str, _interval: &std::time::Duration) -> Result<()> { + Err(Error::Unimplemented("events".to_string())) + } + + /// Execute an additional process inside the container + pub async fn exec(&self, id: &str, spec: &Process, opts: Option<&ExecOpts>) -> Result<()> { + let f = write_value_to_temp_file(spec).await?; + let mut args = vec!["exec".to_string(), "--process".to_string(), f.clone()]; + if let Some(opts) = opts { + args.append(&mut tc!(opts.args(), &f)); + } + args.push(id.to_string()); + let mut cmd = self.command(&args)?; + match opts { + Some(ExecOpts { io: Some(io), .. }) => { + tc!( + io.set(&mut cmd).map_err(|e| Error::IoSet(e.to_string())), + &f + ); + tc!(self.launch(cmd, true).await, &f); + io.close_after_start(); + } + _ => { + tc!(self.launch(cmd, true).await, &f); + } + } + let _ = tokio::fs::remove_file(&f).await; + Ok(()) + } + + /// Send the specified signal to processes inside the container + pub async fn kill(&self, id: &str, sig: u32, opts: Option<&KillOpts>) -> Result<()> { + let mut args = vec!["kill".to_string()]; + if let Some(opts) = opts { + args.append(&mut opts.args()); + } + args.push(id.to_string()); + args.push(sig.to_string()); + let _ = self.launch(self.command(&args)?, true).await?; + Ok(()) + } + + /// List all containers associated with this runc instance + pub async fn list(&self) -> Result> { + let args = ["list".to_string(), "--format=json".to_string()]; + let res = self.launch(self.command(&args)?, true).await?; + let output = res.output.trim(); + + // Ugly hack to work around golang + Ok(if output == "null" { + Vec::new() + } else { + serde_json::from_str(output).map_err(Error::JsonDeserializationFailed)? + }) + } + + /// Pause a container + pub async fn pause(&self, id: &str) -> Result<()> { + let args = ["pause".to_string(), id.to_string()]; + let _ = self.launch(self.command(&args)?, true).await?; + Ok(()) + } + + /// Resume a container + pub async fn resume(&self, id: &str) -> Result<()> { + let args = ["resume".to_string(), id.to_string()]; + let _ = self.launch(self.command(&args)?, true).await?; + Ok(()) + } + + pub async fn checkpoint(&self) -> Result<()> { + Err(Error::Unimplemented("checkpoint".to_string())) + } + + pub async fn restore(&self) -> Result<()> { + Err(Error::Unimplemented("restore".to_string())) + } + + /// List all the processes inside the container, returning their pids + pub async fn ps(&self, id: &str) -> Result> { + let args = [ + "ps".to_string(), + "--format=json".to_string(), + id.to_string(), + ]; + let res = self.launch(self.command(&args)?, true).await?; + let output = res.output.trim(); + + // Ugly hack to work around golang + Ok(if output == "null" { + Vec::new() + } else { + serde_json::from_str(output).map_err(Error::JsonDeserializationFailed)? + }) + } + + /// Run the create, start, delete lifecycle of the container and return its exit status + pub async fn run

(&self, id: &str, bundle: P, opts: Option<&CreateOpts>) -> Result<()> + where + P: AsRef, + { + let mut args = vec![ + "run".to_string(), + "--bundle".to_string(), + utils::abs_string(bundle)?, + ]; + if let Some(opts) = opts { + args.append(&mut opts.args()?); + } + args.push(id.to_string()); + let mut cmd = self.command(&args)?; + if let Some(CreateOpts { io: Some(io), .. }) = opts { + io.set(&mut cmd).map_err(|e| Error::IoSet(e.to_string()))?; + }; + let _ = self.launch(cmd, true).await?; + Ok(()) + } + + /// Start an already created container + pub async fn start(&self, id: &str) -> Result<()> { + let args = vec!["start".to_string(), id.to_string()]; + let _ = self.launch(self.command(&args)?, true).await?; + Ok(()) + } + + /// Return the state of a container + pub async fn state(&self, id: &str) -> Result { + let args = vec!["state".to_string(), id.to_string()]; + let res = self.launch(self.command(&args)?, true).await?; + serde_json::from_str(&res.output).map_err(Error::JsonDeserializationFailed) + } + + /// Return the latest statistics for a container + pub async fn stats(&self, id: &str) -> Result { + let args = vec!["events".to_string(), "--stats".to_string(), id.to_string()]; + let res = self.launch(self.command(&args)?, true).await?; + let event: events::Event = + serde_json::from_str(&res.output).map_err(Error::JsonDeserializationFailed)?; + if let Some(stats) = event.stats { + Ok(stats) + } else { + Err(Error::MissingContainerStats) + } + } + + /// Update a container with the provided resource spec + pub async fn update(&self, id: &str, resources: &LinuxResources) -> Result<()> { + let f = write_value_to_temp_file(resources).await?; + let args = [ + "update".to_string(), + "--resources".to_string(), + f.to_string(), + id.to_string(), + ]; + let _ = tc!(self.launch(self.command(&args)?, true).await, &f); + let _ = tokio::fs::remove_file(&f).await; + Ok(()) + } +} + +#[derive(Debug)] +pub struct DefaultExecutor {} + +#[cfg(feature = "async")] +#[async_trait] +impl Spawner for DefaultExecutor { + async fn execute(&self, cmd: Command) -> Result<(ExitStatus, u32, String, String)> { + let mut cmd = cmd; + let child = cmd.spawn().map_err(Error::ProcessSpawnFailed)?; + let pid = child.id().unwrap(); + let result = child + .wait_with_output() + .await + .map_err(Error::InvalidCommand)?; + let status = result.status; + let stdout = String::from_utf8_lossy(&result.stdout).to_string(); + let stderr = String::from_utf8_lossy(&result.stderr).to_string(); + Ok((status, pid, stdout, stderr)) + } +} + +#[cfg(not(feature = "async"))] +impl Spawner for DefaultExecutor { + fn execute(&self, cmd: Command) -> Result<(ExitStatus, u32, String, String)> { + let mut cmd = cmd; + let child = cmd.spawn().map_err(Error::ProcessSpawnFailed)?; + let pid = child.id(); + let result = child.wait_with_output().map_err(Error::InvalidCommand)?; + let status = result.status; + let stdout = String::from_utf8_lossy(&result.stdout).to_string(); + let stderr = String::from_utf8_lossy(&result.stderr).to_string(); + Ok((status, pid, stdout, stderr)) + } +} + +#[cfg(test)] +#[cfg(all(target_os = "linux", not(feature = "async")))] +mod tests { + use std::sync::Arc; + + use super::{ + io::{InheritedStdIo, PipedStdIo}, + *, + }; + + fn ok_client() -> Runh { + GlobalOpts::new() + .command("/bin/true") + .build() + .expect("unable to create runh instance") + } + + fn fail_client() -> Runh { + GlobalOpts::new() + .command("/bin/false") + .build() + .expect("unable to create runh instance") + } + + fn echo_client() -> Runh { + GlobalOpts::new() + .command("/bin/echo") + .build() + .expect("unable to create runc instance") + } + + fn dummy_process() -> Process { + serde_json::from_str( + " + { + \"user\": { + \"uid\": 1000, + \"gid\": 1000 + }, + \"cwd\": \"/path/to/dir\" + }", + ) + .unwrap() + } + + #[test] + fn test_create() { + let opts = CreateOpts::new(); + let ok_runc = ok_client(); + let response = ok_runc + .create("fake-id", "fake-bundle", Some(&opts)) + .expect("true failed."); + assert_ne!(response.pid, 0); + assert!(response.status.success()); + assert!(response.output.is_empty()); + + let fail_runc = fail_client(); + match fail_runc.create("fake-id", "fake-bundle", Some(&opts)) { + Ok(_) => panic!("fail_runc returned exit status 0."), + Err(Error::CommandFailed { + status, + stdout, + stderr, + }) => { + if status.code().unwrap() == 1 && stdout.is_empty() && stderr.is_empty() { + eprintln!("fail_runc succeeded."); + } else { + panic!("unexpected outputs from fail_runc.") + } + } + Err(e) => panic!("unexpected error from fail_runc: {:?}", e), + } + } + + #[test] + fn test_run() { + let opts = CreateOpts::new(); + let ok_runc = ok_client(); + let response = ok_runc + .run("fake-id", "fake-bundle", Some(&opts)) + .expect("true failed."); + assert_ne!(response.pid, 0); + assert!(response.status.success()); + assert!(response.output.is_empty()); + + let fail_runc = fail_client(); + match fail_runc.run("fake-id", "fake-bundle", Some(&opts)) { + Ok(_) => panic!("fail_runc returned exit status 0."), + Err(Error::CommandFailed { + status, + stdout, + stderr, + }) => { + if status.code().unwrap() == 1 && stdout.is_empty() && stderr.is_empty() { + eprintln!("fail_runc succeeded."); + } else { + panic!("unexpected outputs from fail_runc.") + } + } + Err(e) => panic!("unexpected error from fail_runc: {:?}", e), + } + } + + #[test] + fn test_exec() { + let opts = ExecOpts::new(); + let ok_runc = ok_client(); + let proc = dummy_process(); + ok_runc + .exec("fake-id", &proc, Some(&opts)) + .expect("true failed."); + eprintln!("ok_runc succeeded."); + + let fail_runc = fail_client(); + match fail_runc.exec("fake-id", &proc, Some(&opts)) { + Ok(_) => panic!("fail_runc returned exit status 0."), + Err(Error::CommandFailed { + status, + stdout, + stderr, + }) => { + if status.code().unwrap() == 1 && stdout.is_empty() && stderr.is_empty() { + eprintln!("fail_runc succeeded."); + } else { + panic!("unexpected outputs from fail_runc.") + } + } + Err(e) => panic!("unexpected error from fail_runc: {:?}", e), + } + } + + #[test] + fn test_delete() { + let opts = DeleteOpts::new(); + let ok_runc = ok_client(); + ok_runc + .delete("fake-id", Some(&opts)) + .expect("true failed."); + eprintln!("ok_runc succeeded."); + + let fail_runc = fail_client(); + match fail_runc.delete("fake-id", Some(&opts)) { + Ok(_) => panic!("fail_runc returned exit status 0."), + Err(Error::CommandFailed { + status, + stdout, + stderr, + }) => { + if status.code().unwrap() == 1 && stdout.is_empty() && stderr.is_empty() { + eprintln!("fail_runc succeeded."); + } else { + panic!("unexpected outputs from fail_runc.") + } + } + Err(e) => panic!("unexpected error from fail_runc: {:?}", e), + } + } + + #[test] + fn test_output() { + // test create cmd with inherit Io, expect empty cmd output + let mut opts = CreateOpts::new(); + opts.io = Some(Arc::new(InheritedStdIo::new().unwrap())); + let echo_runc = echo_client(); + let response = echo_runc + .create("fake-id", "fake-bundle", Some(&opts)) + .expect("echo failed."); + assert_ne!(response.pid, 0); + assert!(response.status.success()); + assert!(response.output.is_empty()); + + // test create cmd with pipe Io, expect nonempty cmd output + let mut opts = CreateOpts::new(); + opts.io = Some(Arc::new(PipedStdIo::new().unwrap())); + let echo_runc = echo_client(); + let response = echo_runc + .create("fake-id", "fake-bundle", Some(&opts)) + .expect("echo failed."); + assert_ne!(response.pid, 0); + assert!(response.status.success()); + assert!(!response.output.is_empty()); + } +} + +/// Tokio tests +#[cfg(test)] +#[cfg(all(target_os = "linux", feature = "async"))] +mod tests { + use std::sync::Arc; + + use super::{ + io::{InheritedStdIo, PipedStdIo}, + *, + }; + + fn ok_client() -> Runh { + GlobalOpts::new() + .command("/bin/true") + .build() + .expect("unable to create runc instance") + } + + fn fail_client() -> Runh { + GlobalOpts::new() + .command("/bin/false") + .build() + .expect("unable to create runc instance") + } + + fn echo_client() -> Runh { + GlobalOpts::new() + .command("/bin/echo") + .build() + .expect("unable to create runc instance") + } + + #[tokio::test] + async fn test_async_create() { + let opts = CreateOpts::new(); + let ok_runc = ok_client(); + let ok_task = tokio::spawn(async move { + let response = ok_runc + .create("fake-id", "fake-bundle", Some(&opts)) + .await + .expect("true failed."); + assert_ne!(response.pid, 0); + assert!(response.status.success()); + assert!(response.output.is_empty()); + }); + + let opts = CreateOpts::new(); + let fail_runc = fail_client(); + let fail_task = tokio::spawn(async move { + match fail_runc + .create("fake-id", "fake-bundle", Some(&opts)) + .await + { + Ok(_) => panic!("fail_runc returned exit status 0."), + Err(Error::CommandFailed { + status, + stdout, + stderr, + }) => { + if status.code().unwrap() == 1 && stdout.is_empty() && stderr.is_empty() { + eprintln!("fail_runc succeeded."); + } else { + panic!("unexpected outputs from fail_runc.") + } + } + Err(e) => panic!("unexpected error from fail_runc: {:?}", e), + } + }); + + ok_task.await.expect("ok_task failed."); + fail_task.await.expect("fail_task unexpectedly succeeded."); + } + + #[tokio::test] + async fn test_async_start() { + let ok_runc = ok_client(); + let ok_task = tokio::spawn(async move { + ok_runc.start("fake-id").await.expect("true failed."); + eprintln!("ok_runc succeeded."); + }); + + let fail_runc = fail_client(); + let fail_task = tokio::spawn(async move { + match fail_runc.start("fake-id").await { + Ok(_) => panic!("fail_runc returned exit status 0."), + Err(Error::CommandFailed { + status, + stdout, + stderr, + }) => { + if status.code().unwrap() == 1 && stdout.is_empty() && stderr.is_empty() { + eprintln!("fail_runc succeeded."); + } else { + panic!("unexpected outputs from fail_runc.") + } + } + Err(e) => panic!("unexpected error from fail_runc: {:?}", e), + } + }); + + ok_task.await.expect("ok_task failed."); + fail_task.await.expect("fail_task unexpectedly succeeded."); + } + + #[tokio::test] + async fn test_async_run() { + let opts = CreateOpts::new(); + let ok_runc = ok_client(); + tokio::spawn(async move { + ok_runc + .create("fake-id", "fake-bundle", Some(&opts)) + .await + .expect("true failed."); + eprintln!("ok_runc succeeded."); + }); + + let opts = CreateOpts::new(); + let fail_runc = fail_client(); + tokio::spawn(async move { + match fail_runc + .create("fake-id", "fake-bundle", Some(&opts)) + .await + { + Ok(_) => panic!("fail_runc returned exit status 0."), + Err(Error::CommandFailed { + status, + stdout, + stderr, + }) => { + if status.code().unwrap() == 1 && stdout.is_empty() && stderr.is_empty() { + eprintln!("fail_runc succeeded."); + } else { + panic!("unexpected outputs from fail_runc.") + } + } + Err(e) => panic!("unexpected error from fail_runc: {:?}", e), + } + }) + .await + .expect("tokio spawn falied."); + } + + #[tokio::test] + async fn test_async_delete() { + let opts = DeleteOpts::new(); + let ok_runc = ok_client(); + tokio::spawn(async move { + ok_runc + .delete("fake-id", Some(&opts)) + .await + .expect("true failed."); + eprintln!("ok_runc succeeded."); + }); + + let opts = DeleteOpts::new(); + let fail_runc = fail_client(); + tokio::spawn(async move { + match fail_runc.delete("fake-id", Some(&opts)).await { + Ok(_) => panic!("fail_runc returned exit status 0."), + Err(Error::CommandFailed { + status, + stdout, + stderr, + }) => { + if status.code().unwrap() == 1 && stdout.is_empty() && stderr.is_empty() { + eprintln!("fail_runc succeeded."); + } else { + panic!("unexpected outputs from fail_runc.") + } + } + Err(e) => panic!("unexpected error from fail_runc: {:?}", e), + } + }) + .await + .expect("tokio spawn falied."); + } + + #[tokio::test] + async fn test_async_output() { + // test create cmd with inherit Io, expect empty cmd output + let mut opts = CreateOpts::new(); + opts.io = Some(Arc::new(InheritedStdIo::new().unwrap())); + let echo_runc = echo_client(); + let response = echo_runc + .create("fake-id", "fake-bundle", Some(&opts)) + .await + .expect("echo failed:"); + assert_ne!(response.pid, 0); + assert!(response.status.success()); + assert!(response.output.is_empty()); + + // test create cmd with pipe Io, expect nonempty cmd output + let mut opts = CreateOpts::new(); + opts.io = Some(Arc::new(PipedStdIo::new().unwrap())); + let response = echo_runc + .create("fake-id", "fake-bundle", Some(&opts)) + .await + .expect("echo failed:"); + assert_ne!(response.pid, 0); + assert!(response.status.success()); + assert!(!response.output.is_empty()); + } +} diff --git a/crates/runh/src/monitor.rs b/crates/runh/src/monitor.rs new file mode 100644 index 0000000..cadabc1 --- /dev/null +++ b/crates/runh/src/monitor.rs @@ -0,0 +1,180 @@ +/* + Copyright The containerd Authors. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +*/ + +use std::process::{ExitStatus, Output}; + +use async_trait::async_trait; +use log::error; +use time::OffsetDateTime; +use tokio::{ + process::Command, + sync::oneshot::{channel, Receiver, Sender}, +}; + +use crate::error::Error; + +/// A trait for spawning and waiting for a process. +/// +/// The design is different from Go's, because if you return a `Sender` in [ProcessMonitor::start()] +/// and want to use it in [ProcessMonitor::wait()], then start and wait cannot be executed +/// concurrently. Alternatively, let the caller to prepare the communication channel for +/// [ProcessMonitor::start()] and [ProcessMonitor::wait()] so they could be executed concurrently. +#[async_trait] +pub trait ProcessMonitor { + /// Spawn a process and return its output. + /// + /// In order to capture the output/error, it is necessary for the caller to create new pipes + /// between parent and child. + /// Use [tokio::process::Command::stdout(Stdio::piped())](https://docs.rs/tokio/1.16.1/tokio/process/struct.Command.html#method.stdout) + /// and/or [tokio::process::Command::stderr(Stdio::piped())](https://docs.rs/tokio/1.16.1/tokio/process/struct.Command.html#method.stderr) + /// respectively, when creating the [Command](https://docs.rs/tokio/1.16.1/tokio/process/struct.Command.html#). + async fn start(&self, mut cmd: Command, tx: Sender) -> std::io::Result { + let chi = cmd.spawn()?; + // Safe to expect() because wait() hasn't been called yet, dependence on tokio interanl + // implementation details. + let pid = chi + .id() + .expect("failed to take pid of the container process."); + let out = chi.wait_with_output().await?; + let ts = OffsetDateTime::now_utc(); + // On Unix, out.status.code() will return None if the process was terminated by a signal. + let status = out.status.code().unwrap_or(-1); + match tx.send(Exit { ts, pid, status }) { + Ok(_) => Ok(out), + Err(e) => { + error!("command {:?} exited but receiver dropped.", cmd); + error!("couldn't send messages: {:?}", e); + Err(std::io::ErrorKind::ConnectionRefused.into()) + } + } + } + + /// Wait for the spawned process to exit and return the exit status. + async fn wait(&self, rx: Receiver) -> std::io::Result { + rx.await.map_err(|_| { + error!("sender dropped."); + std::io::ErrorKind::BrokenPipe.into() + }) + } +} + +/// A default implementation of [ProcessMonitor]. +#[derive(Debug, Clone, Default)] +pub struct DefaultMonitor {} + +impl ProcessMonitor for DefaultMonitor {} + +impl DefaultMonitor { + pub const fn new() -> Self { + Self {} + } +} + +/// Process exit status returned by [ProcessMonitor::wait()]. +#[derive(Debug)] +pub struct Exit { + pub ts: OffsetDateTime, + pub pid: u32, + pub status: i32, +} + +/// Execution result returned by `execute()`. +pub struct ExecuteResult { + pub exit: Exit, + pub status: ExitStatus, + pub stdout: String, + pub stderr: String, +} + +/// Execute a `Command` and collect exit status, output and error messages. +/// +/// To collect output and error messages, pipes must be used for Command's stdout and stderr. +/// +/// Note: invalid UTF-8 characters in output and error messages will be replaced with the `�` char. +pub async fn execute( + monitor: &T, + cmd: Command, +) -> Result { + let (tx, rx) = channel::(); + let start = monitor.start(cmd, tx); + let wait = monitor.wait(rx); + let ( + Output { + stdout, + stderr, + status, + }, + exit, + ) = tokio::try_join!(start, wait).map_err(Error::InvalidCommand)?; + let stdout = String::from_utf8_lossy(&stdout).to_string(); + let stderr = String::from_utf8_lossy(&stderr).to_string(); + + Ok(ExecuteResult { + exit, + status, + stdout, + stderr, + }) +} + +#[cfg(test)] +mod tests { + use std::process::Stdio; + + use tokio::{process::Command, sync::oneshot::channel}; + + use super::*; + + #[tokio::test] + async fn test_start_wait_without_output() { + let monitor = DefaultMonitor::new(); + let cmd = Command::new("/bin/ls"); + let (tx, rx) = channel(); + + let output = monitor.start(cmd, tx).await.unwrap(); + assert_eq!(output.stdout.len(), 0); + assert_eq!(output.stderr.len(), 0); + let status = monitor.wait(rx).await.unwrap(); + assert_eq!(status.status, 0); + } + + #[tokio::test] + async fn test_start_wait_with_output() { + let monitor = DefaultMonitor::new(); + let mut cmd = Command::new("/bin/ls"); + cmd.stdout(Stdio::piped()); + let (tx, rx) = channel(); + + let output = monitor.start(cmd, tx).await.unwrap(); + assert!(!output.stdout.is_empty()); + assert_eq!(output.stderr.len(), 0); + let status = monitor.wait(rx).await.unwrap(); + assert_eq!(status.status, 0); + } + + #[tokio::test] + async fn test_execute() { + let mut cmd = Command::new("/bin/ls"); + cmd.stdout(Stdio::piped()); + let monitor = DefaultMonitor::new(); + let result = execute(&monitor, cmd).await.unwrap(); + + assert_eq!(result.exit.status, 0); + assert!(result.status.success()); + assert!(!result.stdout.is_empty()); + assert_eq!(result.stderr.len(), 0); + } +} diff --git a/crates/runh/src/options.rs b/crates/runh/src/options.rs new file mode 100644 index 0000000..d6e5465 --- /dev/null +++ b/crates/runh/src/options.rs @@ -0,0 +1,632 @@ +/* + Copyright The containerd Authors. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +*/ + +// Forked from https://github.com/pwFoo/rust-runc/blob/313e6ae5a79b54455b0a242a795c69adf035141a/src/lib.rs + +/* + * Copyright 2020 fsyncd, Berlin, Germany. + * Additional material, copyright of the containerd authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +use std::{ + path::{Path, PathBuf}, + sync::Arc, + time::Duration, +}; + +use crate::{error::Error, io::Io, utils, DefaultExecutor, LogFormat, Runh, Spawner}; + +// constants for log format +pub const JSON: &str = "json"; +pub const TEXT: &str = "text"; + +// constants for runh global flags +const DEBUG: &str = "--debug"; +const LOG: &str = "--log"; +const LOG_FORMAT: &str = "--log-format"; +const ROOT: &str = "--root"; +const ROOTLESS: &str = "--rootless"; +const SYSTEMD_CGROUP: &str = "--systemd-cgroup"; + +// constants for runh-create/runh-exec flags +const CONSOLE_SOCKET: &str = "--console-socket"; +const DETACH: &str = "--detach"; +const NO_NEW_KEYRING: &str = "--no-new-keyring"; +const NO_PIVOT: &str = "--no-pivot"; +const PID_FILE: &str = "--pid-file"; + +// constants for runh-kill flags +const ALL: &str = "--all"; + +// constants for runh-delete flags +const FORCE: &str = "--force"; + +// constant for command +pub const DEFAULT_COMMAND: &str = "runh"; + +pub trait Args { + type Output; + + fn args(&self) -> Self::Output; +} + +/// Global options builder for the runh binary. +/// +/// These options will be passed for all subsequent runh calls. +/// See +#[derive(Debug, Default)] +pub struct GlobalOpts { + /// Override the name of the runh binary. If [`None`], `runh` is used. + command: Option, + /// Debug logging. + /// + /// If true, debug level logs are emitted. + debug: bool, + /// Path to log file. + log: Option, + /// Log format to use. + log_format: LogFormat, + /// Path to root directory of container rootfs. + root: Option, + /// Whether to use rootless mode. + /// + /// If [`None`], `auto` settings is used. + /// Note that "auto" is different from explicit "true" or "false". + rootless: Option, + /// Set process group ID (gpid). + set_pgid: bool, + /// Use systemd cgroup. + systemd_cgroup: bool, + /// Timeout settings for runh command. + /// + /// Default is 5 seconds. + /// This will be used only in AsyncClient. + timeout: Duration, + /// executor that runs the commands + executor: Option>, +} + +impl GlobalOpts { + /// Create new config builder with no options. + pub fn new() -> Self { + Default::default() + } + + pub fn command(mut self, command: impl AsRef) -> Self { + self.command = Some(command.as_ref().to_path_buf()); + self + } + + /// Set the root directory to store containers' state. + /// + /// The path should be located on tmpfs. + /// Default is `/run/runh`, or `$XDG_RUNTIME_DIR/runh` for rootless containers. + pub fn root(mut self, root: impl AsRef) -> Self { + self.root = Some(root.as_ref().to_path_buf()); + self + } + + /// Enable debug logging. + pub fn debug(mut self, debug: bool) -> Self { + self.debug = debug; + self + } + + /// Set the log destination to path. + /// + /// The default is to log to stderr. + pub fn log(mut self, log: impl AsRef) -> Self { + self.log = Some(log.as_ref().to_path_buf()); + self + } + + /// Set the log format (default is text). + pub fn log_format(mut self, log_format: LogFormat) -> Self { + self.log_format = log_format; + self + } + + /// Set the log format to JSON. + pub fn log_json(self) -> Self { + self.log_format(LogFormat::Json) + } + + /// Set the log format to TEXT. + pub fn log_text(self) -> Self { + self.log_format(LogFormat::Text) + } + + /// Enable systemd cgroup support. + /// + /// If this is set, the container spec (`config.json`) is expected to have `cgroupsPath` value in + // the `slice:prefix:name` form (e.g. `system.slice:runh:434234`). + pub fn systemd_cgroup(mut self, systemd_cgroup: bool) -> Self { + self.systemd_cgroup = systemd_cgroup; + self + } + + /// Enable or disable rootless mode. + /// + // Default is auto, meaning to auto-detect whether rootless should be enabled. + pub fn rootless(mut self, rootless: bool) -> Self { + self.rootless = Some(rootless); + self + } + + /// Set rootless mode to auto. + pub fn rootless_auto(mut self) -> Self { + self.rootless = None; + self + } + + pub fn set_pgid(mut self, set_pgid: bool) -> Self { + self.set_pgid = set_pgid; + self + } + + pub fn timeout(&mut self, millis: u64) -> &mut Self { + self.timeout = Duration::from_millis(millis); + self + } + + pub fn custom_spawner(&mut self, executor: Arc) -> &mut Self { + self.executor = Some(executor); + self + } + + pub fn build(self) -> Result { + self.args() + } + + fn output(&self) -> Result<(PathBuf, Vec), Error> { + let path = self + .command + .clone() + .unwrap_or_else(|| PathBuf::from("runh")); + + let command = utils::binary_path(path).ok_or(Error::NotFound)?; + + let mut args = Vec::new(); + + // --root path : Set the root directory to store containers' state. + if let Some(root) = &self.root { + args.push(ROOT.into()); + args.push(utils::abs_string(root)?); + } + + // --debug : Enable debug logging. + if self.debug { + args.push(DEBUG.into()); + } + + // --log path : Set the log destination to path. The default is to log to stderr. + if let Some(log_path) = &self.log { + args.push(LOG.into()); + args.push(utils::abs_string(log_path)?); + } + + // --log-format text|json : Set the log format (default is text). + args.push(LOG_FORMAT.into()); + args.push(self.log_format.to_string()); + + // --systemd-cgroup : Enable systemd cgroup support. + if self.systemd_cgroup { + args.push(SYSTEMD_CGROUP.into()); + } + + // --rootless true|false|auto : Enable or disable rootless mode. + if let Some(mode) = self.rootless { + let arg = format!("{}={}", ROOTLESS, mode); + args.push(arg); + } + Ok((command, args)) + } +} + +impl Args for GlobalOpts { + type Output = Result; + + fn args(&self) -> Self::Output { + let (command, args) = self.output()?; + let executor = if let Some(exec) = self.executor.clone() { + exec + } else { + Arc::new(DefaultExecutor {}) + }; + Ok(Runh { + command, + args, + spawner: executor, + }) + } +} + +#[derive(Clone, Default)] +pub struct CreateOpts { + pub io: Option>, + /// Path to where a pid file should be created. + pub pid_file: Option, + /// Path to where a console socket should be created. + pub console_socket: Option, + /// Detach from the container's process (only available for run) + pub detach: bool, + /// Don't use pivot_root to jail process inside rootfs. + pub no_pivot: bool, + /// A new session keyring for the container will not be created. + pub no_new_keyring: bool, +} + +impl Args for CreateOpts { + type Output = Result, Error>; + + fn args(&self) -> Self::Output { + let mut args: Vec = vec![]; + if let Some(pid_file) = &self.pid_file { + args.push(PID_FILE.to_string()); + args.push(utils::abs_string(pid_file)?); + } + if let Some(console_socket) = &self.console_socket { + args.push(CONSOLE_SOCKET.to_string()); + args.push(utils::abs_string(console_socket)?); + } + if self.no_pivot { + args.push(NO_PIVOT.to_string()); + } + if self.no_new_keyring { + args.push(NO_NEW_KEYRING.to_string()); + } + if self.detach { + args.push(DETACH.to_string()); + } + Ok(args) + } +} + +impl CreateOpts { + pub fn new() -> Self { + Self::default() + } + + pub fn io(mut self, io: Arc) -> Self { + self.io = Some(io); + self + } + + pub fn pid_file

(mut self, pid_file: P) -> Self + where + P: AsRef, + { + self.pid_file = Some(pid_file.as_ref().to_path_buf()); + self + } + + pub fn console_socket

(mut self, console_socket: P) -> Self + where + P: AsRef, + { + self.console_socket = Some(console_socket.as_ref().to_path_buf()); + self + } + + pub fn detach(mut self, detach: bool) -> Self { + self.detach = detach; + self + } + + pub fn no_pivot(mut self, no_pivot: bool) -> Self { + self.no_pivot = no_pivot; + self + } + + pub fn no_new_keyring(mut self, no_new_keyring: bool) -> Self { + self.no_new_keyring = no_new_keyring; + self + } +} + +/// Container execution options +#[derive(Clone, Default)] +pub struct ExecOpts { + pub io: Option>, + /// Path to where a pid file should be created. + pub pid_file: Option, + /// Path to where a console socket should be created. + pub console_socket: Option, + /// Detach from the container's process (only available for run) + pub detach: bool, +} + +impl Args for ExecOpts { + type Output = Result, Error>; + + fn args(&self) -> Self::Output { + let mut args: Vec = vec![]; + if let Some(pid_file) = &self.pid_file { + args.push(PID_FILE.to_string()); + args.push(utils::abs_string(pid_file)?); + } + if let Some(console_socket) = &self.console_socket { + args.push(CONSOLE_SOCKET.to_string()); + args.push(utils::abs_string(console_socket)?); + } + if self.detach { + args.push(DETACH.to_string()); + } + Ok(args) + } +} + +impl ExecOpts { + pub fn new() -> Self { + Self::default() + } + + pub fn io(mut self, io: Arc) -> Self { + self.io = Some(io); + self + } + + pub fn pid_file

(mut self, pid_file: P) -> Self + where + P: AsRef, + { + self.pid_file = Some(pid_file.as_ref().to_path_buf()); + self + } + + pub fn console_socket

(mut self, console_socket: P) -> Self + where + P: AsRef, + { + self.console_socket = Some(console_socket.as_ref().to_path_buf()); + self + } + + pub fn detach(mut self, detach: bool) -> Self { + self.detach = detach; + self + } +} + +/// Container deletion options +#[derive(Debug, Clone, Default)] +pub struct DeleteOpts { + /// Forcibly delete the container if it is still running + pub force: bool, +} + +impl Args for DeleteOpts { + type Output = Vec; + + fn args(&self) -> Self::Output { + let mut args: Vec = vec![]; + if self.force { + args.push(FORCE.to_string()); + } + args + } +} + +impl DeleteOpts { + pub fn new() -> Self { + Self::default() + } + + pub fn force(mut self, force: bool) -> Self { + self.force = force; + self + } +} + +/// Container killing options +#[derive(Debug, Clone, Default)] +pub struct KillOpts { + /// Seng the kill signal to all the processes inside the container + pub all: bool, +} + +impl Args for KillOpts { + type Output = Vec; + + fn args(&self) -> Self::Output { + let mut args: Vec = vec![]; + if self.all { + args.push(ALL.to_string()); + } + args + } +} + +impl KillOpts { + pub fn new() -> Self { + Self::default() + } + + pub fn all(mut self, all: bool) -> Self { + self.all = all; + self + } +} + +#[cfg(test)] +mod tests { + use std::env; + + use super::*; + + const ARGS_FAIL_MSG: &str = "Args.args() failed."; + + #[test] + fn create_opts_test() { + assert_eq!( + CreateOpts::new().args().expect(ARGS_FAIL_MSG), + vec![String::new(); 0] + ); + + assert_eq!( + CreateOpts::new().pid_file(".").args().expect(ARGS_FAIL_MSG), + vec![ + "--pid-file".to_string(), + env::current_dir() + .unwrap() + .to_string_lossy() + .parse::() + .unwrap() + ] + ); + + assert_eq!( + CreateOpts::new() + .console_socket("..") + .args() + .expect(ARGS_FAIL_MSG), + vec![ + "--console-socket".to_string(), + env::current_dir() + .unwrap() + .parent() + .unwrap() + .to_string_lossy() + .parse::() + .unwrap() + ] + ); + + assert_eq!( + CreateOpts::new() + .detach(true) + .no_pivot(true) + .no_new_keyring(true) + .args() + .expect(ARGS_FAIL_MSG), + vec![ + "--no-pivot".to_string(), + "--no-new-keyring".to_string(), + "--detach".to_string(), + ] + ); + } + + #[test] + fn exec_opts_test() { + assert_eq!( + ExecOpts::new().args().expect(ARGS_FAIL_MSG), + vec![String::new(); 0] + ); + + assert_eq!( + ExecOpts::new().pid_file(".").args().expect(ARGS_FAIL_MSG), + vec![ + "--pid-file".to_string(), + env::current_dir() + .unwrap() + .to_string_lossy() + .parse::() + .unwrap() + ] + ); + + assert_eq!( + ExecOpts::new() + .console_socket("..") + .args() + .expect(ARGS_FAIL_MSG), + vec![ + "--console-socket".to_string(), + env::current_dir() + .unwrap() + .parent() + .unwrap() + .to_string_lossy() + .parse::() + .unwrap() + ] + ); + + assert_eq!( + ExecOpts::new().detach(true).args().expect(ARGS_FAIL_MSG), + vec!["--detach".to_string(),] + ); + } + + #[test] + fn delete_opts_test() { + assert_eq!( + DeleteOpts::new().force(false).args(), + vec![String::new(); 0] + ); + + assert_eq!( + DeleteOpts::new().force(true).args(), + vec!["--force".to_string()], + ); + } + + #[test] + fn kill_opts_test() { + assert_eq!(KillOpts::new().all(false).args(), vec![String::new(); 0]); + + assert_eq!(KillOpts::new().all(true).args(), vec!["--all".to_string()],); + } + + #[cfg(target_os = "linux")] + #[test] + fn global_opts_test() { + let cfg = GlobalOpts::default().command("true"); + let runh = cfg.build().unwrap(); + let args = &runh.args; + assert_eq!(args.len(), 2); + assert!(args.contains(&LOG_FORMAT.to_string())); + assert!(args.contains(&TEXT.to_string())); + + let cfg = GlobalOpts::default().command("/bin/true"); + let runh = cfg.build().unwrap(); + assert_eq!(runh.args.len(), 2); + + let cfg = GlobalOpts::default() + .command("true") + .root("/tmp") + .debug(true) + .log("/tmp/runh.log") + .log_json() + .systemd_cgroup(true) + .rootless(true); + let runh = cfg.build().unwrap(); + let args = &runh.args; + assert!(args.contains(&ROOT.to_string())); + assert!(args.contains(&DEBUG.to_string())); + assert!(args.contains(&"/tmp".to_string())); + assert!(args.contains(&LOG.to_string())); + assert!(args.contains(&"/tmp/runh.log".to_string())); + assert!(args.contains(&LOG_FORMAT.to_string())); + assert!(args.contains(&JSON.to_string())); + assert!(args.contains(&"--rootless=true".to_string())); + assert!(args.contains(&SYSTEMD_CGROUP.to_string())); + assert_eq!(args.len(), 9); + } +} diff --git a/crates/runh/src/utils.rs b/crates/runh/src/utils.rs new file mode 100644 index 0000000..c81e507 --- /dev/null +++ b/crates/runh/src/utils.rs @@ -0,0 +1,133 @@ +/* + Copyright The containerd Authors. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +*/ + +#[cfg(not(feature = "async"))] +use std::io::Write; +use std::{ + env, + path::{Path, PathBuf}, +}; + +use path_absolutize::*; +use serde::Serialize; +#[cfg(not(feature = "async"))] +use tempfile::{Builder, NamedTempFile}; +#[cfg(feature = "async")] +use tokio::io::AsyncWriteExt; +use uuid::Uuid; + +use crate::error::Error; + +// helper to resolve path (such as path for runh binary, pid files, etc. ) +pub fn abs_path_buf

(path: P) -> Result +where + P: AsRef, +{ + Ok(path + .as_ref() + .absolutize() + .map_err(Error::InvalidPath)? + .to_path_buf()) +} + +fn path_to_string(path: impl AsRef) -> Result { + path.as_ref() + .to_str() + .map(|v| v.to_string()) + .ok_or_else(|| { + let e = std::io::Error::new( + std::io::ErrorKind::Other, + format!("invalid UTF-8 string: {}", path.as_ref().to_string_lossy()), + ); + Error::InvalidPath(e) + }) +} + +pub fn abs_string

(path: P) -> Result +where + P: AsRef, +{ + path_to_string(abs_path_buf(path)?) +} + +/// Returns a temp dir. If the environment variable "XDG_RUNTIME_DIR" is set, return its value. +/// Otherwise if `std::env::temp_dir()` failed, return current dir or return the temp dir depended on OS. +fn xdg_runtime_dir() -> String { + env::var("XDG_RUNTIME_DIR") + .unwrap_or_else(|_| abs_string(env::temp_dir()).unwrap_or_else(|_| ".".to_string())) +} + +/// Write the serialized 'value' to a temp file +#[cfg(not(feature = "async"))] +pub fn write_value_to_temp_file(value: &T) -> Result<(NamedTempFile, String), Error> { + let filename = format!("{}/runh-process-{}", xdg_runtime_dir(), Uuid::new_v4()); + let mut temp_file = Builder::new() + .prefix(&filename) + .rand_bytes(0) + .tempfile() + .map_err(Error::SpecFileCreationFailed)?; + let f = temp_file.as_file_mut(); + let spec_json = serde_json::to_string(value).map_err(Error::JsonDeserializationFailed)?; + f.write(spec_json.as_bytes()) + .map_err(Error::SpecFileCreationFailed)?; + f.flush().map_err(Error::SpecFileCreationFailed)?; + Ok((temp_file, filename)) +} + +/// Write the serialized 'value' to a temp file +/// Unlike the same function in non-async feature, +/// it returns the filename, without the NamedTempFile object, +/// which implements Drop trait to remove the file if it goes out of scope. +/// the async Drop is still not supported in rust, +/// in async context, the created file should be removed by the caller +#[cfg(feature = "async")] +pub async fn write_value_to_temp_file(value: &T) -> Result { + let filename = format!("{}/runh-process-{}", xdg_runtime_dir(), Uuid::new_v4()); + let mut f = tokio::fs::OpenOptions::new() + .create(true) + .truncate(true) + .write(true) + .open(&filename) + .await + .map_err(Error::FileSystemError)?; + let spec_json = serde_json::to_string(value).map_err(Error::JsonDeserializationFailed)?; + f.write_all(spec_json.as_bytes()) + .await + .map_err(Error::SpecFileCreationFailed)?; + f.flush().await.map_err(Error::SpecFileCreationFailed)?; + Ok(filename) +} + +/// Resolve a binary path according to the `PATH` environment variable. +/// +/// Note, the case that `path` is already an absolute path is implicitly handled by +/// `dir.join(path.as_ref())`. `Path::join(parent_path, path)` directly returns `path` when `path` +/// is an absolute path. +pub fn binary_path

(path: P) -> Option +where + P: AsRef, +{ + env::var_os("PATH").and_then(|paths| { + env::split_paths(&paths).find_map(|dir| { + let full_path = dir.join(path.as_ref()); + if full_path.is_file() { + Some(full_path) + } else { + None + } + }) + }) +} diff --git a/src/cgroup_memory.rs b/src/cgroup_memory.rs new file mode 100644 index 0000000..0ab407f --- /dev/null +++ b/src/cgroup_memory.rs @@ -0,0 +1,202 @@ +/* + Copyright The containerd Authors. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +*/ + +#![cfg(target_os = "linux")] + +use std::{ + os::unix::io::{AsRawFd, FromRawFd}, + path::Path, +}; + +use containerd_shim::{ + error::{Error, Result}, + io_error, other_error, +}; +use nix::sys::eventfd::{EfdFlags, EventFd}; +use tokio::{ + fs::{self, read_to_string, File}, + io::AsyncReadExt, + sync::mpsc::{self, Receiver}, +}; + +pub async fn get_path_from_cgorup(pid: u32) -> Result { + let proc_path = format!("/proc/{}/cgroup", pid); + let path_string = read_to_string(&proc_path) + .await + .map_err(io_error!(e, "open {}.", &proc_path))?; + + let (_, path) = path_string + .lines() + .find(|line| line.contains("memory")) + .ok_or(Error::Other("Memory line not found".into()))? + .split_once(":memory:") + .ok_or(Error::Other("Failed to parse memory line".into()))?; + + Ok(path.to_string()) +} + +pub async fn get_existing_cgroup_mem_path(pid_path: String) -> Result<(String, String)> { + let (mut mount_root, mount_point) = get_path_from_mountinfo().await?; + if mount_root == "/" { + mount_root = String::from(""); + } + let mount_root = pid_path.trim_start_matches(&mount_root).to_string(); + Ok((mount_root, mount_point)) +} + +async fn get_path_from_mountinfo() -> Result<(String, String)> { + let mountinfo_path = "/proc/self/mountinfo"; + let mountinfo_string = + read_to_string(mountinfo_path) + .await + .map_err(io_error!(e, "open {}.", mountinfo_path))?; + + let line = mountinfo_string + .lines() + .find(|line| line.contains("cgroup") && line.contains("memory")) + .ok_or(Error::Other( + "Lines containers cgroup and memory not found in mountinfo".into(), + ))?; + + parse_memory_mountroot(line) +} + +fn parse_memory_mountroot(line: &str) -> Result<(String, String)> { + let mut columns = line.split_whitespace(); + let mount_root = columns.nth(3).ok_or(Error::Other( + "Invalid input information about mountinfo".into(), + ))?; + let mount_point = columns.next().ok_or(Error::Other( + "Invalid input information about mountinfo".into(), + ))?; + Ok((mount_root.to_string(), mount_point.to_string())) +} + +pub async fn register_memory_event( + key: &str, + cg_dir: &Path, + event_name: &str, +) -> Result> { + let path = cg_dir.join(event_name); + let event_file = fs::File::open(path.clone()) + .await + .map_err(other_error!(e, "Error get path:"))?; + + let eventfd = EventFd::from_value_and_flags(0, EfdFlags::EFD_CLOEXEC)?; + + let event_control_path = cg_dir.join("cgroup.event_control"); + let data = format!("{} {}", eventfd.as_raw_fd(), event_file.as_raw_fd()); + fs::write(&event_control_path, data.clone()) + .await + .map_err(other_error!(e, "Error write eventfd:"))?; + + let mut buf = [0u8; 8]; + + let (sender, receiver) = mpsc::channel(128); + let key = key.to_string(); + + tokio::spawn(async move { + let mut eventfd_file = unsafe { File::from_raw_fd(eventfd.as_raw_fd()) }; + loop { + match eventfd_file.read(&mut buf).await { + Ok(0) => return, + Err(_) => return, + _ => (), + } + if !Path::new(&event_control_path).exists() { + return; + } + sender.send(key.clone()).await.unwrap(); + } + }); + + Ok(receiver) +} + +#[cfg(test)] +mod tests { + use std::path::Path; + + use cgroups_rs::{ + hierarchies::{self, is_cgroup2_unified_mode}, + memory::MemController, + Cgroup, CgroupPid, + }; + use tokio::{fs::remove_file, io::AsyncWriteExt, process::Command}; + + use crate::cgroup_memory; + + #[tokio::test] + async fn test_cgroupv1_oom_monitor() { + if !is_cgroup2_unified_mode() { + // Create a memory cgroup with limits on both memory and swap. + let path = "cgroupv1_oom_monitor"; + let cg = Cgroup::new(hierarchies::auto(), path).unwrap(); + + let mem_controller: &MemController = cg.controller_of().unwrap(); + mem_controller.set_limit(10 * 1024 * 1024).unwrap(); // 10M + mem_controller.set_swappiness(0).unwrap(); + + // Create a sh sub process, and let it wait for the stdinput. + let mut child_process = Command::new("sh") + .stdin(std::process::Stdio::piped()) + .spawn() + .unwrap(); + + let pid = child_process.id().unwrap(); + + // Add the sh subprocess to the cgroup. + cg.add_task_by_tgid(CgroupPid::from(pid as u64)).unwrap(); + + // Set oom monitor + let path_from_cgorup = cgroup_memory::get_path_from_cgorup(pid).await.unwrap(); + let (mount_root, mount_point) = + cgroup_memory::get_existing_cgroup_mem_path(path_from_cgorup) + .await + .unwrap(); + + let mem_cgroup_path = mount_point + &mount_root; + let mut rx = cgroup_memory::register_memory_event( + pid.to_string().as_str(), + Path::new(&mem_cgroup_path), + "memory.oom_control", + ) + .await + .unwrap(); + + // Exec the sh subprocess to a dd command that consumes more than 10M of memory. + if let Some(mut stdin) = child_process.stdin.take() { + stdin + .write_all( + b"exec dd if=/dev/zero of=/tmp/test_oom_monitor_file bs=11M count=1\n", + ) + .await + .unwrap(); + stdin.flush().await.unwrap(); + } + + // Wait for the oom message. + if let Some(item) = rx.recv().await { + assert_eq!(pid.to_string(), item, "Receive error oom message"); + } + + // Clean. + child_process.wait().await.unwrap(); + cg.delete().unwrap(); + remove_file("/tmp/test_oom_monitor_file").await.unwrap(); + } + } +} diff --git a/src/common.rs b/src/common.rs new file mode 100644 index 0000000..d0f237c --- /dev/null +++ b/src/common.rs @@ -0,0 +1,250 @@ +/* + Copyright The containerd Authors. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +*/ + +use std::{ + env, + fs::File, + io::IoSliceMut, + ops::Deref, + os::{ + fd::{AsRawFd, FromRawFd, OwnedFd}, + unix::io::RawFd, + }, + path::Path, + sync::Arc, +}; + +use containerd_shim::{ + api::{ExecProcessRequest, Options}, + io_error, other, other_error, + util::IntoOption, + Error, +}; +use log::{debug, warn}; +use nix::{ + cmsg_space, + sys::{ + socket::{recvmsg, ControlMessageOwned, MsgFlags, UnixAddr}, + termios::tcgetattr, + }, +}; +use oci_spec::runtime::{LinuxNamespaceType, Spec}; +use runh::{ + io::{Io, NullIo, FIFO}, + options::GlobalOpts, + Runh, Spawner, +}; +use serde::Deserialize; + +use super::io::Stdio; + +pub const GROUP_LABELS: [&str; 2] = [ + "io.containerd.runh.v2.group", + "io.kubernetes.cri.sandbox-id", +]; +pub const INIT_PID_FILE: &str = "init.pid"; +pub const LOG_JSON_FILE: &str = "log.json"; +pub const FIFO_SCHEME: &str = "fifo"; + +#[derive(Deserialize)] +pub struct Log { + pub level: String, + pub msg: String, +} + +#[derive(Default)] +pub struct ProcessIO { + pub uri: Option, + pub io: Option>, + pub copy: bool, + pub stdout_r: Option, + pub stderr_r: Option, +} + +pub fn create_io( + id: &str, + _io_uid: u32, + _io_gid: u32, + stdio: &Stdio, +) -> containerd_shim::Result { + let mut pio = ProcessIO::default(); + if stdio.is_null() { + let nio = NullIo::new().map_err(io_error!(e, "new Null Io"))?; + pio.io = Some(Arc::new(nio)); + return Ok(pio); + } + let stdout = stdio.stdout.as_str(); + let scheme_path = stdout.trim().split("://").collect::>(); + let scheme: &str; + if scheme_path.len() <= 1 { + // no scheme specified, default schema to fifo + scheme = FIFO_SCHEME; + pio.uri = Some(format!("{}://{}", scheme, stdout)); + } else { + scheme = scheme_path[0]; + pio.uri = Some(stdout.to_string()); + } + + if scheme == FIFO_SCHEME { + debug!( + "create named pipe io for container {}, stdin: {}, stdout: {}, stderr: {}", + id, + stdio.stdin.as_str(), + stdio.stdout.as_str(), + stdio.stderr.as_str() + ); + let io = FIFO { + stdin: stdio.stdin.to_string().none_if(|x| x.is_empty()), + stdout: stdio.stdout.to_string().none_if(|x| x.is_empty()), + stderr: stdio.stderr.to_string().none_if(|x| x.is_empty()), + }; + pio.io = Some(Arc::new(io)); + pio.copy = false; + } + Ok(pio) +} + +#[derive(Default, Debug)] +pub struct ShimExecutor {} + +pub fn get_spec_from_request( + req: &ExecProcessRequest, +) -> containerd_shim::Result { + if let Some(val) = req.spec.as_ref() { + let mut p = serde_json::from_slice::(val.value.as_slice())?; + p.set_terminal(Some(req.terminal)); + Ok(p) + } else { + Err(Error::InvalidArgument("no spec in request".to_string())) + } +} + +pub fn check_kill_error(emsg: String) -> Error { + let emsg = emsg.to_lowercase(); + if emsg.contains("process already finished") + || emsg.contains("container not running") + || emsg.contains("no such process") + { + Error::NotFoundError("process already finished".to_string()) + } else if emsg.contains("does not exist") { + Error::NotFoundError("no such container".to_string()) + } else { + other!("unknown error after kill {}", emsg) + } +} + +const DEFAULT_RUNC_ROOT: &str = "/run/containerd/runh"; +const DEFAULT_COMMAND: &str = "runh"; + +pub fn create_runh( + runtime: &str, + namespace: &str, + bundle: impl AsRef, + opts: &Options, + spawner: Option>, +) -> containerd_shim::Result { + let runtime = if runtime.is_empty() { + DEFAULT_COMMAND + } else { + runtime + }; + let root = opts.root.as_str(); + let root = Path::new(if root.is_empty() { + DEFAULT_RUNC_ROOT + } else { + root + }) + .join(namespace); + + let log = bundle.as_ref().join(LOG_JSON_FILE); + let mut gopts = GlobalOpts::default() + .command(runtime) + .root(root) + .log(log) + .log_json() + .systemd_cgroup(opts.systemd_cgroup); + if let Some(s) = spawner { + gopts.custom_spawner(s); + } + gopts + .build() + .map_err(other_error!(e, "unable to create runh instance")) +} + +#[derive(Default)] +pub(crate) struct CreateConfig {} + +pub fn receive_socket(stream_fd: RawFd) -> containerd_shim::Result { + let mut buf = [0u8; 4096]; + let mut iovec = [IoSliceMut::new(&mut buf)]; + let mut space = cmsg_space!([RawFd; 2]); + let (path, fds) = + match recvmsg::(stream_fd, &mut iovec, Some(&mut space), MsgFlags::empty()) { + Ok(msg) => { + let iter = msg.cmsgs(); + if let Some(ControlMessageOwned::ScmRights(fds)) = iter?.next() { + (iovec[0].deref(), fds) + } else { + return Err(other!("received message is empty")); + } + } + Err(e) => { + return Err(other!("failed to receive message: {}", e)); + } + }; + if fds.is_empty() { + return Err(other!("received message is empty")); + } + let path = String::from_utf8(Vec::from(path)).unwrap_or_else(|e| { + warn!("failed to get path from array {}", e); + "".to_string() + }); + + let fd = unsafe { OwnedFd::from_raw_fd(fds[0]) }; + + let path = path.trim_matches(char::from(0)); + debug!( + "copy_console: console socket get path: {}, fd: {}", + path, + fd.as_raw_fd(), + ); + tcgetattr(&fd)?; + Ok(fd) +} + +pub fn has_shared_pid_namespace(spec: &Spec) -> bool { + match spec.linux() { + None => true, + Some(linux) => match linux.namespaces() { + None => true, + Some(namespaces) => { + for ns in namespaces { + if ns.typ() == LinuxNamespaceType::Pid && ns.path().is_none() { + return false; + } + } + true + } + }, + } +} + +/// Returns a temp dir. If the environment variable "XDG_RUNTIME_DIR" is set, return its value. +/// Otherwise if `std::env::temp_dir()` failed, return current dir or return the temp dir depended on OS. +pub(crate) fn xdg_runtime_dir() -> String { + env::var("XDG_RUNTIME_DIR") + .unwrap_or_else(|_| env::temp_dir().to_str().unwrap_or(".").to_string()) +} diff --git a/src/console.rs b/src/console.rs new file mode 100644 index 0000000..853e5e8 --- /dev/null +++ b/src/console.rs @@ -0,0 +1,74 @@ +/* + Copyright The containerd Authors. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +*/ + +use std::path::{Path, PathBuf}; + +use containerd_shim::{io_error, util::mkdir, Error, Result}; +use log::warn; +use tokio::net::{UnixListener, UnixStream}; +use uuid::Uuid; + +use crate::common::xdg_runtime_dir; + +pub struct ConsoleSocket { + pub listener: UnixListener, + pub path: PathBuf, + pub rmdir: bool, +} + +impl ConsoleSocket { + pub async fn new() -> Result { + let dir = format!("{}/pty{}", xdg_runtime_dir(), Uuid::new_v4()); + mkdir(&dir, 0o711).await?; + let file_name = Path::new(&dir).join("pty.sock"); + let listener = UnixListener::bind(&file_name).map_err(io_error!( + e, + "bind socket {}", + file_name.display() + ))?; + Ok(ConsoleSocket { + listener, + path: file_name, + rmdir: true, + }) + } + + pub async fn accept(&self) -> Result { + let (stream, _addr) = self + .listener + .accept() + .await + .map_err(io_error!(e, "failed to list console socket"))?; + Ok(stream) + } + + // async drop is not supported yet, we can only call clean manually after socket received + pub async fn clean(self) { + if self.rmdir { + if let Some(tmp_socket_dir) = self.path.parent() { + tokio::fs::remove_dir_all(tmp_socket_dir) + .await + .unwrap_or_else(|e| { + warn!( + "remove tmp console socket path {} : {}", + tmp_socket_dir.display(), + e + ) + }) + } + } + } +} diff --git a/src/container.rs b/src/container.rs new file mode 100644 index 0000000..7837dac --- /dev/null +++ b/src/container.rs @@ -0,0 +1,248 @@ +/* + Copyright The containerd Authors. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +*/ + +use std::collections::HashMap; + +use async_trait::async_trait; +use containerd_shim::{ + error::Result, + protos::{ + api::{CreateTaskRequest, ExecProcessRequest, ProcessInfo, StateResponse}, + cgroups::metrics::Metrics, + protobuf::{well_known_types::any::Any, Message, MessageDyn}, + shim::oci::ProcessDetails, + }, + Error, +}; +use log::debug; +use oci_spec::runtime::LinuxResources; +use time::OffsetDateTime; +use tokio::sync::oneshot::Receiver; + +use super::processes::Process; + +#[async_trait] +pub trait Container { + async fn start(&mut self, exec_id: Option<&str>) -> Result; + async fn state(&self, exec_id: Option<&str>) -> Result; + async fn kill(&mut self, exec_id: Option<&str>, signal: u32, all: bool) -> Result<()>; + async fn wait_channel(&mut self, exec_id: Option<&str>) -> Result>; + async fn get_exit_info( + &self, + exec_id: Option<&str>, + ) -> Result<(i32, i32, Option)>; + async fn delete( + &mut self, + exec_id_opt: Option<&str>, + ) -> Result<(i32, i32, Option)>; + async fn exec(&mut self, req: ExecProcessRequest) -> Result<()>; + async fn resize_pty(&mut self, exec_id: Option<&str>, height: u32, width: u32) -> Result<()>; + async fn pid(&self) -> i32; + async fn id(&self) -> String; + async fn update(&mut self, resources: &LinuxResources) -> Result<()>; + async fn stats(&self) -> Result; + async fn all_processes(&self) -> Result>; + async fn close_io(&mut self, exec_id: Option<&str>) -> Result<()>; +} + +#[async_trait] +pub trait ContainerFactory { + async fn create(&self, ns: &str, req: &CreateTaskRequest) -> Result; + async fn cleanup(&self, ns: &str, c: &C) -> Result<()>; +} + +#[async_trait] +pub trait ProcessFactory { + async fn create(&self, req: &ExecProcessRequest) -> Result; +} + +/// ContainerTemplate is a template struct to implement Container, +/// most of the methods can be delegated to either init process or exec process. +/// that's why we provides a ContainerTemplate struct, +/// library users only need to implements Process for their own. +pub struct ContainerTemplate { + /// container id + pub id: String, + /// container bundle path + pub bundle: String, + /// init process of this container + pub init: T, + /// process factory that create processes when exec + pub process_factory: P, + /// exec processes of this container + pub processes: HashMap, +} + +#[async_trait] +impl Container for ContainerTemplate +where + T: Process + Send + Sync, + E: Process + Send + Sync, + P: ProcessFactory + Send + Sync, +{ + async fn start(&mut self, exec_id: Option<&str>) -> Result { + let process = self.get_mut_process(exec_id)?; + process.start().await?; + Ok(process.pid().await) + } + + async fn state(&self, exec_id: Option<&str>) -> Result { + let process = self.get_process(exec_id)?; + let mut resp = process.state().await?; + resp.bundle = self.bundle.to_string(); + debug!("container state: {:?}", resp); + Ok(resp) + } + + async fn kill(&mut self, exec_id: Option<&str>, signal: u32, all: bool) -> Result<()> { + let process = self.get_mut_process(exec_id)?; + process.kill(signal, all).await + } + + async fn wait_channel(&mut self, exec_id: Option<&str>) -> Result> { + let process = self.get_mut_process(exec_id)?; + process.wait_channel().await + } + + async fn get_exit_info( + &self, + exec_id: Option<&str>, + ) -> Result<(i32, i32, Option)> { + let process = self.get_process(exec_id)?; + Ok(( + process.pid().await, + process.exit_code().await, + process.exited_at().await, + )) + } + + async fn delete( + &mut self, + exec_id_opt: Option<&str>, + ) -> Result<(i32, i32, Option)> { + let (pid, code, exited_at) = self.get_exit_info(exec_id_opt).await?; + let process = self.get_mut_process(exec_id_opt); + match process { + Ok(p) => p.delete().await?, + Err(Error::NotFoundError(_)) => return Ok((pid, code, exited_at)), + Err(e) => return Err(e), + } + if let Some(exec_id) = exec_id_opt { + self.processes.remove(exec_id); + } + Ok((pid, code, exited_at)) + } + + async fn exec(&mut self, req: ExecProcessRequest) -> Result<()> { + let exec_id = req.exec_id.to_string(); + let exec_process = self.process_factory.create(&req).await?; + self.processes.insert(exec_id, exec_process); + Ok(()) + } + + async fn resize_pty(&mut self, exec_id: Option<&str>, height: u32, width: u32) -> Result<()> { + let process = self.get_mut_process(exec_id)?; + process.resize_pty(height, width).await + } + + async fn pid(&self) -> i32 { + self.init.pid().await + } + + async fn id(&self) -> String { + self.id.to_string() + } + + #[cfg(target_os = "linux")] + async fn update(&mut self, resources: &LinuxResources) -> Result<()> { + self.init.update(resources).await + } + + #[cfg(not(target_os = "linux"))] + async fn update(&mut self, _resources: &LinuxResources) -> Result<()> { + Err(Error::Unimplemented("update".to_string())) + } + + #[cfg(target_os = "linux")] + async fn stats(&self) -> Result { + self.init.stats().await + } + + #[cfg(not(target_os = "linux"))] + async fn stats(&self) -> Result { + Err(Error::Unimplemented("stats".to_string())) + } + + async fn all_processes(&self) -> Result> { + let mut processes_info = self.init.ps().await?; + for process_info in &mut processes_info { + for (exec_id, process) in &self.processes { + if process_info.pid as i32 == process.pid().await { + let process_details = ProcessDetails { + exec_id: exec_id.to_string(), + special_fields: Default::default(), + }; + let v = Any { + type_url: process_details.descriptor_dyn().full_name().to_string(), + value: process_details.write_to_bytes()?, + special_fields: Default::default(), + }; + process_info.set_info(v); + break; + } + } + } + Ok(processes_info) + } + + async fn close_io(&mut self, exec_id: Option<&str>) -> Result<()> { + let process = self.get_mut_process(exec_id)?; + process.close_io().await + } +} + +impl ContainerTemplate +where + T: Process + Send + Sync, + E: Process + Send + Sync, +{ + pub fn get_process(&self, exec_id: Option<&str>) -> Result<&(dyn Process + Send + Sync)> { + match exec_id { + Some(exec_id) => { + let p = self.processes.get(exec_id).ok_or_else(|| { + Error::NotFoundError("can not find the exec by id".to_string()) + })?; + Ok(p) + } + None => Ok(&self.init), + } + } + + pub fn get_mut_process( + &mut self, + exec_id: Option<&str>, + ) -> Result<&mut (dyn Process + Send + Sync)> { + match exec_id { + Some(exec_id) => { + let p = self.processes.get_mut(exec_id).ok_or_else(|| { + Error::NotFoundError("can not find the exec by id".to_string()) + })?; + Ok(p) + } + None => Ok(&mut self.init), + } + } +} diff --git a/src/io.rs b/src/io.rs new file mode 100644 index 0000000..e7a4367 --- /dev/null +++ b/src/io.rs @@ -0,0 +1,38 @@ +/* + Copyright The containerd Authors. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +*/ + +#[derive(Clone, Debug, Default)] +pub struct Stdio { + pub stdin: String, + pub stdout: String, + pub stderr: String, + pub terminal: bool, +} + +impl Stdio { + pub fn new(stdin: &str, stdout: &str, stderr: &str, terminal: bool) -> Self { + Self { + stdin: stdin.to_string(), + stdout: stdout.to_string(), + stderr: stderr.to_string(), + terminal, + } + } + + pub fn is_null(&self) -> bool { + self.stdin.is_empty() && self.stdout.is_empty() && self.stderr.is_empty() + } +} diff --git a/src/main.rs b/src/main.rs new file mode 100644 index 0000000..d67f706 --- /dev/null +++ b/src/main.rs @@ -0,0 +1,56 @@ +/* + Copyright The containerd Authors. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +*/ + +use std::env; + +use containerd_shim::{asynchronous::run, parse}; + +mod cgroup_memory; +mod common; +mod console; +mod container; +mod io; +mod processes; +mod runh; +mod service; +mod task; + +use service::Service; + +fn parse_version() { + let os_args: Vec<_> = env::args_os().collect(); + let flags = match parse(&os_args[1..]) { + Ok(flags) => flags, + Err(e) => { + eprintln!("Error parsing arguments: {}", e); + std::process::exit(1); + } + }; + if flags.version { + println!("{}:", os_args[0].to_string_lossy()); + println!(" Version: {}", env!("CARGO_PKG_VERSION")); + println!(" Revision: {}", env!("CARGO_GIT_HASH")); + println!(); + + std::process::exit(0); + } +} + +#[tokio::main] +async fn main() { + parse_version(); + run::("io.containerd.runh.v2-rs", None).await; +} diff --git a/src/processes.rs b/src/processes.rs new file mode 100644 index 0000000..f0b982d --- /dev/null +++ b/src/processes.rs @@ -0,0 +1,201 @@ +/* + Copyright The containerd Authors. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +*/ + +use std::{ + os::unix::io::AsRawFd, + sync::{Arc, Mutex}, +}; + +use async_trait::async_trait; +use containerd_shim::{ + ioctl_set_winsz, + protos::{ + api::{ProcessInfo, StateResponse, Status}, + cgroups::metrics::Metrics, + protobuf::well_known_types::timestamp::Timestamp, + }, + util::asyncify, + Console, Result, +}; +use oci_spec::runtime::LinuxResources; +use time::OffsetDateTime; +use tokio::{ + fs::File, + sync::oneshot::{channel, Receiver, Sender}, +}; + +use crate::io::Stdio; + +#[async_trait] +pub trait Process { + async fn start(&mut self) -> Result<()>; + async fn set_exited(&mut self, exit_code: i32); + async fn pid(&self) -> i32; + async fn state(&self) -> Result; + async fn kill(&mut self, signal: u32, all: bool) -> Result<()>; + async fn delete(&mut self) -> Result<()>; + async fn wait_channel(&mut self) -> Result>; + async fn exit_code(&self) -> i32; + async fn exited_at(&self) -> Option; + async fn resize_pty(&mut self, height: u32, width: u32) -> Result<()>; + async fn update(&mut self, resources: &LinuxResources) -> Result<()>; + async fn stats(&self) -> Result; + async fn ps(&self) -> Result>; + async fn close_io(&mut self) -> Result<()>; +} + +#[async_trait] +pub trait ProcessLifecycle { + async fn start(&self, p: &mut P) -> Result<()>; + async fn kill(&self, p: &mut P, signal: u32, all: bool) -> Result<()>; + async fn delete(&self, p: &mut P) -> Result<()>; + async fn update(&self, p: &mut P, resources: &LinuxResources) -> Result<()>; + async fn stats(&self, p: &P) -> Result; + async fn ps(&self, p: &P) -> Result>; +} + +pub struct ProcessTemplate { + pub state: Status, + pub id: String, + pub stdio: Stdio, + pub pid: i32, + pub exit_code: i32, + pub exited_at: Option, + pub wait_chan_tx: Vec>, + pub console: Option, + pub lifecycle: Arc, + pub stdin: Arc>>, +} + +impl ProcessTemplate { + pub fn new(id: &str, stdio: Stdio, lifecycle: S) -> Self { + Self { + state: Status::CREATED, + id: id.to_string(), + stdio, + pid: 0, + exit_code: 0, + exited_at: None, + wait_chan_tx: vec![], + console: None, + lifecycle: Arc::new(lifecycle), + stdin: Arc::new(Mutex::new(None)), + } + } +} + +#[async_trait] +impl Process for ProcessTemplate +where + S: ProcessLifecycle + Sync + Send, +{ + async fn start(&mut self) -> Result<()> { + self.lifecycle.clone().start(self).await?; + Ok(()) + } + + async fn set_exited(&mut self, exit_code: i32) { + self.state = Status::STOPPED; + self.exit_code = exit_code; + self.exited_at = Some(OffsetDateTime::now_utc()); + // set wait_chan_tx to empty, to trigger the drop of the initialized Receiver. + self.wait_chan_tx = vec![]; + } + + async fn pid(&self) -> i32 { + self.pid + } + + async fn state(&self) -> Result { + let mut resp = StateResponse::new(); + resp.id = self.id.to_string(); + resp.set_status(self.state); + resp.pid = self.pid as u32; + resp.terminal = self.stdio.terminal; + resp.stdin = self.stdio.stdin.to_string(); + resp.stdout = self.stdio.stdout.to_string(); + resp.stderr = self.stdio.stderr.to_string(); + resp.exit_status = self.exit_code as u32; + if let Some(exit_at) = self.exited_at { + let mut time_stamp = Timestamp::new(); + time_stamp.seconds = exit_at.unix_timestamp(); + time_stamp.nanos = exit_at.nanosecond() as i32; + resp.exited_at = Some(time_stamp).into(); + } + Ok(resp) + } + + async fn kill(&mut self, signal: u32, all: bool) -> Result<()> { + self.lifecycle.clone().kill(self, signal, all).await + } + + async fn delete(&mut self) -> Result<()> { + self.lifecycle.clone().delete(self).await + } + + async fn wait_channel(&mut self) -> Result> { + let (tx, rx) = channel::<()>(); + if self.state != Status::STOPPED { + self.wait_chan_tx.push(tx); + } + Ok(rx) + } + + async fn exit_code(&self) -> i32 { + self.exit_code + } + + async fn exited_at(&self) -> Option { + self.exited_at + } + + async fn resize_pty(&mut self, height: u32, width: u32) -> Result<()> { + if let Some(console) = self.console.as_ref() { + let w = libc::winsize { + ws_row: height as u16, + ws_col: width as u16, + ws_xpixel: 0, + ws_ypixel: 0, + }; + let fd = console.file.as_raw_fd(); + asyncify(move || -> Result<()> { + unsafe { ioctl_set_winsz(fd, &w).map(|_x| ()).map_err(Into::into) } + }) + .await?; + } + Ok(()) + } + + async fn update(&mut self, resources: &LinuxResources) -> Result<()> { + self.lifecycle.clone().update(self, resources).await + } + + async fn stats(&self) -> Result { + self.lifecycle.stats(self).await + } + + async fn ps(&self) -> Result> { + self.lifecycle.ps(self).await + } + + async fn close_io(&mut self) -> Result<()> { + let mut lock_guard = self.stdin.lock().unwrap(); + if let Some(stdin_w_file) = lock_guard.take() { + drop(stdin_w_file); + } + Ok(()) + } +} diff --git a/src/runh.rs b/src/runh.rs new file mode 100644 index 0000000..ba30b96 --- /dev/null +++ b/src/runh.rs @@ -0,0 +1,756 @@ +/* + Copyright The containerd Authors. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +*/ + +use std::{ + convert::TryFrom, + os::{ + fd::{IntoRawFd, OwnedFd}, + unix::{ + io::{AsRawFd, FromRawFd}, + prelude::ExitStatusExt, + }, + }, + path::{Path, PathBuf}, + process::ExitStatus, + sync::{Arc, Mutex}, +}; + +use async_trait::async_trait; +use containerd_shim::{ + api::{CreateTaskRequest, ExecProcessRequest, Options, Status}, + asynchronous::monitor::{monitor_subscribe, monitor_unsubscribe, Subscription}, + io_error, + monitor::{ExitEvent, Subject, Topic}, + other, other_error, + protos::{ + api::ProcessInfo, + cgroups::metrics::Metrics, + protobuf::{CodedInputStream, Message}, + }, + util::{asyncify, mkdir, mount_rootfs, read_file_to_str, write_options, write_runtime}, + Console, Error, ExitSignal, Result, +}; +use log::{debug, error}; +use nix::{sys::signal::kill, unistd::Pid}; +use oci_spec::runtime::{LinuxResources, Process}; +use runh::{Command, Runh, Spawner}; +use tokio::{ + fs::{remove_file, File, OpenOptions}, + io::{AsyncBufReadExt, AsyncRead, AsyncReadExt, AsyncWrite, BufReader}, +}; + +use super::{ + console::ConsoleSocket, + container::{ContainerFactory, ContainerTemplate, ProcessFactory}, + processes::{ProcessLifecycle, ProcessTemplate}, +}; +use crate::{ + common::{ + check_kill_error, create_io, create_runh, get_spec_from_request, receive_socket, + CreateConfig, Log, ProcessIO, ShimExecutor, INIT_PID_FILE, LOG_JSON_FILE, + }, + io::Stdio, +}; + +pub type ExecProcess = ProcessTemplate; +pub type InitProcess = ProcessTemplate; + +pub type RunhContainer = ContainerTemplate; + +#[derive(Clone, Default)] +pub(crate) struct RunhFactory {} + +#[async_trait] +impl ContainerFactory for RunhFactory { + async fn create( + &self, + ns: &str, + req: &CreateTaskRequest, + ) -> containerd_shim::Result { + let bundle = req.bundle(); + let mut opts = Options::new(); + if let Some(any) = req.options.as_ref() { + let mut input = CodedInputStream::from_bytes(any.value.as_ref()); + opts.merge_from(&mut input)?; + } + if opts.compute_size() > 0 { + debug!("create options: {:?}", &opts); + } + let runtime = opts.binary_name.as_str(); + write_options(bundle, &opts).await?; + write_runtime(bundle, runtime).await?; + + let rootfs_vec = req.rootfs().to_vec(); + let rootfs = if !rootfs_vec.is_empty() { + let tmp_rootfs = Path::new(bundle).join("rootfs"); + mkdir(&tmp_rootfs, 0o711).await?; + tmp_rootfs + } else { + PathBuf::new() + }; + + for m in rootfs_vec { + mount_rootfs(&m, rootfs.as_path()).await? + } + + let runh = create_runh( + runtime, + ns, + bundle, + &opts, + Some(Arc::new(ShimExecutor::default())), + )?; + + let id = req.id(); + let stdio = Stdio::new(req.stdin(), req.stdout(), req.stderr(), req.terminal()); + + let mut init = InitProcess::new( + id, + stdio, + RunhInitLifecycle::new(runh.clone(), opts.clone(), bundle), + ); + + let config = CreateConfig::default(); + self.do_create(&mut init, config).await?; + let container = RunhContainer { + id: id.to_string(), + bundle: bundle.to_string(), + init, + process_factory: RunhExecFactory { + runtime: runh, + bundle: bundle.to_string(), + io_uid: opts.io_uid, + io_gid: opts.io_gid, + }, + processes: Default::default(), + }; + Ok(container) + } + + async fn cleanup(&self, _ns: &str, _c: &RunhContainer) -> containerd_shim::Result<()> { + Ok(()) + } +} + +impl RunhFactory { + async fn do_create(&self, init: &mut InitProcess, _config: CreateConfig) -> Result<()> { + let id = init.id.to_string(); + let stdio = &init.stdio; + let opts = &init.lifecycle.opts; + let bundle = &init.lifecycle.bundle; + let pid_path = Path::new(bundle).join(INIT_PID_FILE); + let mut create_opts = runh::options::CreateOpts::new() + .pid_file(&pid_path) + .no_pivot(opts.no_pivot_root) + .no_new_keyring(opts.no_new_keyring) + .detach(false); + let (socket, pio) = if stdio.terminal { + let s = ConsoleSocket::new().await?; + create_opts.console_socket = Some(s.path.to_owned()); + (Some(s), None) + } else { + let pio = create_io(&id, opts.io_uid, opts.io_gid, stdio)?; + create_opts.io = pio.io.as_ref().cloned(); + (None, Some(pio)) + }; + + let resp = init + .lifecycle + .runtime + .create(&id, bundle, Some(&create_opts)) + .await; + if let Err(e) = resp { + if let Some(s) = socket { + s.clean().await; + } + return Err(runtime_error(bundle, e, "OCI runtime create failed").await); + } + copy_io_or_console(init, socket, pio, init.lifecycle.exit_signal.clone()).await?; + let pid = read_file_to_str(pid_path).await?.parse::()?; + init.pid = pid; + Ok(()) + } +} + +// runtime_error will read the OCI runtime logfile retrieving OCI runtime error +pub async fn runtime_error(bundle: &str, e: runh::error::Error, msg: &str) -> Error { + let mut rt_msg = String::new(); + match File::open(Path::new(bundle).join(LOG_JSON_FILE)).await { + Err(err) => other!("{}: unable to open OCI runtime log file){}", msg, err), + Ok(file) => { + let mut lines = BufReader::new(file).lines(); + while let Ok(Some(line)) = lines.next_line().await { + // Retrieve the last runtime error + match serde_json::from_str::(&line) { + Err(err) => return other!("{}: unable to parse log msg: {}", msg, err), + Ok(log) => { + if log.level == "error" { + rt_msg = log.msg.trim().to_string(); + } + } + } + } + if !rt_msg.is_empty() { + other!("{}: {}", msg, rt_msg) + } else { + other!("{}: (no OCI runtime error in logfile) {}", msg, e) + } + } + } +} + +pub struct RunhExecFactory { + runtime: Runh, + bundle: String, + io_uid: u32, + io_gid: u32, +} + +#[async_trait] +impl ProcessFactory for RunhExecFactory { + async fn create(&self, req: &ExecProcessRequest) -> Result { + let p = get_spec_from_request(req)?; + Ok(ExecProcess { + state: Status::CREATED, + id: req.exec_id.to_string(), + stdio: Stdio { + stdin: req.stdin.to_string(), + stdout: req.stdout.to_string(), + stderr: req.stderr.to_string(), + terminal: req.terminal, + }, + pid: 0, + exit_code: 0, + exited_at: None, + wait_chan_tx: vec![], + console: None, + lifecycle: Arc::from(RunhExecLifecycle { + runtime: self.runtime.clone(), + bundle: self.bundle.to_string(), + container_id: req.id.to_string(), + io_uid: self.io_uid, + io_gid: self.io_gid, + spec: p, + exit_signal: Default::default(), + }), + stdin: Arc::new(Mutex::new(None)), + }) + } +} + +pub struct RunhInitLifecycle { + runtime: Runh, + opts: Options, + bundle: String, + exit_signal: Arc, +} + +#[async_trait] +impl ProcessLifecycle for RunhInitLifecycle { + async fn start(&self, p: &mut InitProcess) -> containerd_shim::Result<()> { + if let Err(e) = self.runtime.start(p.id.as_str()).await { + return Err(runtime_error(&p.lifecycle.bundle, e, "OCI runtime start failed").await); + } + p.state = Status::RUNNING; + Ok(()) + } + + async fn kill( + &self, + p: &mut InitProcess, + signal: u32, + all: bool, + ) -> containerd_shim::Result<()> { + self.runtime + .kill( + p.id.as_str(), + signal, + Some(&runh::options::KillOpts { all }), + ) + .await + .map_err(|e| check_kill_error(e.to_string())) + } + + async fn delete(&self, p: &mut InitProcess) -> containerd_shim::Result<()> { + if let Err(e) = self + .runtime + .delete( + p.id.as_str(), + Some(&runh::options::DeleteOpts { force: true }), + ) + .await + { + if !e.to_string().to_lowercase().contains("does not exist") { + return Err( + runtime_error(&p.lifecycle.bundle, e, "OCI runtime delete failed").await, + ); + } + } + self.exit_signal.signal(); + Ok(()) + } + + #[cfg(target_os = "linux")] + async fn update(&self, p: &mut InitProcess, resources: &LinuxResources) -> Result<()> { + if p.pid <= 0 { + return Err(other!( + "failed to update resources because init process is {}", + p.pid + )); + } + containerd_shim::cgroup::update_resources(p.pid as u32, resources) + } + + #[cfg(not(target_os = "linux"))] + async fn update(&self, _p: &mut InitProcess, _resources: &LinuxResources) -> Result<()> { + Err(Error::Unimplemented("update resource".to_string())) + } + + #[cfg(target_os = "linux")] + async fn stats(&self, p: &InitProcess) -> Result { + if p.pid <= 0 { + return Err(other!( + "failed to collect metrics because init process is {}", + p.pid + )); + } + containerd_shim::cgroup::collect_metrics(p.pid as u32) + } + + #[cfg(not(target_os = "linux"))] + async fn stats(&self, _p: &InitProcess) -> Result { + Err(Error::Unimplemented("process stats".to_string())) + } + + async fn ps(&self, p: &InitProcess) -> Result> { + let pids = self + .runtime + .ps(&p.id) + .await + .map_err(other_error!(e, "failed to execute runh ps"))?; + Ok(pids + .iter() + .map(|&x| ProcessInfo { + pid: x as u32, + ..Default::default() + }) + .collect()) + } +} + +impl RunhInitLifecycle { + pub fn new(runtime: Runh, opts: Options, bundle: &str) -> Self { + let work_dir = Path::new(bundle).join("work"); + let mut opts = opts; + if opts.criu_path().is_empty() { + opts.criu_path = work_dir.to_string_lossy().to_string(); + } + Self { + runtime, + opts, + bundle: bundle.to_string(), + exit_signal: Default::default(), + } + } +} + +pub struct RunhExecLifecycle { + runtime: Runh, + bundle: String, + container_id: String, + io_uid: u32, + io_gid: u32, + spec: Process, + exit_signal: Arc, +} + +#[async_trait] +impl ProcessLifecycle for RunhExecLifecycle { + async fn start(&self, p: &mut ExecProcess) -> containerd_shim::Result<()> { + let bundle = self.bundle.to_string(); + let pid_path = Path::new(&bundle).join(format!("{}.pid", &p.id)); + let mut exec_opts = runh::options::ExecOpts { + io: None, + pid_file: Some(pid_path.to_owned()), + console_socket: None, + detach: true, + }; + let (socket, pio) = if p.stdio.terminal { + let s = ConsoleSocket::new().await?; + exec_opts.console_socket = Some(s.path.to_owned()); + (Some(s), None) + } else { + let pio = create_io(&p.id, self.io_uid, self.io_gid, &p.stdio)?; + exec_opts.io = pio.io.as_ref().cloned(); + (None, Some(pio)) + }; + //TODO checkpoint support + let exec_result = self + .runtime + .exec(&self.container_id, &self.spec, Some(&exec_opts)) + .await; + if let Err(e) = exec_result { + if let Some(s) = socket { + s.clean().await; + } + return Err(runtime_error(&bundle, e, "OCI runtime exec failed").await); + } + + if !p.stdio.stdin.is_empty() { + let stdin_clone = p.stdio.stdin.clone(); + let stdin_w = p.stdin.clone(); + // Open the write side in advance to make sure read side will not block, + // open it in another thread otherwise it will block too. + tokio::spawn(async move { + if let Ok(stdin_w_file) = OpenOptions::new() + .write(true) + .open(stdin_clone.as_str()) + .await + { + let mut lock_guard = stdin_w.lock().unwrap(); + *lock_guard = Some(stdin_w_file); + } + }); + } + + copy_io_or_console(p, socket, pio, p.lifecycle.exit_signal.clone()).await?; + let pid = read_file_to_str(pid_path).await?.parse::()?; + p.pid = pid; + p.state = Status::RUNNING; + Ok(()) + } + + async fn kill( + &self, + p: &mut ExecProcess, + signal: u32, + _all: bool, + ) -> containerd_shim::Result<()> { + if p.pid <= 0 { + Err(Error::FailedPreconditionError( + "process not created".to_string(), + )) + } else if p.exited_at.is_some() { + Err(Error::NotFoundError("process already finished".to_string())) + } else { + // TODO this is kill from nix crate, it is os specific, maybe have annotated with target os + kill( + Pid::from_raw(p.pid), + nix::sys::signal::Signal::try_from(signal as i32).unwrap(), + ) + .map_err(Into::into) + } + } + + async fn delete(&self, p: &mut ExecProcess) -> Result<()> { + self.exit_signal.signal(); + let exec_pid_path = Path::new(self.bundle.as_str()).join(format!("{}.pid", p.id)); + remove_file(exec_pid_path).await.unwrap_or_default(); + Ok(()) + } + + async fn update(&self, _p: &mut ExecProcess, _resources: &LinuxResources) -> Result<()> { + Err(Error::Unimplemented("exec update".to_string())) + } + + async fn stats(&self, _p: &ExecProcess) -> Result { + Err(Error::Unimplemented("exec stats".to_string())) + } + + async fn ps(&self, _p: &ExecProcess) -> Result> { + Err(Error::Unimplemented("exec ps".to_string())) + } +} + +async fn copy_console( + console_socket: &ConsoleSocket, + stdio: &Stdio, + exit_signal: Arc, +) -> Result { + debug!("copy_console: waiting for runtime to send console fd"); + let stream = console_socket.accept().await?; + let fd = asyncify(move || -> Result { receive_socket(stream.as_raw_fd()) }).await?; + let f = unsafe { File::from_raw_fd(fd.into_raw_fd()) }; + if !stdio.stdin.is_empty() { + debug!("copy_console: pipe stdin to console"); + let console_stdin = f + .try_clone() + .await + .map_err(io_error!(e, "failed to clone console file"))?; + let stdin = OpenOptions::new() + .read(true) + .open(stdio.stdin.as_str()) + .await + .map_err(io_error!(e, "failed to open stdin"))?; + spawn_copy(stdin, console_stdin, exit_signal.clone(), None::); + } + + if !stdio.stdout.is_empty() { + let console_stdout = f + .try_clone() + .await + .map_err(io_error!(e, "failed to clone console file"))?; + debug!("copy_console: pipe stdout from console"); + let stdout = OpenOptions::new() + .write(true) + .open(stdio.stdout.as_str()) + .await + .map_err(io_error!(e, "open stdout"))?; + // open a read to make sure even if the read end of containerd shutdown, + // copy still continue until the restart of containerd succeed + let stdout_r = OpenOptions::new() + .read(true) + .open(stdio.stdout.as_str()) + .await + .map_err(io_error!(e, "open stdout for read"))?; + spawn_copy( + console_stdout, + stdout, + exit_signal, + Some(move || { + drop(stdout_r); + }), + ); + } + let console = Console { + file: f.into_std().await, + }; + Ok(console) +} + +pub async fn copy_io(pio: &ProcessIO, stdio: &Stdio, exit_signal: Arc) -> Result<()> { + if !pio.copy { + return Ok(()); + }; + if let Some(io) = &pio.io { + if let Some(w) = io.stdin() { + debug!("copy_io: pipe stdin from {}", stdio.stdin.as_str()); + if !stdio.stdin.is_empty() { + let stdin = OpenOptions::new() + .read(true) + .open(stdio.stdin.as_str()) + .await + .map_err(io_error!(e, "open stdin"))?; + spawn_copy(stdin, w, exit_signal.clone(), None::); + } + } + + if let Some(r) = io.stdout() { + debug!("copy_io: pipe stdout from to {}", stdio.stdout.as_str()); + if !stdio.stdout.is_empty() { + let stdout = OpenOptions::new() + .write(true) + .open(stdio.stdout.as_str()) + .await + .map_err(io_error!(e, "open stdout"))?; + // open a read to make sure even if the read end of containerd shutdown, + // copy still continue until the restart of containerd succeed + let stdout_r = OpenOptions::new() + .read(true) + .open(stdio.stdout.as_str()) + .await + .map_err(io_error!(e, "open stdout for read"))?; + spawn_copy( + r, + stdout, + exit_signal.clone(), + Some(move || { + drop(stdout_r); + }), + ); + } + } + + if let Some(r) = io.stderr() { + if !stdio.stderr.is_empty() { + debug!("copy_io: pipe stderr from to {}", stdio.stderr.as_str()); + let stderr = OpenOptions::new() + .write(true) + .open(stdio.stderr.as_str()) + .await + .map_err(io_error!(e, "open stderr"))?; + // open a read to make sure even if the read end of containerd shutdown, + // copy still continue until the restart of containerd succeed + let stderr_r = OpenOptions::new() + .read(true) + .open(stdio.stderr.as_str()) + .await + .map_err(io_error!(e, "open stderr for read"))?; + spawn_copy( + r, + stderr, + exit_signal, + Some(move || { + drop(stderr_r); + }), + ); + } + } + } + + Ok(()) +} + +fn spawn_copy(from: R, to: W, exit_signal: Arc, on_close: Option) +where + R: AsyncRead + Send + Unpin + 'static, + W: AsyncWrite + Send + Unpin + 'static, + F: FnOnce() + Send + 'static, +{ + let mut src = from; + let mut dst = to; + tokio::spawn(async move { + tokio::select! { + _ = exit_signal.wait() => { + debug!("container exit, copy task should exit too"); + }, + res = tokio::io::copy(&mut src, &mut dst) => { + if let Err(e) = res { + error!("copy io failed {}", e); + } + } + } + if let Some(f) = on_close { + f(); + } + }); +} + +async fn copy_io_or_console

( + p: &mut ProcessTemplate

, + socket: Option, + pio: Option, + exit_signal: Arc, +) -> Result<()> { + if p.stdio.terminal { + if let Some(console_socket) = socket { + let console_result = copy_console(&console_socket, &p.stdio, exit_signal).await; + console_socket.clean().await; + match console_result { + Ok(c) => { + p.console = Some(c); + } + Err(e) => { + return Err(e); + } + } + } + } else if let Some(pio) = pio { + copy_io(&pio, &p.stdio, exit_signal).await?; + } + Ok(()) +} + +#[async_trait] +impl Spawner for ShimExecutor { + async fn execute(&self, cmd: Command) -> runh::Result<(ExitStatus, u32, String, String)> { + let mut cmd = cmd; + let subscription = monitor_subscribe(Topic::Pid) + .await + .map_err(|e| runh::error::Error::Other(Box::new(e)))?; + let sid = subscription.id; + let child = match cmd.spawn() { + Ok(c) => c, + Err(e) => { + monitor_unsubscribe(sid).await.unwrap_or_default(); + return Err(runh::error::Error::ProcessSpawnFailed(e)); + } + }; + let pid = child.id().unwrap(); + let (stdout, stderr, exit_code) = tokio::join!( + read_std(child.stdout), + read_std(child.stderr), + wait_pid(pid as i32, subscription) + ); + let status = ExitStatus::from_raw(exit_code); + monitor_unsubscribe(sid).await.unwrap_or_default(); + Ok((status, pid, stdout, stderr)) + } +} + +async fn read_std(std: Option) -> String +where + T: AsyncRead + Unpin, +{ + let mut std = std; + if let Some(mut std) = std.take() { + let mut out = String::new(); + std.read_to_string(&mut out).await.unwrap_or_else(|e| { + error!("failed to read stdout {}", e); + 0 + }); + return out; + } + "".to_string() +} + +async fn wait_pid(pid: i32, s: Subscription) -> i32 { + let mut s = s; + loop { + if let Some(ExitEvent { + subject: Subject::Pid(epid), + exit_code: code, + }) = s.rx.recv().await + { + if pid == epid { + monitor_unsubscribe(s.id).await.unwrap_or_default(); + return code; + } + } + } +} + +#[cfg(test)] +mod tests { + use std::{os::unix::process::ExitStatusExt, path::Path, process::ExitStatus}; + + use containerd_shim::util::{mkdir, write_str_to_file}; + use runh::error::Error::CommandFailed; + use tokio::fs::remove_dir_all; + + use crate::{common::LOG_JSON_FILE, runh::runtime_error}; + + #[tokio::test] + async fn test_runtime_error() { + let empty_err = CommandFailed { + status: ExitStatus::from_raw(1), + stdout: "".to_string(), + stderr: "".to_string(), + }; + let log_json = "\ + {\"level\":\"info\",\"msg\":\"hello world\",\"time\":\"2022-11-25\"}\n\ + {\"level\":\"error\",\"msg\":\"failed error\",\"time\":\"2022-11-26\"}\n\ + {\"level\":\"error\",\"msg\":\"panic\",\"time\":\"2022-11-27\"}\n\ + "; + let test_dir = "/tmp/shim-test"; + let _ = mkdir(test_dir, 0o744).await; + write_str_to_file(Path::new(test_dir).join(LOG_JSON_FILE).as_path(), log_json) + .await + .expect("write log json should not be error"); + + let expectd_msg = "panic"; + let actual_err = runtime_error(test_dir, empty_err, "").await; + remove_dir_all(test_dir) + .await + .expect("remove test dir should not be error"); + assert!( + actual_err.to_string().contains(expectd_msg), + "actual error \"{}\" should contains \"{}\"", + actual_err, + expectd_msg + ); + } +} diff --git a/src/service.rs b/src/service.rs new file mode 100644 index 0000000..1aaaef0 --- /dev/null +++ b/src/service.rs @@ -0,0 +1,240 @@ +/* + Copyright The containerd Authors. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +*/ + +use std::{env::current_dir, sync::Arc}; + +use ::runh::options::DeleteOpts; +use async_trait::async_trait; +use containerd_shim::{ + asynchronous::{ + monitor::{monitor_subscribe, monitor_unsubscribe, Subscription}, + publisher::RemotePublisher, + spawn, ExitSignal, Shim, + }, + event::Event, + io_error, + monitor::{Subject, Topic}, + protos::{events::task::TaskExit, protobuf::MessageDyn}, + util::{ + convert_to_timestamp, read_options, read_pid_from_file, read_runtime, read_spec, timestamp, + write_str_to_file, + }, + Config, Context, DeleteResponse, Error, Flags, StartOpts, +}; +use log::{debug, error, warn}; +use tokio::sync::mpsc::{channel, Receiver, Sender}; + +use crate::{ + common::{create_runh, has_shared_pid_namespace, ShimExecutor, GROUP_LABELS, INIT_PID_FILE}, + container::Container, + processes::Process, + runh::{RunhContainer, RunhFactory}, + task::TaskService, +}; + +pub(crate) struct Service { + exit: Arc, + id: String, + namespace: String, +} + +#[async_trait] +impl Shim for Service { + type T = TaskService; + + async fn new(_runtime_id: &str, args: &Flags, _config: &mut Config) -> Self { + let exit = Arc::new(ExitSignal::default()); + // TODO: add publisher + Service { + exit, + id: args.id.to_string(), + namespace: args.namespace.to_string(), + } + } + + async fn start_shim(&mut self, opts: StartOpts) -> containerd_shim::Result { + let mut grouping = opts.id.clone(); + let spec = read_spec("").await?; + match spec.annotations() { + Some(annotations) => { + for &label in GROUP_LABELS.iter() { + if let Some(value) = annotations.get(label) { + grouping = value.to_string(); + break; + } + } + } + None => {} + } + #[cfg(not(target_os = "linux"))] + let thp_disabled = String::new(); + #[cfg(target_os = "linux")] + // Our goal is to set thp disable = true on the shim side and then restore thp + // disable before starting runh. So we only need to focus on the return value + // of the function get_thp_disabled, which is Result. + let thp_disabled = match prctl::get_thp_disable() { + Ok(x) => { + // The return value of the function set_thp_disabled is Result<(), i32>, + // we don't care if the setting is successful, because even if the + // setting failed, we should not exit the shim process, therefore, + // there is no need to pay attention to the set_thp_disabled function's + // return value. + let _ = prctl::set_thp_disable(true); + x.to_string() + } + Err(_) => String::new(), + }; + let vars: Vec<(&str, &str)> = vec![("THP_DISABLED", thp_disabled.as_str())]; + + let address = spawn(opts, &grouping, vars).await?; + write_str_to_file("address", &address).await?; + Ok(address) + } + + async fn delete_shim(&mut self) -> containerd_shim::Result { + let namespace = self.namespace.as_str(); + let bundle = current_dir().map_err(io_error!(e, "get current dir"))?; + let opts = read_options(&bundle).await?; + let runtime = read_runtime(&bundle).await?; + + let runh = create_runh( + &runtime, + namespace, + &bundle, + &opts, + Some(Arc::new(ShimExecutor::default())), + )?; + let pid = read_pid_from_file(&bundle.join(INIT_PID_FILE)).await?; + + runh.delete(&self.id, Some(&DeleteOpts { force: true })) + .await + .unwrap_or_else(|e| warn!("failed to remove runh container: {}", e)); + let mut resp = DeleteResponse::new(); + // sigkill + resp.set_exit_status(137); + resp.set_exited_at(timestamp()?); + resp.set_pid(pid as u32); + Ok(resp) + } + + async fn wait(&mut self) { + self.exit.wait().await; + } + + async fn create_task_service(&self, publisher: RemotePublisher) -> Self::T { + let (tx, rx) = channel(128); + let exit_clone = self.exit.clone(); + let task = TaskService::new(&self.namespace, exit_clone, tx.clone()); + let s = monitor_subscribe(Topic::Pid) + .await + .expect("monitor subscribe failed"); + process_exits(s, &task, tx).await; + forward(publisher, self.namespace.to_string(), rx).await; + task + } +} + +async fn process_exits( + s: Subscription, + task: &TaskService, + tx: Sender<(String, Box)>, +) { + let containers = task.containers.clone(); + let mut s = s; + tokio::spawn(async move { + while let Some(e) = s.rx.recv().await { + if let Subject::Pid(pid) = e.subject { + debug!("receive exit event: {}", &e); + let exit_code = e.exit_code; + for (_k, cont) in containers.lock().await.iter_mut() { + let bundle = cont.bundle.to_string(); + // pid belongs to container init process + if cont.init.pid == pid { + // kill all children process if the container has a private PID namespace + if should_kill_all_on_exit(&bundle).await { + cont.kill(None, 9, true).await.unwrap_or_else(|e| { + error!("failed to kill init's children: {}", e) + }); + } + // set exit for init process + cont.init.set_exited(exit_code).await; + + // publish event + let (_, code, exited_at) = match cont.get_exit_info(None).await { + Ok(info) => info, + Err(_) => break, + }; + + let ts = convert_to_timestamp(exited_at); + let event = TaskExit { + container_id: cont.id.to_string(), + id: cont.id.to_string(), + pid: cont.pid().await as u32, + exit_status: code as u32, + exited_at: Some(ts).into(), + ..Default::default() + }; + let topic = event.topic(); + tx.send((topic.to_string(), Box::new(event))) + .await + .unwrap_or_else(|e| warn!("send {} to publisher: {}", topic, e)); + + break; + } + + // pid belongs to container common process + for (_exec_id, p) in cont.processes.iter_mut() { + // set exit for exec process + if p.pid == pid { + p.set_exited(exit_code).await; + // TODO: publish event + break; + } + } + } + } + } + monitor_unsubscribe(s.id).await.unwrap_or_default(); + }); +} + +async fn forward( + publisher: RemotePublisher, + ns: String, + mut rx: Receiver<(String, Box)>, +) { + tokio::spawn(async move { + while let Some((topic, e)) = rx.recv().await { + publisher + .publish(Context::default(), &topic, &ns, e) + .await + .unwrap_or_else(|e| warn!("publish {} to containerd: {}", topic, e)); + } + }); +} + +async fn should_kill_all_on_exit(bundle_path: &str) -> bool { + match read_spec(bundle_path).await { + Ok(spec) => has_shared_pid_namespace(&spec), + Err(e) => { + error!( + "failed to read spec when call should_kill_all_on_exit: {}", + e + ); + false + } + } +} diff --git a/src/task.rs b/src/task.rs new file mode 100644 index 0000000..3e677c9 --- /dev/null +++ b/src/task.rs @@ -0,0 +1,423 @@ +/* + Copyright The containerd Authors. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +*/ +use std::{collections::HashMap, sync::Arc}; + +use async_trait::async_trait; +use containerd_shim::{ + api::{ + CreateTaskRequest, CreateTaskResponse, DeleteRequest, Empty, ExecProcessRequest, + KillRequest, ResizePtyRequest, ShutdownRequest, StartRequest, StartResponse, StateRequest, + StateResponse, Status, WaitRequest, WaitResponse, + }, + asynchronous::ExitSignal, + event::Event, + protos::{ + api::{ + CloseIORequest, ConnectRequest, ConnectResponse, DeleteResponse, PidsRequest, + PidsResponse, StatsRequest, StatsResponse, UpdateTaskRequest, + }, + events::task::{TaskCreate, TaskDelete, TaskExecAdded, TaskExecStarted, TaskIO, TaskStart}, + protobuf::MessageDyn, + shim_async::Task, + ttrpc, + ttrpc::r#async::TtrpcContext, + }, + util::{convert_to_any, convert_to_timestamp, AsOption}, + TtrpcResult, +}; +use log::{debug, info, warn}; +use oci_spec::runtime::LinuxResources; +use tokio::sync::{mpsc::Sender, MappedMutexGuard, Mutex, MutexGuard}; + +use super::container::{Container, ContainerFactory}; +type EventSender = Sender<(String, Box)>; + +#[cfg(target_os = "linux")] +use std::path::Path; + +#[cfg(target_os = "linux")] +use cgroups_rs::hierarchies::is_cgroup2_unified_mode; +#[cfg(target_os = "linux")] +use containerd_shim::{ + error::{Error, Result}, + other_error, + protos::events::task::TaskOOM, +}; +#[cfg(target_os = "linux")] +use log::error; +#[cfg(target_os = "linux")] +use tokio::{sync::mpsc::Receiver, task::spawn}; + +#[cfg(target_os = "linux")] +use crate::cgroup_memory; + +/// TaskService is a Task template struct, it is considered a helper struct, +/// which has already implemented `Task` trait, so that users can make it the type `T` +/// parameter of `Service`, and implements their own `ContainerFactory` and `Container`. +pub struct TaskService { + pub factory: F, + pub containers: Arc>>, + pub namespace: String, + pub exit: Arc, + pub tx: EventSender, +} + +impl TaskService +where + F: Default, +{ + pub fn new(ns: &str, exit: Arc, tx: EventSender) -> Self { + Self { + factory: Default::default(), + containers: Arc::new(Mutex::new(Default::default())), + namespace: ns.to_string(), + exit, + tx, + } + } +} + +impl TaskService { + pub async fn get_container(&self, id: &str) -> TtrpcResult> { + let mut containers = self.containers.lock().await; + containers.get_mut(id).ok_or_else(|| { + ttrpc::Error::RpcStatus(ttrpc::get_status( + ttrpc::Code::NOT_FOUND, + format!("can not find container by id {}", id), + )) + })?; + let container = MutexGuard::map(containers, |m| m.get_mut(id).unwrap()); + Ok(container) + } + + pub async fn send_event(&self, event: impl Event) { + let topic = event.topic(); + self.tx + .send((topic.to_string(), Box::new(event))) + .await + .unwrap_or_else(|e| warn!("send {} to publisher: {}", topic, e)); + } +} + +#[cfg(target_os = "linux")] +fn run_oom_monitor(mut rx: Receiver, id: String, tx: EventSender) { + let oom_event = TaskOOM { + container_id: id, + ..Default::default() + }; + let topic = oom_event.topic(); + let oom_box = Box::new(oom_event); + spawn(async move { + while let Some(_item) = rx.recv().await { + tx.send((topic.to_string(), oom_box.clone())) + .await + .unwrap_or_else(|e| warn!("send {} to publisher: {}", topic, e)); + } + }); +} + +#[cfg(target_os = "linux")] +async fn monitor_oom(id: &String, pid: u32, tx: EventSender) -> Result<()> { + if !is_cgroup2_unified_mode() { + let path_from_cgorup = cgroup_memory::get_path_from_cgorup(pid).await?; + let (mount_root, mount_point) = + cgroup_memory::get_existing_cgroup_mem_path(path_from_cgorup).await?; + + let mem_cgroup_path = mount_point + &mount_root; + let rx = cgroup_memory::register_memory_event( + id, + Path::new(&mem_cgroup_path), + "memory.oom_control", + ) + .await + .map_err(other_error!(e, "register_memory_event failed:"))?; + + run_oom_monitor(rx, id.to_string(), tx); + } + Ok(()) +} + +#[async_trait] +impl Task for TaskService +where + F: ContainerFactory + Sync + Send, + C: Container + Sync + Send + 'static, +{ + async fn state(&self, _ctx: &TtrpcContext, req: StateRequest) -> TtrpcResult { + let container = self.get_container(req.id()).await?; + let exec_id = req.exec_id().as_option(); + let resp = container.state(exec_id).await?; + Ok(resp) + } + + async fn create( + &self, + _ctx: &TtrpcContext, + req: CreateTaskRequest, + ) -> TtrpcResult { + info!("Create request for {:?}", &req); + // Note: Get containers here is for getting the lock, + // to make sure no other threads manipulate the containers metadata; + let mut containers = self.containers.lock().await; + + let ns = self.namespace.as_str(); + let id = req.id.as_str(); + + let container = self.factory.create(ns, &req).await?; + let mut resp = CreateTaskResponse::new(); + let pid = container.pid().await as u32; + resp.pid = pid; + + containers.insert(id.to_string(), container); + + self.send_event(TaskCreate { + container_id: req.id.to_string(), + bundle: req.bundle.to_string(), + rootfs: req.rootfs, + io: Some(TaskIO { + stdin: req.stdin.to_string(), + stdout: req.stdout.to_string(), + stderr: req.stderr.to_string(), + terminal: req.terminal, + ..Default::default() + }) + .into(), + checkpoint: req.checkpoint.to_string(), + pid, + ..Default::default() + }) + .await; + info!("Create request for {} returns pid {}", id, resp.pid); + Ok(resp) + } + + async fn start(&self, _ctx: &TtrpcContext, req: StartRequest) -> TtrpcResult { + info!("Start request for {:?}", &req); + let mut container = self.get_container(req.id()).await?; + let pid = container.start(req.exec_id.as_str().as_option()).await?; + + let mut resp = StartResponse::new(); + resp.pid = pid as u32; + + if req.exec_id.is_empty() { + self.send_event(TaskStart { + container_id: req.id.to_string(), + pid: pid as u32, + ..Default::default() + }) + .await; + #[cfg(target_os = "linux")] + if let Err(e) = monitor_oom(&req.id, resp.pid, self.tx.clone()).await { + error!("monitor_oom failed: {:?}.", e); + } + } else { + self.send_event(TaskExecStarted { + container_id: req.id.to_string(), + exec_id: req.exec_id.to_string(), + pid: pid as u32, + ..Default::default() + }) + .await; + }; + + info!("Start request for {:?} returns pid {}", req, resp.pid()); + Ok(resp) + } + + async fn delete(&self, _ctx: &TtrpcContext, req: DeleteRequest) -> TtrpcResult { + info!("Delete request for {:?}", &req); + let mut containers = self.containers.lock().await; + let container = containers.get_mut(req.id()).ok_or_else(|| { + ttrpc::Error::RpcStatus(ttrpc::get_status( + ttrpc::Code::NOT_FOUND, + format!("can not find container by id {}", req.id()), + )) + })?; + let id = container.id().await; + let exec_id_opt = req.exec_id().as_option(); + let (pid, exit_status, exited_at) = container.delete(exec_id_opt).await?; + self.factory.cleanup(&self.namespace, container).await?; + if req.exec_id().is_empty() { + containers.remove(req.id()); + } + + let ts = convert_to_timestamp(exited_at); + self.send_event(TaskDelete { + container_id: id, + pid: pid as u32, + exit_status: exit_status as u32, + exited_at: Some(ts.clone()).into(), + ..Default::default() + }) + .await; + + let mut resp = DeleteResponse::new(); + resp.set_exited_at(ts); + resp.set_pid(pid as u32); + resp.set_exit_status(exit_status as u32); + info!( + "Delete request for {} {} returns {:?}", + req.id(), + req.exec_id(), + resp + ); + Ok(resp) + } + + async fn pids(&self, _ctx: &TtrpcContext, req: PidsRequest) -> TtrpcResult { + debug!("Pids request for {:?}", req); + let container = self.get_container(req.id()).await?; + let processes = container.all_processes().await?; + debug!("Pids request for {:?} returns successfully", req); + Ok(PidsResponse { + processes, + ..Default::default() + }) + } + + async fn kill(&self, _ctx: &TtrpcContext, req: KillRequest) -> TtrpcResult { + info!("Kill request for {:?}", req); + let mut container = self.get_container(req.id()).await?; + container + .kill(req.exec_id().as_option(), req.signal, req.all) + .await?; + info!("Kill request for {:?} returns successfully", req); + Ok(Empty::new()) + } + + async fn exec(&self, _ctx: &TtrpcContext, req: ExecProcessRequest) -> TtrpcResult { + info!("Exec request for {:?}", req); + let exec_id = req.exec_id().to_string(); + let mut container = self.get_container(req.id()).await?; + container.exec(req).await?; + + self.send_event(TaskExecAdded { + container_id: container.id().await, + exec_id, + ..Default::default() + }) + .await; + + Ok(Empty::new()) + } + + async fn resize_pty(&self, _ctx: &TtrpcContext, req: ResizePtyRequest) -> TtrpcResult { + debug!( + "Resize pty request for container {}, exec_id: {}", + &req.id, &req.exec_id + ); + let mut container = self.get_container(req.id()).await?; + container + .resize_pty(req.exec_id().as_option(), req.height, req.width) + .await?; + Ok(Empty::new()) + } + + async fn close_io(&self, _ctx: &TtrpcContext, req: CloseIORequest) -> TtrpcResult { + let mut container = self.get_container(req.id()).await?; + container.close_io(req.exec_id().as_option()).await?; + Ok(Empty::new()) + } + + async fn update(&self, _ctx: &TtrpcContext, mut req: UpdateTaskRequest) -> TtrpcResult { + debug!("Update request for {:?}", req); + + let id = req.take_id(); + + let data = req + .resources + .into_option() + .map(|r| r.value) + .unwrap_or_default(); + + let resources: LinuxResources = serde_json::from_slice(&data).map_err(|e| { + ttrpc::Error::RpcStatus(ttrpc::get_status( + ttrpc::Code::INVALID_ARGUMENT, + format!("failed to parse resource spec: {}", e), + )) + })?; + + let mut container = self.get_container(&id).await?; + container.update(&resources).await?; + Ok(Empty::new()) + } + + async fn wait(&self, _ctx: &TtrpcContext, req: WaitRequest) -> TtrpcResult { + info!("Wait request for {:?}", req); + let exec_id = req.exec_id.as_str().as_option(); + let wait_rx = { + let mut container = self.get_container(req.id()).await?; + let state = container.state(exec_id).await?; + if state.status() != Status::RUNNING && state.status() != Status::CREATED { + let mut resp = WaitResponse::new(); + resp.exit_status = state.exit_status; + resp.exited_at = state.exited_at; + info!("Wait request for {:?} returns {:?}", req, &resp); + return Ok(resp); + } + container.wait_channel(req.exec_id().as_option()).await? + }; + + wait_rx.await.unwrap_or_default(); + // get lock again. + let container = self.get_container(req.id()).await?; + let (_, code, exited_at) = container.get_exit_info(exec_id).await?; + let mut resp = WaitResponse::new(); + resp.set_exit_status(code as u32); + let ts = convert_to_timestamp(exited_at); + resp.set_exited_at(ts); + info!("Wait request for {:?} returns {:?}", req, &resp); + Ok(resp) + } + + async fn stats(&self, _ctx: &TtrpcContext, req: StatsRequest) -> TtrpcResult { + debug!("Stats request for {:?}", req); + let container = self.get_container(req.id()).await?; + let stats = container.stats().await?; + + let mut resp = StatsResponse::new(); + resp.set_stats(convert_to_any(Box::new(stats))?); + Ok(resp) + } + + async fn connect( + &self, + _ctx: &TtrpcContext, + req: ConnectRequest, + ) -> TtrpcResult { + info!("Connect request for {:?}", req); + let mut pid: u32 = 0; + if let Ok(container) = self.get_container(req.id()).await { + pid = container.pid().await as u32; + } + + Ok(ConnectResponse { + shim_pid: std::process::id(), + task_pid: pid, + ..Default::default() + }) + } + + async fn shutdown(&self, _ctx: &TtrpcContext, _req: ShutdownRequest) -> TtrpcResult { + debug!("Shutdown request"); + let containers = self.containers.lock().await; + if containers.len() > 0 { + return Ok(Empty::new()); + } + self.exit.signal(); + Ok(Empty::default()) + } +}