Merge pull request #6639 from openanolis/nerdctl

runtime-rs: enable nerdctl to setup cni plugin
This commit is contained in:
Bin Liu
2023-04-12 12:04:37 +08:00
committed by GitHub
13 changed files with 263 additions and 86 deletions

View File

@@ -192,11 +192,23 @@ pub struct Hook {
pub struct Hooks { pub struct Hooks {
#[serde(default, skip_serializing_if = "Vec::is_empty")] #[serde(default, skip_serializing_if = "Vec::is_empty")]
pub prestart: Vec<Hook>, pub prestart: Vec<Hook>,
#[serde(default, skip_serializing_if = "Vec::is_empty")] #[serde(
rename = "createRuntime",
default,
skip_serializing_if = "Vec::is_empty"
)]
pub create_runtime: Vec<Hook>, pub create_runtime: Vec<Hook>,
#[serde(default, skip_serializing_if = "Vec::is_empty")] #[serde(
rename = "createContainer",
default,
skip_serializing_if = "Vec::is_empty"
)]
pub create_container: Vec<Hook>, pub create_container: Vec<Hook>,
#[serde(default, skip_serializing_if = "Vec::is_empty")] #[serde(
rename = "startContainer",
default,
skip_serializing_if = "Vec::is_empty"
)]
pub start_container: Vec<Hook>, pub start_container: Vec<Hook>,
#[serde(default, skip_serializing_if = "Vec::is_empty")] #[serde(default, skip_serializing_if = "Vec::is_empty")]
pub poststart: Vec<Hook>, pub poststart: Vec<Hook>,
@@ -837,6 +849,8 @@ pub struct State {
#[cfg(test)] #[cfg(test)]
mod tests { mod tests {
use std::vec;
use super::*; use super::*;
#[test] #[test]
@@ -1027,6 +1041,11 @@ mod tests {
"path": "/usr/bin/setup-network" "path": "/usr/bin/setup-network"
} }
], ],
"createRuntime": [
{
"path": "/usr/local/bin/nerdctl"
}
],
"poststart": [ "poststart": [
{ {
"path": "/usr/bin/notify-start", "path": "/usr/bin/notify-start",
@@ -1395,6 +1414,12 @@ mod tests {
timeout: None, timeout: None,
}, },
], ],
create_runtime: vec![crate::Hook {
path: "/usr/local/bin/nerdctl".to_string(),
args: vec![],
env: vec![],
timeout: None,
}],
poststart: vec![crate::Hook { poststart: vec![crate::Hook {
path: "/usr/bin/notify-start".to_string(), path: "/usr/bin/notify-start".to_string(),
args: vec![], args: vec![],

View File

@@ -1734,6 +1734,16 @@ dependencies = [
"tokio", "tokio",
] ]
[[package]]
name = "netns-rs"
version = "0.1.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "23541694f1d7d18cd1a0da3a1352a6ea48b01cbb4a8e7a6e547963823fd5276e"
dependencies = [
"nix 0.23.2",
"thiserror",
]
[[package]] [[package]]
name = "nix" name = "nix"
version = "0.23.2" version = "0.23.2"
@@ -2401,6 +2411,7 @@ dependencies = [
"byte-unit 4.0.18", "byte-unit 4.0.18",
"cgroups-rs", "cgroups-rs",
"futures 0.3.26", "futures 0.3.26",
"hex",
"hypervisor", "hypervisor",
"kata-sys-util", "kata-sys-util",
"kata-types", "kata-types",
@@ -2409,6 +2420,7 @@ dependencies = [
"logging", "logging",
"netlink-packet-route", "netlink-packet-route",
"netlink-sys", "netlink-sys",
"netns-rs",
"nix 0.24.3", "nix 0.24.3",
"oci", "oci",
"persist", "persist",
@@ -2464,9 +2476,11 @@ dependencies = [
"lazy_static", "lazy_static",
"linux_container", "linux_container",
"logging", "logging",
"netns-rs",
"nix 0.25.1", "nix 0.25.1",
"oci", "oci",
"persist", "persist",
"resource",
"serde_json", "serde_json",
"shim-interface", "shim-interface",
"slog", "slog",

View File

@@ -16,8 +16,10 @@ bitflags = "1.2.1"
byte-unit = "4.0.14" byte-unit = "4.0.14"
cgroups-rs = "0.3.2" cgroups-rs = "0.3.2"
futures = "0.3.11" futures = "0.3.11"
hex = "0.4.3"
lazy_static = "1.4.0" lazy_static = "1.4.0"
libc = ">=0.2.39" libc = ">=0.2.39"
netns-rs = "0.1.0"
netlink-sys = "0.8.3" netlink-sys = "0.8.3"
netlink-packet-route = "0.13.0" netlink-packet-route = "0.13.0"
nix = "0.24.2" nix = "0.24.2"

View File

@@ -4,6 +4,7 @@
// SPDX-License-Identifier: Apache-2.0 // SPDX-License-Identifier: Apache-2.0
// //
use crate::network::NetworkConfig;
use crate::resource_persist::ResourceState; use crate::resource_persist::ResourceState;
use crate::{manager_inner::ResourceManagerInner, rootfs::Rootfs, volume::Volume, ResourceConfig}; use crate::{manager_inner::ResourceManagerInner, rootfs::Rootfs, volume::Volume, ResourceConfig};
use agent::{Agent, Storage}; use agent::{Agent, Storage};
@@ -55,6 +56,11 @@ impl ResourceManager {
inner.prepare_before_start_vm(device_configs).await inner.prepare_before_start_vm(device_configs).await
} }
pub async fn handle_network(&self, network_config: NetworkConfig) -> Result<()> {
let mut inner = self.inner.write().await;
inner.handle_network(network_config).await
}
pub async fn setup_after_start_vm(&self) -> Result<()> { pub async fn setup_after_start_vm(&self) -> Result<()> {
let mut inner = self.inner.write().await; let mut inner = self.inner.write().await;
inner.setup_after_start_vm().await inner.setup_after_start_vm().await

View File

@@ -6,7 +6,7 @@
use std::{sync::Arc, thread}; use std::{sync::Arc, thread};
use crate::resource_persist::ResourceState; use crate::{network::NetworkConfig, resource_persist::ResourceState};
use agent::{Agent, Storage}; use agent::{Agent, Storage};
use anyhow::{anyhow, Context, Ok, Result}; use anyhow::{anyhow, Context, Ok, Result};
use async_trait::async_trait; use async_trait::async_trait;
@@ -89,32 +89,9 @@ impl ResourceManagerInner {
}; };
} }
ResourceConfig::Network(c) => { ResourceConfig::Network(c) => {
// 1. When using Rust asynchronous programming, we use .await to self.handle_network(c)
// allow other task to run instead of waiting for the completion of the current task. .await
// 2. Also, when handling the pod network, we need to set the shim threads .context("failed to handle network")?;
// into the network namespace to perform those operations.
// However, as the increase of the I/O intensive tasks, two issues could be caused by the two points above:
// a. When the future is blocked, the current thread (which is in the pod netns)
// might be take over by other tasks. After the future is finished, the thread take over
// the current task might not be in the pod netns. But the current task still need to run in pod netns
// b. When finish setting up the network, the current thread will be set back to the host namespace.
// In Rust Async, if the current thread is taken over by other task, the netns is dropped on another thread,
// but it is not in netns. So, the previous thread would still remain in the pod netns.
// The solution is to block the future on the current thread, it is enabled by spawn an os thread, create a
// tokio runtime, and block the task on it.
let hypervisor = self.hypervisor.clone();
let network = thread::spawn(move || -> Result<Arc<dyn Network>> {
let rt = runtime::Builder::new_current_thread().enable_io().build()?;
let d = rt.block_on(network::new(&c)).context("new network")?;
rt.block_on(d.setup(hypervisor.as_ref()))
.context("setup network")?;
Ok(d)
})
.join()
.map_err(|e| anyhow!("{:?}", e))
.context("Couldn't join on the associated thread")?
.context("failed to set up network")?;
self.network = Some(network);
} }
}; };
} }
@@ -122,6 +99,38 @@ impl ResourceManagerInner {
Ok(()) Ok(())
} }
pub async fn handle_network(&mut self, network_config: NetworkConfig) -> Result<()> {
// 1. When using Rust asynchronous programming, we use .await to
// allow other task to run instead of waiting for the completion of the current task.
// 2. Also, when handling the pod network, we need to set the shim threads
// into the network namespace to perform those operations.
// However, as the increase of the I/O intensive tasks, two issues could be caused by the two points above:
// a. When the future is blocked, the current thread (which is in the pod netns)
// might be take over by other tasks. After the future is finished, the thread take over
// the current task might not be in the pod netns. But the current task still need to run in pod netns
// b. When finish setting up the network, the current thread will be set back to the host namespace.
// In Rust Async, if the current thread is taken over by other task, the netns is dropped on another thread,
// but it is not in netns. So, the previous thread would still remain in the pod netns.
// The solution is to block the future on the current thread, it is enabled by spawn an os thread, create a
// tokio runtime, and block the task on it.
let hypervisor = self.hypervisor.clone();
let network = thread::spawn(move || -> Result<Arc<dyn Network>> {
let rt = runtime::Builder::new_current_thread().enable_io().build()?;
let d = rt
.block_on(network::new(&network_config))
.context("new network")?;
rt.block_on(d.setup(hypervisor.as_ref()))
.context("setup network")?;
Ok(d)
})
.join()
.map_err(|e| anyhow!("{:?}", e))
.context("Couldn't join on the associated thread")?
.context("failed to set up network")?;
self.network = Some(network);
Ok(())
}
async fn handle_interfaces(&self, network: &dyn Network) -> Result<()> { async fn handle_interfaces(&self, network: &dyn Network) -> Result<()> {
for i in network.interfaces().await.context("get interfaces")? { for i in network.interfaces().await.context("get interfaces")? {
// update interface // update interface

View File

@@ -18,7 +18,7 @@ use network_with_netns::NetworkWithNetns;
mod network_pair; mod network_pair;
use network_pair::NetworkPair; use network_pair::NetworkPair;
mod utils; mod utils;
pub use utils::netns::NetnsGuard; pub use utils::netns::{generate_netns_name, NetnsGuard};
use std::sync::Arc; use std::sync::Arc;
@@ -38,6 +38,7 @@ pub trait Network: Send + Sync {
async fn routes(&self) -> Result<Vec<agent::Route>>; async fn routes(&self) -> Result<Vec<agent::Route>>;
async fn neighs(&self) -> Result<Vec<agent::ARPNeighbor>>; async fn neighs(&self) -> Result<Vec<agent::ARPNeighbor>>;
async fn save(&self) -> Option<Vec<EndpointState>>; async fn save(&self) -> Option<Vec<EndpointState>>;
async fn remove(&self, h: &dyn Hypervisor) -> Result<()>;
} }
pub async fn new(config: &NetworkConfig) -> Result<Arc<dyn Network>> { pub async fn new(config: &NetworkConfig) -> Result<Arc<dyn Network>> {

View File

@@ -4,9 +4,12 @@
// SPDX-License-Identifier: Apache-2.0 // SPDX-License-Identifier: Apache-2.0
// //
use std::sync::{ use std::{
atomic::{AtomicU32, Ordering}, fs,
Arc, sync::{
atomic::{AtomicU32, Ordering},
Arc,
},
}; };
use super::endpoint::endpoint_persist::EndpointState; use super::endpoint::endpoint_persist::EndpointState;
@@ -14,6 +17,7 @@ use anyhow::{anyhow, Context, Result};
use async_trait::async_trait; use async_trait::async_trait;
use futures::stream::TryStreamExt; use futures::stream::TryStreamExt;
use hypervisor::Hypervisor; use hypervisor::Hypervisor;
use netns_rs::get_from_path;
use scopeguard::defer; use scopeguard::defer;
use tokio::sync::RwLock; use tokio::sync::RwLock;
@@ -33,11 +37,13 @@ pub struct NetworkWithNetNsConfig {
pub network_model: String, pub network_model: String,
pub netns_path: String, pub netns_path: String,
pub queues: usize, pub queues: usize,
pub network_created: bool,
} }
struct NetworkWithNetnsInner { struct NetworkWithNetnsInner {
netns_path: String, netns_path: String,
entity_list: Vec<NetworkEntity>, entity_list: Vec<NetworkEntity>,
network_created: bool,
} }
impl NetworkWithNetnsInner { impl NetworkWithNetnsInner {
@@ -54,6 +60,7 @@ impl NetworkWithNetnsInner {
Ok(Self { Ok(Self {
netns_path: config.netns_path.to_string(), netns_path: config.netns_path.to_string(),
entity_list, entity_list,
network_created: config.network_created,
}) })
} }
} }
@@ -120,6 +127,26 @@ impl Network for NetworkWithNetns {
} }
Some(endpoint) Some(endpoint)
} }
async fn remove(&self, h: &dyn Hypervisor) -> Result<()> {
let inner = self.inner.read().await;
// The network namespace would have been deleted at this point
// if it has not been created by virtcontainers.
if !inner.network_created {
return Ok(());
}
{
let _netns_guard =
netns::NetnsGuard::new(&inner.netns_path).context("net netns guard")?;
for e in &inner.entity_list {
e.endpoint.detach(h).await.context("detach")?;
}
}
let netns = get_from_path(inner.netns_path.clone())?;
netns.remove()?;
fs::remove_dir_all(inner.netns_path.clone()).context("failed to remove netns path")?;
Ok(())
}
} }
async fn get_entity_from_netns(config: &NetworkWithNetNsConfig) -> Result<Vec<NetworkEntity>> { async fn get_entity_from_netns(config: &NetworkWithNetNsConfig) -> Result<Vec<NetworkEntity>> {

View File

@@ -9,6 +9,7 @@ use std::{fs::File, os::unix::io::AsRawFd};
use anyhow::{Context, Result}; use anyhow::{Context, Result};
use nix::sched::{setns, CloneFlags}; use nix::sched::{setns, CloneFlags};
use nix::unistd::{getpid, gettid}; use nix::unistd::{getpid, gettid};
use rand::Rng;
pub struct NetnsGuard { pub struct NetnsGuard {
old_netns: Option<File>, old_netns: Option<File>,
@@ -50,6 +51,20 @@ impl Drop for NetnsGuard {
} }
} }
// generate the network namespace name
pub fn generate_netns_name() -> String {
let mut rng = rand::thread_rng();
let random_bytes: [u8; 16] = rng.gen();
format!(
"cnitest-{}-{}-{}-{}-{}",
hex::encode(&random_bytes[..4]),
hex::encode(&random_bytes[4..6]),
hex::encode(&random_bytes[6..8]),
hex::encode(&random_bytes[8..10]),
hex::encode(&random_bytes[10..])
)
}
#[cfg(test)] #[cfg(test)]
mod tests { mod tests {
use super::*; use super::*;
@@ -67,4 +82,14 @@ mod tests {
let empty_path = ""; let empty_path = "";
assert!(NetnsGuard::new(empty_path).unwrap().old_netns.is_none()); assert!(NetnsGuard::new(empty_path).unwrap().old_netns.is_none());
} }
#[test]
fn test_generate_netns_name() {
let name1 = generate_netns_name();
let name2 = generate_netns_name();
let name3 = generate_netns_name();
assert_ne!(name1, name2);
assert_ne!(name2, name3);
assert_ne!(name1, name3);
}
} }

View File

@@ -8,6 +8,7 @@ license = "Apache-2.0"
[dependencies] [dependencies]
anyhow = "^1.0" anyhow = "^1.0"
lazy_static = "1.4.0" lazy_static = "1.4.0"
netns-rs = "0.1.0"
slog = "2.5.2" slog = "2.5.2"
slog-scope = "4.4.0" slog-scope = "4.4.0"
tokio = { version = "1.8.0", features = ["rt-multi-thread"] } tokio = { version = "1.8.0", features = ["rt-multi-thread"] }
@@ -26,6 +27,8 @@ oci = { path = "../../../libs/oci" }
shim-interface = { path = "../../../libs/shim-interface" } shim-interface = { path = "../../../libs/shim-interface" }
persist = { path = "../persist" } persist = { path = "../persist" }
hypervisor = { path = "../hypervisor" } hypervisor = { path = "../hypervisor" }
resource = { path = "../resource" }
# runtime handler # runtime handler
linux_container = { path = "./linux_container", optional = true } linux_container = { path = "./linux_container", optional = true }
virt_container = { path = "./virt_container", optional = true } virt_container = { path = "./virt_container", optional = true }

View File

@@ -11,5 +11,5 @@ pub mod message;
mod runtime_handler; mod runtime_handler;
pub use runtime_handler::{RuntimeHandler, RuntimeInstance}; pub use runtime_handler::{RuntimeHandler, RuntimeInstance};
mod sandbox; mod sandbox;
pub use sandbox::Sandbox; pub use sandbox::{Sandbox, SandboxNetworkEnv};
pub mod types; pub mod types;

View File

@@ -7,14 +7,20 @@
use anyhow::Result; use anyhow::Result;
use async_trait::async_trait; use async_trait::async_trait;
#[derive(Clone)]
pub struct SandboxNetworkEnv {
pub netns: Option<String>,
pub network_created: bool,
}
#[async_trait] #[async_trait]
pub trait Sandbox: Send + Sync { pub trait Sandbox: Send + Sync {
async fn start( async fn start(
&self, &self,
netns: Option<String>,
dns: Vec<String>, dns: Vec<String>,
spec: &oci::Spec, spec: &oci::Spec,
state: &oci::State, state: &oci::State,
network_env: SandboxNetworkEnv,
) -> Result<()>; ) -> Result<()>;
async fn stop(&self) -> Result<()>; async fn stop(&self) -> Result<()>;
async fn cleanup(&self) -> Result<()>; async fn cleanup(&self) -> Result<()>;

View File

@@ -4,20 +4,21 @@
// SPDX-License-Identifier: Apache-2.0 // SPDX-License-Identifier: Apache-2.0
// //
use std::{str::from_utf8, sync::Arc}; use std::{path::PathBuf, str::from_utf8, sync::Arc};
use anyhow::{anyhow, Context, Result};
use crate::{shim_mgmt::server::MgmtServer, static_resource::StaticResourceManager}; use crate::{shim_mgmt::server::MgmtServer, static_resource::StaticResourceManager};
use anyhow::{anyhow, Context, Result};
use common::{ use common::{
message::Message, message::Message,
types::{Request, Response}, types::{Request, Response},
RuntimeHandler, RuntimeInstance, Sandbox, RuntimeHandler, RuntimeInstance, Sandbox, SandboxNetworkEnv,
}; };
use hypervisor::Param; use hypervisor::Param;
use kata_types::{ use kata_types::{
annotations::Annotation, config::default::DEFAULT_GUEST_DNS_FILE, config::TomlConfig, annotations::Annotation, config::default::DEFAULT_GUEST_DNS_FILE, config::TomlConfig,
}; };
use netns_rs::NetNs;
use resource::network::generate_netns_name;
#[cfg(feature = "linux")] #[cfg(feature = "linux")]
use linux_container::LinuxContainer; use linux_container::LinuxContainer;
@@ -53,7 +54,7 @@ impl RuntimeHandlerManagerInner {
&mut self, &mut self,
spec: &oci::Spec, spec: &oci::Spec,
state: &oci::State, state: &oci::State,
netns: Option<String>, network_env: SandboxNetworkEnv,
dns: Vec<String>, dns: Vec<String>,
config: Arc<TomlConfig>, config: Arc<TomlConfig>,
) -> Result<()> { ) -> Result<()> {
@@ -77,7 +78,7 @@ impl RuntimeHandlerManagerInner {
// start sandbox // start sandbox
runtime_instance runtime_instance
.sandbox .sandbox
.start(netns, dns, spec, state) .start(dns, spec, state, network_env)
.await .await
.context("start sandbox")?; .context("start sandbox")?;
self.runtime_instance = Some(Arc::new(runtime_instance)); self.runtime_instance = Some(Arc::new(runtime_instance));
@@ -104,23 +105,6 @@ impl RuntimeHandlerManagerInner {
#[cfg(feature = "virt")] #[cfg(feature = "virt")]
VirtContainer::init().context("init virt container")?; VirtContainer::init().context("init virt container")?;
let netns = if let Some(linux) = &spec.linux {
let mut netns = None;
for ns in &linux.namespaces {
if ns.r#type.as_str() != oci::NETWORKNAMESPACE {
continue;
}
if !ns.path.is_empty() {
netns = Some(ns.path.clone());
break;
}
}
netns
} else {
None
};
for m in &spec.mounts { for m in &spec.mounts {
if m.destination == DEFAULT_GUEST_DNS_FILE { if m.destination == DEFAULT_GUEST_DNS_FILE {
let contents = fs::read_to_string(&m.source).await?; let contents = fs::read_to_string(&m.source).await?;
@@ -129,7 +113,42 @@ impl RuntimeHandlerManagerInner {
} }
let config = load_config(spec, options).context("load config")?; let config = load_config(spec, options).context("load config")?;
self.init_runtime_handler(spec, state, netns, dns, Arc::new(config))
let mut network_created = false;
// set netns to None if we want no network for the VM
let netns = if config.runtime.disable_new_netns {
None
} else {
let mut netns_path = None;
if let Some(linux) = &spec.linux {
for ns in &linux.namespaces {
if ns.r#type.as_str() != oci::NETWORKNAMESPACE {
continue;
}
// get netns path from oci spec
if !ns.path.is_empty() {
netns_path = Some(ns.path.clone());
}
// if we get empty netns from oci spec, we need to create netns for the VM
else {
let ns_name = generate_netns_name();
let netns = NetNs::new(ns_name)?;
let path = PathBuf::from(netns.path()).to_str().map(|s| s.to_string());
info!(sl!(), "the netns path is {:?}", path);
netns_path = path;
network_created = true;
}
break;
}
}
netns_path
};
let network_env = SandboxNetworkEnv {
netns,
network_created,
};
self.init_runtime_handler(spec, state, network_env, dns, Arc::new(config))
.await .await
.context("init runtime handler")?; .context("init runtime handler")?;
@@ -236,7 +255,7 @@ impl RuntimeHandlerManager {
id: container_config.container_id.to_string(), id: container_config.container_id.to_string(),
status: oci::ContainerState::Creating, status: oci::ContainerState::Creating,
pid: 0, pid: 0,
bundle: bundler_path, bundle: container_config.bundle.clone(),
annotations: spec.annotations.clone(), annotations: spec.annotations.clone(),
}; };

View File

@@ -14,7 +14,7 @@ use anyhow::{anyhow, Context, Result};
use async_trait::async_trait; use async_trait::async_trait;
use common::{ use common::{
message::{Action, Message}, message::{Action, Message},
Sandbox, Sandbox, SandboxNetworkEnv,
}; };
use containerd_shim_protos::events::task::TaskOOM; use containerd_shim_protos::events::task::TaskOOM;
use hypervisor::{dragonball::Dragonball, Hypervisor, HYPERVISOR_DRAGONBALL}; use hypervisor::{dragonball::Dragonball, Hypervisor, HYPERVISOR_DRAGONBALL};
@@ -86,30 +86,21 @@ impl VirtSandbox {
}) })
} }
async fn prepare_for_start_sandbox( async fn prepare_config_for_sandbox(
&self, &self,
_id: &str, _id: &str,
netns: Option<String>, network_env: SandboxNetworkEnv,
) -> Result<Vec<ResourceConfig>> { ) -> Result<Vec<ResourceConfig>> {
let mut resource_configs = vec![]; let mut resource_configs = vec![];
if !network_env.network_created {
let config = self.resource_manager.config().await; if let Some(netns_path) = network_env.netns {
if let Some(netns_path) = netns { let network_config = ResourceConfig::Network(
let network_config = ResourceConfig::Network(NetworkConfig::NetworkResourceWithNetNs( self.prepare_network_config(netns_path, network_env.network_created)
NetworkWithNetNsConfig { .await,
network_model: config.runtime.internetworking_model.clone(), );
netns_path, resource_configs.push(network_config);
queues: self }
.hypervisor
.hypervisor_config()
.await
.network_info
.network_queues as usize,
},
));
resource_configs.push(network_config);
} }
let hypervisor_config = self.hypervisor.hypervisor_config().await; let hypervisor_config = self.hypervisor.hypervisor_config().await;
let virtio_fs_config = ResourceConfig::ShareFs(hypervisor_config.shared_fs); let virtio_fs_config = ResourceConfig::ShareFs(hypervisor_config.shared_fs);
resource_configs.push(virtio_fs_config); resource_configs.push(virtio_fs_config);
@@ -149,16 +140,43 @@ impl VirtSandbox {
Ok(()) Ok(())
} }
async fn prepare_network_config(
&self,
netns_path: String,
network_created: bool,
) -> NetworkConfig {
let config = self.resource_manager.config().await;
NetworkConfig::NetworkResourceWithNetNs(NetworkWithNetNsConfig {
network_model: config.runtime.internetworking_model.clone(),
netns_path,
queues: self
.hypervisor
.hypervisor_config()
.await
.network_info
.network_queues as usize,
network_created,
})
}
fn has_prestart_hooks(
&self,
prestart_hooks: Vec<oci::Hook>,
create_runtime_hooks: Vec<oci::Hook>,
) -> bool {
!prestart_hooks.is_empty() || !create_runtime_hooks.is_empty()
}
} }
#[async_trait] #[async_trait]
impl Sandbox for VirtSandbox { impl Sandbox for VirtSandbox {
async fn start( async fn start(
&self, &self,
netns: Option<String>,
dns: Vec<String>, dns: Vec<String>,
spec: &oci::Spec, spec: &oci::Spec,
state: &oci::State, state: &oci::State,
network_env: SandboxNetworkEnv,
) -> Result<()> { ) -> Result<()> {
let id = &self.sid; let id = &self.sid;
@@ -171,13 +189,15 @@ impl Sandbox for VirtSandbox {
} }
self.hypervisor self.hypervisor
.prepare_vm(id, netns.clone()) .prepare_vm(id, network_env.netns.clone())
.await .await
.context("prepare vm")?; .context("prepare vm")?;
// generate device and setup before start vm // generate device and setup before start vm
// should after hypervisor.prepare_vm // should after hypervisor.prepare_vm
let resources = self.prepare_for_start_sandbox(id, netns).await?; let resources = self
.prepare_config_for_sandbox(id, network_env.clone())
.await?;
self.resource_manager self.resource_manager
.prepare_before_start_vm(resources) .prepare_before_start_vm(resources)
.await .await
@@ -195,8 +215,28 @@ impl Sandbox for VirtSandbox {
self.execute_oci_hook_functions(&prestart_hooks, &create_runtime_hooks, state) self.execute_oci_hook_functions(&prestart_hooks, &create_runtime_hooks, state)
.await?; .await?;
// TODO: if prestart_hooks is not empty, rescan the network endpoints(rely on hotplug endpoints). // 1. if there are pre-start hook functions, network config might have been changed.
// see: https://github.com/kata-containers/kata-containers/issues/6378 // We need to rescan the netns to handle the change.
// 2. Do not scan the netns if we want no network for the VM.
// TODO In case of vm factory, scan the netns to hotplug interfaces after the VM is started.
if self.has_prestart_hooks(prestart_hooks, create_runtime_hooks)
&& !self
.resource_manager
.config()
.await
.runtime
.disable_new_netns
{
if let Some(netns_path) = network_env.netns {
let network_resource = self
.prepare_network_config(netns_path, network_env.network_created)
.await;
self.resource_manager
.handle_network(network_resource)
.await
.context("set up device after start vm")?;
}
}
// connect agent // connect agent
// set agent socket // set agent socket