diff --git a/rafs/src/fs.rs b/rafs/src/fs.rs index 5113f3fd104..4f23f20a2f0 100644 --- a/rafs/src/fs.rs +++ b/rafs/src/fs.rs @@ -40,8 +40,7 @@ use nydus_utils::metrics::{self, FopRecorder, StatsFop::*}; use crate::metadata::layout::RAFS_ROOT_INODE; use crate::metadata::{ - Inode, PostWalkAction, RafsInode, RafsSuper, RafsSuperMeta, DOT, DOTDOT, - RAFS_DEFAULT_CHUNK_SIZE, + Inode, PostWalkAction, RafsInode, RafsSuper, RafsSuperMeta, DOT, DOTDOT, RAFS_MAX_CHUNK_SIZE, }; use crate::{RafsError, RafsIoReader, RafsResult}; @@ -104,9 +103,9 @@ impl TryFrom<&RafsConfig> for BlobPrefetchConfig { type Error = RafsError; fn try_from(c: &RafsConfig) -> RafsResult { - if c.fs_prefetch.merging_size as u64 > RAFS_DEFAULT_CHUNK_SIZE { + if c.fs_prefetch.merging_size as u64 > RAFS_MAX_CHUNK_SIZE { return Err(RafsError::Configure( - "Merging size can't exceed chunk size".to_string(), + "merging size can't exceed max chunk size".to_string(), )); } else if c.fs_prefetch.enable && c.fs_prefetch.threads_count == 0 { return Err(RafsError::Configure( @@ -923,8 +922,8 @@ impl FileSystem for Rafs { #[cfg(test)] pub(crate) mod tests { use super::*; + use crate::metadata::RAFS_DEFAULT_CHUNK_SIZE; use crate::RafsIoRead; - use storage::RAFS_MAX_CHUNK_SIZE; pub fn new_rafs_backend() -> Box { let config = r#" @@ -1074,7 +1073,7 @@ pub(crate) mod tests { config.fs_prefetch.merging_size = RAFS_MAX_CHUNK_SIZE as usize + 1; assert!(BlobPrefetchConfig::try_from(&config).is_err()); - config.fs_prefetch.merging_size = RAFS_MAX_CHUNK_SIZE as usize; + config.fs_prefetch.merging_size = RAFS_DEFAULT_CHUNK_SIZE as usize; config.fs_prefetch.bandwidth_rate = 1; config.fs_prefetch.prefetch_all = true; assert!(BlobPrefetchConfig::try_from(&config).is_ok()); diff --git a/src/bin/nydus-image/builder/diff.rs b/src/bin/nydus-image/builder/diff.rs index 38327392ceb..b1fad56805f 100644 --- a/src/bin/nydus-image/builder/diff.rs +++ b/src/bin/nydus-image/builder/diff.rs @@ -511,6 +511,7 @@ impl DiffBuilder { fn prepare_parent_chunk_map( &mut self, + ctx: &BuildContext, bootstrap_mgr: &mut BootstrapManager, ) -> Result { let mut blob_mgr = BlobManager::new(); @@ -523,7 +524,7 @@ impl DiffBuilder { rs.load(r) .context("failed to load superblock from bootstrap")?; // Load blobs from the blob table of parent bootstrap. - blob_mgr.from_blob_table(rs.superblock.get_blob_infos()); + blob_mgr.from_blob_table(ctx, rs.superblock.get_blob_infos()); rs.walk_dir(RAFS_ROOT_INODE, None, &mut |inode: &dyn RafsInode, path: &Path| -> Result<()> { @@ -817,7 +818,7 @@ impl Builder for DiffBuilder { ) -> Result { // Add blobs in parent bootstrap to blob map. let mut parent_blob_mgr = self - .prepare_parent_chunk_map(bootstrap_mgr) + .prepare_parent_chunk_map(ctx, bootstrap_mgr) .context("failed to load chunks from bootstrap")?; let len = parent_blob_mgr.len(); if len > 0 { @@ -829,7 +830,7 @@ impl Builder for DiffBuilder { } // Add blobs in chunk dict bootstrap to blob map. - dict_blob_mgr.extend_blob_table_from_chunk_dict()?; + dict_blob_mgr.extend_blob_table_from_chunk_dict(ctx)?; let len = dict_blob_mgr.len(); if len > 0 { for blob_idx in (0..len).rev() { diff --git a/src/bin/nydus-image/builder/directory.rs b/src/bin/nydus-image/builder/directory.rs index 0e05bf3610a..21c6a8ecae0 100644 --- a/src/bin/nydus-image/builder/directory.rs +++ b/src/bin/nydus-image/builder/directory.rs @@ -151,7 +151,7 @@ impl Builder for DirectoryBuilder { blob_ctx.set_chunk_dict(blob_mgr.get_chunk_dict()); blob_ctx.set_chunk_size(ctx.chunk_size); blob_ctx.set_meta_info_enabled(ctx.fs_version == RafsVersion::V6); - blob_mgr.extend_blob_table_from_chunk_dict()?; + blob_mgr.extend_blob_table_from_chunk_dict(ctx)?; let blob_index = blob_mgr.alloc_index()?; let mut blob = Blob::new(); diff --git a/src/bin/nydus-image/builder/stargz.rs b/src/bin/nydus-image/builder/stargz.rs index dec9106fc94..aa4471bf3c4 100644 --- a/src/bin/nydus-image/builder/stargz.rs +++ b/src/bin/nydus-image/builder/stargz.rs @@ -18,13 +18,15 @@ use nix::sys::stat::makedev; use serde::{Deserialize, Serialize}; use nydus_utils::digest::{self, Algorithm, DigestHasher, RafsDigest}; -use nydus_utils::ByteSize; +use nydus_utils::{try_round_up_4k, ByteSize}; use rafs::metadata::layout::v5::{RafsV5ChunkInfo, RafsV5Inode, RafsV5InodeFlags}; -use rafs::metadata::layout::{RafsBlobTable, RafsXAttrs}; +use rafs::metadata::layout::RafsXAttrs; use rafs::metadata::Inode; use storage::device::BlobChunkFlags; +use storage::meta::BlobMetaHeaderOndisk; use crate::builder::Builder; +use crate::core::blob::Blob; use crate::core::bootstrap::Bootstrap; use crate::core::context::{ BlobContext, BlobManager, BootstrapContext, BootstrapManager, BuildContext, BuildOutput, @@ -296,7 +298,7 @@ impl TocIndex { fn load(path: &Path) -> Result { let index_file = File::open(path) .with_context(|| format!("failed to open stargz index file {:?}", path))?; - let toc_index: TocIndex = serde_json::from_reader(index_file) + let mut toc_index: TocIndex = serde_json::from_reader(index_file) .with_context(|| format!("invalid stargz index file {:?}", path))?; if toc_index.version != 1 { @@ -306,6 +308,15 @@ impl TocIndex { ))); } + // Append root directory entry if not exists. + if !toc_index.entries.is_empty() && toc_index.entries[0].name()? != PathBuf::from("/") { + let root_entry = TocEntry { + toc_type: String::from("dir"), + ..Default::default() + }; + toc_index.entries.insert(0, root_entry); + } + Ok(toc_index) } } @@ -351,6 +362,7 @@ impl StargzIndexTreeBuilder { let mut tree: Option = None; let mut last_reg_entry: Option<&TocEntry> = None; + let mut uncompress_offset = 0; for entry in toc_index.entries.iter() { if !entry.is_supported() { continue; @@ -369,28 +381,35 @@ impl StargzIndexTreeBuilder { } if (entry.is_reg() || entry.is_chunk()) && decompress_size != 0 { + let aligned_chunk_size = if ctx.aligned_chunk { + // Safe to unwrap because `chunk_size` is much less than u32::MAX. + try_round_up_4k(decompress_size).unwrap() + } else { + decompress_size + }; + let pre_uncompress_offset = uncompress_offset; + uncompress_offset += aligned_chunk_size; + let block_id = entry.block_id(&ctx.blob_id)?; + let v5_chunk_info = ChunkWrapper::V5(RafsV5ChunkInfo { + block_id, + // Will be set later + blob_index: 0, + flags: BlobChunkFlags::COMPRESSED, + // No available data on entry + compress_size: 0, + uncompress_size: decompress_size as u32, + compress_offset: entry.offset as u64, + uncompress_offset: pre_uncompress_offset, + file_offset: entry.chunk_offset as u64, + index: 0, + reserved: 0, + }); let chunk = NodeChunk { source: ChunkSource::Build, inner: match ctx.fs_version { - RafsVersion::V5 => { - ChunkWrapper::V5(RafsV5ChunkInfo { - block_id, - // Will be set later - blob_index: 0, - flags: BlobChunkFlags::COMPRESSED, - // No available data on entry - compress_size: 0, - uncompress_size: decompress_size as u32, - compress_offset: entry.offset as u64, - // No available data on entry - uncompress_offset: 0, - file_offset: entry.chunk_offset as u64, - index: 0, - reserved: 0, - }) - } - RafsVersion::V6 => todo!(), + RafsVersion::V5 => v5_chunk_info, + RafsVersion::V6 => v5_chunk_info, }, }; @@ -504,32 +523,31 @@ impl StargzIndexTreeBuilder { let gid = if explicit_uidgid { entry.gid } else { 0 }; // Parse inode info + let v5_inode = RafsV5Inode { + i_digest: RafsDigest::default(), + i_parent: 0, + i_ino: ino, + i_projid: 0, + i_uid: uid, + i_gid: gid, + i_mode: entry.mode(), + i_size: file_size, + i_nlink: entry.num_link, + i_blocks: 0, + i_flags: flags, + i_child_index: 0, + i_child_count: 0, + i_name_size: name_size, + i_symlink_size: symlink_size, + i_rdev: entry.rdev(), + // TODO: add mtime from entry.ModTime() + i_mtime: 0, + i_mtime_nsec: 0, + i_reserved: [0; 8], + }; let inode = match version { - RafsVersion::V5 => { - InodeWrapper::V5(RafsV5Inode { - i_digest: RafsDigest::default(), - i_parent: 0, - i_ino: ino, - i_projid: 0, - i_uid: uid, - i_gid: gid, - i_mode: entry.mode(), - i_size: file_size, - i_nlink: entry.num_link, - i_blocks: 0, - i_flags: flags, - i_child_index: 0, - i_child_count: 0, - i_name_size: name_size, - i_symlink_size: symlink_size, - i_rdev: entry.rdev(), - // TODO: add mtime from entry.ModTime() - i_mtime: 0, - i_mtime_nsec: 0, - i_reserved: [0; 8], - }) - } - RafsVersion::V6 => todo!(), + RafsVersion::V5 => InodeWrapper::V5(v5_inode), + RafsVersion::V6 => InodeWrapper::V6(v5_inode), }; let path = entry.path()?; @@ -575,19 +593,22 @@ impl StargzBuilder { &mut self, ctx: &mut BuildContext, bootstrap_ctx: &mut BootstrapContext, + blob_ctx: &mut BlobContext, blob_mgr: &mut BlobManager, ) -> Result<()> { let mut decompressed_blob_size = 0u64; let mut compressed_blob_size = 0u64; let blob_index = blob_mgr.alloc_index()?; - let mut blob_ctx = BlobContext::new( - ctx.blob_id.clone(), - ctx.blob_storage.clone(), - 0, - ctx.inline_bootstrap, - )?; + + let mut header = BlobMetaHeaderOndisk::default(); + header.set_4k_aligned(true); + blob_ctx.set_chunk_dict(blob_mgr.get_chunk_dict()); blob_ctx.set_chunk_size(ctx.chunk_size); + blob_ctx.set_meta_info_enabled(ctx.fs_version == RafsVersion::V6); + blob_ctx.blob_meta_header = header; + + let mut chunk_map = HashMap::new(); // Set blob index and inode digest for upper nodes for node in &mut bootstrap_ctx.nodes { @@ -598,9 +619,15 @@ impl StargzBuilder { let mut inode_hasher = RafsDigest::hasher(digest::Algorithm::Sha256); for chunk in node.chunks.iter_mut() { - let chunk_index = blob_ctx.alloc_index()?; - chunk.inner.set_index(chunk_index); chunk.inner.set_blob_index(blob_index); + if let Some(chunk_index) = chunk_map.get(chunk.inner.id()) { + chunk.inner.set_index(*chunk_index); + } else { + let chunk_index = blob_ctx.alloc_index()?; + chunk.inner.set_index(chunk_index); + chunk_map.insert(*chunk.inner.id(), chunk.inner.index()); + blob_ctx.add_chunk_meta_info(&chunk.inner)?; + } decompressed_blob_size += chunk.inner.uncompressed_size() as u64; compressed_blob_size += chunk.inner.compressed_size() as u64; inode_hasher.digest_update(chunk.inner.id().as_ref()); @@ -619,9 +646,6 @@ impl StargzBuilder { blob_ctx.decompressed_blob_size = decompressed_blob_size; blob_ctx.compressed_blob_size = compressed_blob_size; - if blob_ctx.decompressed_blob_size > 0 { - blob_mgr.add(blob_ctx); - } Ok(()) } @@ -645,28 +669,43 @@ impl Builder for StargzBuilder { // Build tree from source let layer_idx = if bootstrap_ctx.layered { 1u16 } else { 0u16 }; let mut tree = self.build_tree_from_index(ctx, layer_idx)?; + let origin_bootstarp_offset = bootstrap_ctx.offset; let mut bootstrap = Bootstrap::new()?; - if bootstrap_mgr.f_parent_bootstrap.is_some() { + if bootstrap_ctx.layered { // Merge with lower layer if there's one. + ctx.prefetch.disable(); bootstrap.build(ctx, &mut bootstrap_ctx, &mut tree)?; tree = bootstrap.apply(ctx, &mut bootstrap_ctx, bootstrap_mgr, blob_mgr, None)?; } + // If layered, the bootstrap_ctx.offset will be set in first build, so we need restore it here + bootstrap_ctx.offset = origin_bootstarp_offset; + bootstrap_ctx.layered = false; + timing_tracer!( { bootstrap.build(ctx, &mut bootstrap_ctx, &mut tree) }, "build_bootstrap" )?; // Generate node chunks and digest - self.generate_nodes(ctx, &mut bootstrap_ctx, blob_mgr)?; + let mut blob_ctx = BlobContext::new( + ctx.blob_id.clone(), + ctx.blob_storage.clone(), + 0, + ctx.inline_bootstrap, + )?; + self.generate_nodes(ctx, &mut bootstrap_ctx, &mut blob_ctx, blob_mgr)?; + + // Dump blob meta + Blob::dump_meta_data(ctx, &mut blob_ctx)?; + if blob_ctx.decompressed_blob_size > 0 { + blob_mgr.add(blob_ctx); + } // Dump bootstrap file let blob_table = blob_mgr.to_blob_table(ctx)?; - if let RafsBlobTable::V6(_) = blob_table { - todo!(); - } bootstrap.dump(ctx, &mut bootstrap_ctx, &blob_table)?; - bootstrap_mgr.add(bootstrap_ctx); + BuildOutput::new(blob_mgr, bootstrap_mgr) } } diff --git a/src/bin/nydus-image/core/blob.rs b/src/bin/nydus-image/core/blob.rs index 3f4f3ee0809..7192981c540 100644 --- a/src/bin/nydus-image/core/blob.rs +++ b/src/bin/nydus-image/core/blob.rs @@ -6,13 +6,13 @@ use std::io::Write; use std::os::unix::ffi::OsStrExt; use anyhow::{Context, Result}; -use nydus_utils::compress; use nydus_utils::digest::{self, DigestHasher, RafsDigest}; +use nydus_utils::{compress, try_round_up_4k}; use sha2::Digest; use storage::meta::{BlobChunkInfoOndisk, BlobMetaHeaderOndisk}; use super::chunk_dict::ChunkDict; -use super::context::{BlobContext, BuildContext, SourceType}; +use super::context::{ArtifactWriter, BlobContext, BuildContext, SourceType}; use super::node::Node; pub struct Blob {} @@ -45,8 +45,7 @@ impl Blob { blob_ctx.blob_readahead_size += size; } } - - self.dump_meta_data(blob_ctx)?; + Self::dump_meta_data(ctx, blob_ctx)?; } SourceType::StargzIndex => { for node in nodes { @@ -82,41 +81,70 @@ impl Blob { Ok(blob_exists) } - pub(crate) fn dump_meta_data(&mut self, blob_ctx: &mut BlobContext) -> Result<()> { + pub fn dump_meta_data_raw( + pos: u64, + blob_meta_info: &[BlobChunkInfoOndisk], + compressor: compress::Algorithm, + ) -> Result<(std::borrow::Cow<[u8]>, BlobMetaHeaderOndisk)> { + let data = unsafe { + std::slice::from_raw_parts( + blob_meta_info.as_ptr() as *const u8, + blob_meta_info.len() * std::mem::size_of::(), + ) + }; + let (buf, compressed) = compress::compress(data, compressor) + .with_context(|| "failed to compress blob chunk info array".to_string())?; + let mut header = BlobMetaHeaderOndisk::default(); + + if compressed { + header.set_ci_compressor(compressor); + } else { + header.set_ci_compressor(compress::Algorithm::None); + } + header.set_ci_entries(blob_meta_info.len() as u32); + header.set_ci_compressed_offset(pos); + header.set_ci_compressed_size(buf.len() as u64); + header.set_ci_uncompressed_size(data.len() as u64); + header.set_4k_aligned(true); + + Ok((buf, header)) + } + + pub(crate) fn dump_meta_data(ctx: &BuildContext, blob_ctx: &mut BlobContext) -> Result<()> { // Dump is only required if there is chunk in the blob or blob meta info enabled - if !blob_ctx.blob_meta_info_enabled || blob_ctx.compressed_blob_size == 0 { + if !blob_ctx.blob_meta_info_enabled || blob_ctx.decompressed_blob_size == 0 { return Ok(()); } if let Some(writer) = &mut blob_ctx.writer { let pos = writer.pos()?; - let data = unsafe { - std::slice::from_raw_parts( - blob_ctx.blob_meta_info.as_ptr() as *const u8, - blob_ctx.blob_meta_info.len() * std::mem::size_of::(), - ) - }; - let (buf, compressed) = compress::compress(data, compress::Algorithm::Lz4Block) - .with_context(|| "failed to compress blob chunk info array".to_string())?; - let mut header = BlobMetaHeaderOndisk::default(); - - if compressed { - header.set_ci_compressor(compress::Algorithm::Lz4Block); - } else { - header.set_ci_compressor(compress::Algorithm::None); - } - header.set_ci_entries(blob_ctx.blob_meta_info.len() as u32); - header.set_ci_compressed_offset(pos); - header.set_ci_compressed_size(buf.len() as u64); - header.set_ci_uncompressed_size(data.len() as u64); - header.set_4k_aligned(true); + let (data, header) = Self::dump_meta_data_raw( + pos, + &blob_ctx.blob_meta_info, + compress::Algorithm::Lz4Block, + )?; - blob_ctx.blob_meta_header = header; - - writer.write_all(&buf)?; + writer.write_all(&data)?; writer.write_all(header.as_bytes())?; - blob_ctx.blob_hash.update(&buf); + + blob_ctx.blob_meta_header = header; + blob_ctx.blob_hash.update(&data); blob_ctx.blob_hash.update(header.as_bytes()); + } else if let Some(stor) = ctx.blob_meta_storage.clone() { + // Dump blob meta to an independent local file, use uncompressed format. + let mut writer = ArtifactWriter::new(stor, false)?; + let (data, header) = + Self::dump_meta_data_raw(0, &blob_ctx.blob_meta_info, compress::Algorithm::None)?; + + writer.write_all(&data)?; + // For uncompressed blob meta, keeping 4k alignment to make compatible + // with nydusd runtime, allowing runtime to use the blob meta in blob + // cache directory directly. + let aligned_len: usize = try_round_up_4k(data.len() as u64).unwrap(); + writer.write_all(&vec![0u8; aligned_len - data.len()])?; + writer.write_all(header.as_bytes())?; + + blob_ctx.blob_meta_header = header; } Ok(()) diff --git a/src/bin/nydus-image/core/blob_compact.rs b/src/bin/nydus-image/core/blob_compact.rs index 008918fd1d4..2cb7b15f5b4 100644 --- a/src/bin/nydus-image/core/blob_compact.rs +++ b/src/bin/nydus-image/core/blob_compact.rs @@ -120,6 +120,7 @@ impl ChunkSet { fn dump( &self, + build_ctx: &BuildContext, ori_blob_ids: &[String], new_blob_ctx: &mut BlobContext, new_blob_idx: u32, @@ -176,7 +177,7 @@ impl ChunkSet { } new_blob_ctx.blob_id = format!("{:x}", new_blob_ctx.blob_hash.clone().finalize()); // dump blob meta for v6 - Blob::new().dump_meta_data(new_blob_ctx)?; + Blob::dump_meta_data(build_ctx, new_blob_ctx)?; let blob_id = new_blob_ctx.blob_id(); if let Some(writer) = &mut new_blob_ctx.writer { writer.finalize(blob_id)?; @@ -504,7 +505,12 @@ impl BlobCompactor { .collect() } - pub fn dump_new_blobs(&mut self, dir: &str, aligned_chunk: bool) -> Result<()> { + pub fn dump_new_blobs( + &mut self, + build_ctx: &BuildContext, + dir: &str, + aligned_chunk: bool, + ) -> Result<()> { let ori_blob_ids = self.original_blob_ids(); ensure!(self.states.len() == self.ori_blob_mgr.len()); for idx in 0..self.states.len() { @@ -529,6 +535,7 @@ impl BlobCompactor { blob_ctx.set_meta_info_enabled(self.is_v6()); let blob_idx = self.new_blob_mgr.alloc_index()?; let new_chunks = cs.dump( + build_ctx, &ori_blob_ids, &mut blob_ctx, blob_idx, @@ -575,16 +582,17 @@ impl BlobCompactor { PathBuf::from(""), Default::default(), None, + None, false, ); let mut bootstrap_mgr = BootstrapManager::new(Some(ArtifactStorage::SingleFile(d_bootstrap)), None); let mut bootstrap_ctx = bootstrap_mgr.create_ctx(false)?; let mut ori_blob_mgr = BlobManager::new(); - ori_blob_mgr.from_blob_table(rs.superblock.get_blob_infos()); + ori_blob_mgr.from_blob_table(&build_ctx, rs.superblock.get_blob_infos()); if let Some(dict) = chunk_dict { ori_blob_mgr.set_chunk_dict(dict); - ori_blob_mgr.extend_blob_table_from_chunk_dict()?; + ori_blob_mgr.extend_blob_table_from_chunk_dict(&build_ctx)?; } if ori_blob_mgr.len() < cfg.layers_to_compact { return Ok(None); @@ -598,7 +606,7 @@ impl BlobCompactor { std::mem::swap(&mut bootstrap_ctx.nodes, &mut nodes); let mut compactor = Self::new(build_ctx.fs_version, ori_blob_mgr, nodes, backend.clone())?; compactor.compact(cfg)?; - compactor.dump_new_blobs(&cfg.blobs_dir, build_ctx.aligned_chunk)?; + compactor.dump_new_blobs(&build_ctx, &cfg.blobs_dir, build_ctx.aligned_chunk)?; if compactor.new_blob_mgr.len() == 0 { info!("blobs of {:?} have already been optimized", s_bootstrap); return Ok(None); diff --git a/src/bin/nydus-image/core/bootstrap.rs b/src/bin/nydus-image/core/bootstrap.rs index 3b6909ee247..65d169e121f 100644 --- a/src/bin/nydus-image/core/bootstrap.rs +++ b/src/bin/nydus-image/core/bootstrap.rs @@ -326,7 +326,7 @@ impl Bootstrap { // Reuse lower layer blob table, // we need to append the blob entry of upper layer to the table - blob_mgr.from_blob_table(rs.superblock.get_blob_infos()); + blob_mgr.from_blob_table(ctx, rs.superblock.get_blob_infos()); // Build node tree of lower layer from a bootstrap file, drop by to add // chunks of lower node to chunk_cache for chunk deduplication on next. @@ -531,7 +531,12 @@ impl Bootstrap { for entry in blob_table.entries.iter() { let mut devslot = RafsV6Device::new(); // blob id is String, which is processed by sha256.finalize(). - debug_assert!(entry.blob_id().len() == 64); + if entry.blob_id().len() != 64 { + bail!(format!( + "only blob id of length 64 is supported, blob id {:?}", + entry.blob_id() + )); + } devslot.set_blob_id(entry.blob_id().as_bytes()[0..64].try_into().unwrap()); devslot.set_blocks(entry.uncompressed_size()); devslot.set_mapped_blkaddr(0); diff --git a/src/bin/nydus-image/core/context.rs b/src/bin/nydus-image/core/context.rs index fa43d22f8c6..72328c6de47 100644 --- a/src/bin/nydus-image/core/context.rs +++ b/src/bin/nydus-image/core/context.rs @@ -384,30 +384,36 @@ impl BlobContext { Ok(Self::new_with_writer(blob_id, writer, blob_offset)) } - pub fn from(blob: &BlobInfo, chunk_source: ChunkSource) -> Self { - let mut ctx = Self::new_with_writer(blob.blob_id().to_owned(), None, 0); + pub fn from(ctx: &BuildContext, blob: &BlobInfo, chunk_source: ChunkSource) -> Self { + let mut blob_ctx = Self::new_with_writer(blob.blob_id().to_owned(), None, 0); - ctx.blob_readahead_size = blob.readahead_size(); - ctx.chunk_count = blob.chunk_count(); - ctx.decompressed_blob_size = blob.uncompressed_size(); - ctx.compressed_blob_size = blob.compressed_size(); - ctx.chunk_source = chunk_source; + blob_ctx.blob_readahead_size = blob.readahead_size(); + blob_ctx.chunk_count = blob.chunk_count(); + blob_ctx.decompressed_blob_size = blob.uncompressed_size(); + blob_ctx.compressed_blob_size = blob.compressed_size(); + blob_ctx.chunk_size = blob.chunk_size(); + blob_ctx.chunk_source = chunk_source; + blob_ctx.blob_meta_header.set_4k_aligned(ctx.aligned_chunk); if blob.meta_ci_is_valid() { - ctx.blob_meta_header + blob_ctx + .blob_meta_header .set_ci_compressor(blob.meta_ci_compressor()); - ctx.blob_meta_header.set_ci_entries(blob.chunk_count()); - ctx.blob_meta_header + blob_ctx.blob_meta_header.set_ci_entries(blob.chunk_count()); + blob_ctx + .blob_meta_header .set_ci_compressed_offset(blob.meta_ci_offset()); - ctx.blob_meta_header + blob_ctx + .blob_meta_header .set_ci_compressed_size(blob.meta_ci_compressed_size()); - ctx.blob_meta_header + blob_ctx + .blob_meta_header .set_ci_uncompressed_size(blob.meta_ci_uncompressed_size()); - ctx.blob_meta_header.set_4k_aligned(true); - ctx.blob_meta_info_enabled = true; + blob_ctx.blob_meta_header.set_4k_aligned(true); + blob_ctx.blob_meta_info_enabled = true; } - ctx + blob_ctx } pub fn new_with_writer( @@ -582,10 +588,10 @@ impl BlobManager { } #[allow(clippy::wrong_self_convention)] - pub fn from_blob_table(&mut self, blob_table: Vec>) { + pub fn from_blob_table(&mut self, ctx: &BuildContext, blob_table: Vec>) { self.blobs = blob_table .iter() - .map(|entry| BlobContext::from(entry.as_ref(), ChunkSource::Parent)) + .map(|entry| BlobContext::from(ctx, entry.as_ref(), ChunkSource::Parent)) .collect(); } @@ -605,7 +611,7 @@ impl BlobManager { /// Extend blobs which belong to ChunkDict and setup real_blob_idx map /// should call this function after import parent bootstrap /// otherwise will break blobs order - pub fn extend_blob_table_from_chunk_dict(&mut self) -> Result<()> { + pub fn extend_blob_table_from_chunk_dict(&mut self, ctx: &BuildContext) -> Result<()> { let blobs = self.chunk_dict_ref.get_blobs(); for blob in blobs.iter() { @@ -614,7 +620,7 @@ impl BlobManager { .set_real_blob_idx(blob.blob_index(), real_idx); } else { let idx = self.alloc_index()?; - self.add(BlobContext::from(blob.as_ref(), ChunkSource::Dict)); + self.add(BlobContext::from(ctx, blob.as_ref(), ChunkSource::Dict)); self.chunk_dict_ref .set_real_blob_idx(blob.blob_index(), idx); } @@ -844,6 +850,7 @@ pub struct BuildContext { /// Storage writing blob to single file or a directory. pub blob_storage: Option, + pub blob_meta_storage: Option, pub inline_bootstrap: bool, pub has_xattr: bool, } @@ -862,6 +869,7 @@ impl BuildContext { source_path: PathBuf, prefetch: Prefetch, blob_storage: Option, + blob_meta_storage: Option, inline_bootstrap: bool, ) -> Self { BuildContext { @@ -881,6 +889,7 @@ impl BuildContext { prefetch, blob_storage, + blob_meta_storage, inline_bootstrap, has_xattr: false, } @@ -914,6 +923,7 @@ impl Default for BuildContext { prefetch: Prefetch::default(), blob_storage: None, + blob_meta_storage: None, has_xattr: true, inline_bootstrap: false, } diff --git a/src/bin/nydus-image/main.rs b/src/bin/nydus-image/main.rs index cac33f38875..fddb9c8ff45 100644 --- a/src/bin/nydus-image/main.rs +++ b/src/bin/nydus-image/main.rs @@ -29,7 +29,7 @@ use nydus_api::http::BackendConfig; use nydus_app::{setup_logging, BuildTimeInfo}; use nydus_rafs::RafsIoReader; use nydus_storage::factory::BlobFactory; -use nydus_storage::RAFS_DEFAULT_CHUNK_SIZE; +use nydus_storage::{RAFS_DEFAULT_CHUNK_SIZE, RAFS_MAX_CHUNK_SIZE}; use nydus_utils::{compress, digest}; use crate::builder::{Builder, DiffBuilder, DirectoryBuilder, StargzBuilder}; @@ -213,6 +213,11 @@ fn prepare_cmd_args(bti_string: String) -> ArgMatches<'static> { .required_unless("source-type") .required_unless("blob-dir") .takes_value(true) + ).arg( + Arg::with_name("blob-meta") + .long("blob-meta") + .help("path to store nydus blob metadata") + .takes_value(true) ).arg( Arg::with_name("inline-bootstrap") .long("inline-bootstrap") @@ -583,6 +588,7 @@ impl Command { .unwrap(); let source_type: SourceType = matches.value_of("source-type").unwrap().parse()?; let blob_stor = Self::get_blob_storage(matches, source_type)?; + let blob_meta_stor = Self::get_blob_meta_storage(matches)?; let repeatable = matches.is_present("repeatable"); let version = Self::get_fs_version(matches)?; let aligned_chunk = if version.is_v6() { @@ -634,6 +640,7 @@ impl Command { source_path, prefetch, blob_stor, + blob_meta_stor, inline_bootstrap, ); build_ctx.set_fs_version(version); @@ -892,6 +899,13 @@ impl Command { Ok(blob_stor) } + fn get_blob_meta_storage(matches: &clap::ArgMatches) -> Result> { + let blob_meta_stor = matches + .value_of("blob-meta") + .map(|b| ArtifactStorage::SingleFile(b.into())); + Ok(blob_meta_stor) + } + fn get_parent_bootstrap(matches: &clap::ArgMatches) -> Result> { let mut parent_bootstrap_path = Path::new(""); if let Some(_parent_bootstrap_path) = matches.value_of("parent-bootstrap") { @@ -953,7 +967,7 @@ impl Command { let param = v.trim_start_matches("0x").trim_end_matches("0X"); let chunk_size = u32::from_str_radix(param, 16).context(format!("invalid chunk size {}", v))?; - if chunk_size as u64 > RAFS_DEFAULT_CHUNK_SIZE + if chunk_size as u64 > RAFS_MAX_CHUNK_SIZE || chunk_size < 0x1000 || !chunk_size.is_power_of_two() { diff --git a/src/bin/nydus-image/merge.rs b/src/bin/nydus-image/merge.rs index bbb23b462ca..bbb20a4aad4 100644 --- a/src/bin/nydus-image/merge.rs +++ b/src/bin/nydus-image/merge.rs @@ -81,6 +81,7 @@ impl Merger { } } let mut fs_version = None; + let mut chunk_size = None; for (layer_idx, bootstrap_path) in sources.iter().enumerate() { let rs = RafsSuper::load_from_metadata(bootstrap_path, RafsMode::Direct, true) .context(format!("load bootstrap {:?}", bootstrap_path))?; @@ -118,7 +119,7 @@ impl Merger { let mut parent_blob_added = false; for blob in &parent_blobs { - let mut blob_ctx = BlobContext::from(blob, ChunkSource::Parent); + let mut blob_ctx = BlobContext::from(ctx, blob, ChunkSource::Parent); if chunk_dict_blobs.get(blob.blob_id()).is_none() { // It is assumed that the `nydus-image create` at each layer and `nydus-image merge` commands // use the same chunk dict bootstrap. So the parent bootstrap includes multiple blobs, but @@ -129,6 +130,7 @@ impl Merger { // The blob id (blob sha256 hash) in parent bootstrap is invalid for nydusd // runtime, should change it to the hash of whole tar blob. blob_ctx.blob_id = blob_hash.to_owned(); + chunk_size = Some(blob_ctx.chunk_size); parent_blob_added = true; } blob_idx_map.push(blob_mgr.len() as u32); @@ -186,6 +188,9 @@ impl Merger { ctx.fs_version = RafsVersion::try_from(fs_version.unwrap())?; // Safe to unwrap because there is at least one source bootstrap. let mut tree = tree.unwrap(); + if let Some(chunk_size) = chunk_size { + ctx.chunk_size = chunk_size; + } let mut bootstrap = Bootstrap::new()?; let storage = ArtifactStorage::SingleFile(target.clone()); let mut bootstrap_ctx = BootstrapContext::new(Some(storage), false, false)?; diff --git a/src/bin/nydusd/fs_cache.rs b/src/bin/nydusd/fs_cache.rs index 7e399938513..18bde582fb7 100644 --- a/src/bin/nydusd/fs_cache.rs +++ b/src/bin/nydusd/fs_cache.rs @@ -584,7 +584,16 @@ impl FsCacheHandler { } Some(obj) => match obj.fetch_range_uncompressed(msg.off, msg.len) { Ok(v) if v == msg.len as usize => {} - _ => debug!("fscache: failed to read data from blob object"), + Ok(v) => { + warn!( + "fscache: read data from blob object not matched: {} != {}", + v, msg.len + ); + } + Err(e) => error!( + "{}", + format!("fscache: failed to read data from blob object: {}", e,) + ), }, } } diff --git a/storage/src/cache/cachedfile.rs b/storage/src/cache/cachedfile.rs index ca2b8f48ca2..9d0952bd397 100644 --- a/storage/src/cache/cachedfile.rs +++ b/storage/src/cache/cachedfile.rs @@ -359,6 +359,34 @@ impl BlobObject for FileCacheEntry { impl FileCacheEntry { fn do_fetch_chunks(&self, chunks: &[BlobIoChunk]) -> Result { + if self.is_stargz() { + // FIXME: for stargz, we need to implement fetching multiple chunks. here + // is a heavy overhead workaround, needs to be optimized. + for chunk in chunks { + let mut buf = alloc_buf(chunk.uncompress_size() as usize); + self.read_raw_chunk(chunk, &mut buf, false, None) + .map_err(|e| { + eio!(format!( + "read_raw_chunk failed to read and decompress stargz chunk, {:?}", + e + )) + })?; + if self.dio_enabled { + self.adjust_buffer_for_dio(&mut buf) + } + Self::persist_chunk(&self.file, chunk.uncompress_offset(), &buf).map_err(|e| { + eio!(format!( + "do_fetch_chunk failed to persist stargz chunk, {:?}", + e + )) + })?; + self.chunk_map + .set_ready_and_clear_pending(chunk.as_base()) + .unwrap_or_else(|e| error!("set stargz chunk ready failed, {}", e)); + } + return Ok(0); + } + debug_assert!(!chunks.is_empty()); let bitmap = self .chunk_map diff --git a/storage/src/cache/filecache/mod.rs b/storage/src/cache/filecache/mod.rs index 6c7b2f95fe8..132e9753109 100644 --- a/storage/src/cache/filecache/mod.rs +++ b/storage/src/cache/filecache/mod.rs @@ -188,10 +188,10 @@ impl FileCacheEntry { let is_stargz = blob_info.is_stargz(); let is_compressed = mgr.is_compressed || is_stargz; let need_validate = (mgr.validate || !is_direct_chunkmap) && !is_stargz; - let is_get_blob_object_supported = !mgr.is_compressed && is_direct_chunkmap && !is_stargz; + let is_get_blob_object_supported = !mgr.is_compressed && is_direct_chunkmap; trace!( - "comp {} direct {} startgz {}", + "comp {} direct {} stargz {}", mgr.is_compressed, is_direct_chunkmap, is_stargz @@ -248,7 +248,6 @@ impl FileCacheEntry { // use IndexedChunkMap as a chunk map, but for the old Nydus bootstrap, we // need downgrade to use DigestedChunkMap as a compatible solution. let chunk_map: Arc = if mgr.disable_indexed_map - || blob_info.is_stargz() || blob_info.has_feature(BlobFeatures::V5_NO_EXT_BLOB_TABLE) { direct_chunkmap = false; diff --git a/storage/src/cache/fscache/mod.rs b/storage/src/cache/fscache/mod.rs index dbe2efe9033..e7c9b98a7db 100644 --- a/storage/src/cache/fscache/mod.rs +++ b/storage/src/cache/fscache/mod.rs @@ -165,9 +165,6 @@ impl FileCacheEntry { if blob_info.has_feature(BlobFeatures::V5_NO_EXT_BLOB_TABLE) { return Err(einval!("fscache does not support Rafs v5 blobs")); } - if blob_info.is_stargz() { - return Err(einval!("fscache does not support stargz blob file")); - } let file = blob_info .get_fscache_file() .ok_or_else(|| einval!("No fscache file associated with the blob_info"))?; @@ -210,7 +207,7 @@ impl FileCacheEntry { is_get_blob_object_supported: true, is_compressed: false, is_direct_chunkmap: true, - is_stargz: false, + is_stargz: blob_info.is_stargz(), dio_enabled: true, need_validate: mgr.validate, prefetch_config, diff --git a/storage/src/cache/worker.rs b/storage/src/cache/worker.rs index d8c3992a736..45c073aa6cb 100644 --- a/storage/src/cache/worker.rs +++ b/storage/src/cache/worker.rs @@ -422,8 +422,8 @@ mod tests { let config = Arc::new(AsyncPrefetchConfig { enable: true, threads_count: 4, - merging_size: 0x100000, - bandwidth_rate: 0x100000, + merging_size: 0x1000000, + bandwidth_rate: 0x1000000, }); let mgr = Arc::new(AsyncWorkerMgr::new(metrics, config).unwrap()); @@ -444,7 +444,7 @@ mod tests { assert_eq!(mgr.prefetch_inflight.load(Ordering::Acquire), 0); assert!(mgr - .send_prefetch_message(AsyncPrefetchMessage::RateLimiter(0x100001)) + .send_prefetch_message(AsyncPrefetchMessage::RateLimiter(0x1000001)) .is_ok()); assert!(mgr .send_prefetch_message(AsyncPrefetchMessage::RateLimiter(u64::MAX)) diff --git a/storage/src/device.rs b/storage/src/device.rs index 42c5e053c88..ec1a5d450e5 100644 --- a/storage/src/device.rs +++ b/storage/src/device.rs @@ -323,9 +323,7 @@ impl BlobInfo { /// Check whether compression metadata is available. pub fn meta_ci_is_valid(&self) -> bool { - self.meta_ci_offset != 0 - && self.meta_ci_compressed_size != 0 - && self.meta_ci_uncompressed_size != 0 + self.meta_ci_compressed_size != 0 && self.meta_ci_uncompressed_size != 0 } /// Set the associated `File` object provided by Linux fscache subsystem. diff --git a/storage/src/lib.rs b/storage/src/lib.rs index 7ce2f5b9acd..74e5decae17 100644 --- a/storage/src/lib.rs +++ b/storage/src/lib.rs @@ -72,8 +72,8 @@ macro_rules! impl_getter { /// Default blob chunk size. pub const RAFS_DEFAULT_CHUNK_SIZE: u64 = 1024 * 1024; -/// Maximum blob chunk size. -pub const RAFS_MAX_CHUNK_SIZE: u64 = 1024 * 1024; +/// Maximum blob chunk size, 16MB. +pub const RAFS_MAX_CHUNK_SIZE: u64 = 1024 * 1024 * 16; /// Error codes related to storage subsystem. #[derive(Debug)] diff --git a/storage/src/meta/mod.rs b/storage/src/meta/mod.rs index c8cef54e111..368dddbf490 100644 --- a/storage/src/meta/mod.rs +++ b/storage/src/meta/mod.rs @@ -34,10 +34,11 @@ const BLOB_METADATA_MAX_SIZE: u64 = 0x100_0000u64; const BLOB_METADTAT_HEADER_SIZE: u64 = 0x1000u64; const BLOB_METADATA_RESERVED_SIZE: u64 = BLOB_METADTAT_HEADER_SIZE - 44; const BLOB_METADATA_MAGIC: u32 = 0xb10bb10bu32; -const BLOB_CHUNK_COMP_OFFSET_MASK: u64 = 0xfff_ffff_ffff; +const BLOB_CHUNK_COMP_OFFSET_MASK: u64 = 0xff_ffff_ffff; const BLOB_CHUNK_UNCOMP_OFFSET_MASK: u64 = 0xfff_ffff_f000; -const BLOB_CHUNK_SIZE_MASK: u64 = 0xf_ffff; -const BLOB_CHUNK_SIZE_SHIFT: u64 = 44; +const BLOB_CHUNK_SIZE_LOW_MASK: u64 = 0xf_ffff; +const BLOB_CHUNK_SIZE_LOW_SHIFT: u64 = 44; +const BLOB_CHUNK_SIZE_LOW_SHIFT_LEFT: u64 = 12; const FILE_SUFFIX: &str = "blob.meta"; pub const BLOB_FEATURE_4K_ALIGNED: u32 = 0x1; @@ -171,9 +172,9 @@ impl BlobMetaHeaderOndisk { #[repr(C)] #[derive(Clone, Copy, Default)] pub struct BlobChunkInfoOndisk { - // size: 20bits, offset: 32bits, reserved(available for use): 12bits + // 20bits: size (low), 32bits: offset, 4bits: size (high), 8bits reserved uncomp_info: u64, - // size: 20bits, reserved: 4bits, offset: 40bits + // 20bits: size (low), 4bits: size (high), offset: 40bits comp_info: u64, } @@ -195,16 +196,22 @@ impl BlobChunkInfoOndisk { /// Get compressed size of the chunk. #[inline] pub fn compressed_size(&self) -> u32 { - (self.comp_info >> BLOB_CHUNK_SIZE_SHIFT) as u32 + 1 + let bit20 = self.comp_info >> BLOB_CHUNK_SIZE_LOW_SHIFT; + let bit4 = (self.comp_info & 0xf0000000000) >> 20; + (bit4 | bit20) as u32 + 1 } /// Set compressed size of the chunk. #[inline] pub fn set_compressed_size(&mut self, size: u32) { let size = size as u64; - debug_assert!(size > 0 && size <= BLOB_CHUNK_SIZE_MASK + 1); - self.comp_info &= !(BLOB_CHUNK_SIZE_MASK << BLOB_CHUNK_SIZE_SHIFT); - self.comp_info |= ((size - 1) & BLOB_CHUNK_SIZE_MASK) << BLOB_CHUNK_SIZE_SHIFT; + debug_assert!(size > 0 && size <= BLOB_CHUNK_SIZE_LOW_MASK + 1); + + let size_low = ((size - 1) & BLOB_CHUNK_SIZE_LOW_MASK) << BLOB_CHUNK_SIZE_LOW_SHIFT; + let size_high = ((size - 1) & 0xf0_0000) << 20; + let offset = self.comp_info & BLOB_CHUNK_COMP_OFFSET_MASK; + + self.comp_info = size_low | size_high | offset; } /// Get compressed end of the chunk. @@ -230,16 +237,22 @@ impl BlobChunkInfoOndisk { /// Get uncompressed end of the chunk. #[inline] pub fn uncompressed_size(&self) -> u32 { - (self.uncomp_info >> BLOB_CHUNK_SIZE_SHIFT) as u32 + 1 + let size_high = (self.uncomp_info & 0xf00) << BLOB_CHUNK_SIZE_LOW_SHIFT_LEFT; + let size_low = self.uncomp_info >> BLOB_CHUNK_SIZE_LOW_SHIFT; + (size_high | size_low) as u32 + 1 } /// Set uncompressed end of the chunk. #[inline] pub fn set_uncompressed_size(&mut self, size: u32) { let size = size as u64; - debug_assert!(size != 0 && size <= BLOB_CHUNK_SIZE_MASK + 1); - self.uncomp_info &= !(BLOB_CHUNK_SIZE_MASK << BLOB_CHUNK_SIZE_SHIFT); - self.uncomp_info |= ((size - 1) & BLOB_CHUNK_SIZE_MASK) << BLOB_CHUNK_SIZE_SHIFT; + debug_assert!(size != 0 && size <= BLOB_CHUNK_SIZE_LOW_MASK + 1); + + let size_low = ((size - 1) & BLOB_CHUNK_SIZE_LOW_MASK) << BLOB_CHUNK_SIZE_LOW_SHIFT; + let offset = self.uncomp_info & BLOB_CHUNK_UNCOMP_OFFSET_MASK; + let bit12 = ((size - 1) & 0xf0_0000) >> BLOB_CHUNK_SIZE_LOW_SHIFT_LEFT; + + self.uncomp_info = size_low | offset | bit12; } /// Get uncompressed size of the chunk. @@ -394,6 +407,7 @@ impl BlobMetaInfo { chunks: chunk_infos, base: base as *const u8, unmap_len: expected_size, + is_stargz: blob_info.is_stargz(), }); Ok(BlobMetaInfo { state }) @@ -452,7 +466,9 @@ impl BlobMetaInfo { index += 1; let entry = &infos[index]; self.validate_chunk(entry)?; - if entry.uncompressed_offset() != last_end { + + // For stargz chunks, disable this check. + if !self.state.is_stargz && entry.uncompressed_offset() != last_end { return Err(einval!(format!( "mismatch uncompressed {} size {} last_end {}", entry.uncompressed_offset(), @@ -549,7 +565,8 @@ impl BlobMetaInfo { #[inline] fn validate_chunk(&self, entry: &BlobChunkInfoOndisk) -> Result<()> { - if entry.compressed_end() > self.state.compressed_size + // For stargz blob, self.state.compressed_size == 0, so don't validate it. + if (!self.state.is_stargz && entry.compressed_end() > self.state.compressed_size) || entry.uncompressed_end() > self.state.uncompressed_size { Err(einval!()) @@ -633,6 +650,8 @@ pub struct BlobMetaState { chunks: ManuallyDrop>, base: *const u8, unmap_len: usize, + /// The blob meta is for an stargz image. + is_stargz: bool, } // // Safe to Send/Sync because the underlying data structures are readonly @@ -658,6 +677,25 @@ impl BlobMetaState { let mut start = 0; let mut end = 0; + if self.is_stargz { + // FIXME: since stargz chunks are not currently allocated chunk index in the order of uncompressed_offset, + // a binary search is not available for now, here is a heavy overhead workaround, need to be fixed. + for i in 0..self.chunk_count { + let off = if compressed { + chunks[i as usize].compressed_offset() + } else { + chunks[i as usize].uncompressed_offset() + }; + if addr == off { + return Ok(i as usize); + } + } + return Err(einval!(format!( + "can't find stargz chunk by offset {}", + addr, + ))); + } + while left < right { let mid = left + size / 2; // SAFETY: the call is made safe by the following invariants: @@ -786,6 +824,7 @@ mod tests { ]), base: std::ptr::null(), unmap_len: 0, + is_stargz: false, }; assert_eq!(state.get_chunk_index_nocheck(0, false).unwrap(), 0); @@ -812,18 +851,31 @@ mod tests { chunk.set_compressed_size(0x100); assert_eq!(chunk.compressed_offset(), 0x1000); assert_eq!(chunk.compressed_size(), 0x100); - assert_eq!(chunk.compressed_end(), 0x1100); - chunk.set_uncompressed_offset(0x2000); + + chunk.set_uncompressed_offset(0x1000); chunk.set_uncompressed_size(0x100); - assert_eq!(chunk.uncompressed_offset(), 0x2000); + assert_eq!(chunk.uncompressed_offset(), 0x1000); assert_eq!(chunk.uncompressed_size(), 0x100); - assert_eq!(chunk.uncompressed_end(), 0x2100); - assert_eq!(chunk.aligned_uncompressed_end(), 0x3000); - assert!(!chunk.is_compressed()); - chunk.set_uncompressed_size(0x200); - assert_eq!(chunk.uncompressed_size(), 0x200); - assert!(chunk.is_compressed()); + chunk.set_compressed_offset(0x1000000); + chunk.set_compressed_size(0x1000000); + assert_eq!(chunk.compressed_offset(), 0x1000000); + assert_eq!(chunk.compressed_size(), 0x1000000); + + chunk.set_uncompressed_offset(0x1000000); + chunk.set_uncompressed_size(0x1000000); + assert_eq!(chunk.uncompressed_offset(), 0x1000000); + assert_eq!(chunk.uncompressed_size(), 0x1000000); + + // For testing old format compatibility. + let chunk = BlobChunkInfoOndisk { + uncomp_info: 0xffff_ffff_f100_0000, + comp_info: 0xffff_f0ff_ffff_ffff, + }; + assert_eq!(chunk.uncompressed_size(), 0x000f_ffff + 1); + assert_eq!(chunk.uncompressed_offset(), 0xffff_1000 * 0x1000); + assert_eq!(chunk.compressed_size(), 0x000f_ffff + 1); + assert_eq!(chunk.compressed_offset(), 0x00ff_ffff_ffff); } #[test] @@ -857,6 +909,7 @@ mod tests { ]), base: std::ptr::null(), unmap_len: 0, + is_stargz: false, }; let info = BlobMetaInfo { state: Arc::new(state), diff --git a/tests/builder.rs b/tests/builder.rs index 4c5afccd4b0..57bc5bc1976 100644 --- a/tests/builder.rs +++ b/tests/builder.rs @@ -231,13 +231,27 @@ impl<'a> Builder<'a> { ).unwrap(); } - pub fn build_stargz_empty(&mut self) { + pub fn build_stargz_empty(&mut self, rafs_version: &str, blob_id: &str) { + self.create_dir(&self.work_dir.join("cache")); + + let mut blob_meta = String::new(); + if rafs_version == "6" { + blob_meta = format!( + "--blob-meta {:?}", + self.work_dir + .join("cache") + .join(format!("{}.blob.meta", blob_id)) + ); + } + exec( format!( - "{:?} create --source-type stargz_index --bootstrap {:?} --blob-id {} --log-level info {:?}", + "{:?} create --source-type stargz_index --chunk-size 0x400000 --bootstrap {:?} {} --blob-id {} --fs-version {} --log-level info {:?}", self.builder, - self.work_dir.join("bootstrap-empty"), - "empty.stargz", + self.work_dir.join(blob_id), + blob_meta, + blob_id, + rafs_version, self.work_dir.join("stargz.index-empty.json"), ) .as_str(), @@ -246,14 +260,27 @@ impl<'a> Builder<'a> { ).unwrap(); } - pub fn build_stargz_lower(&mut self) { + pub fn build_stargz_lower(&mut self, rafs_version: &str, blob_id: &str) { + self.create_dir(&self.work_dir.join("cache")); + + let mut blob_meta = String::new(); + if rafs_version == "6" { + blob_meta = format!( + "--blob-meta {:?}", + self.work_dir + .join("cache") + .join(format!("{}.blob.meta", blob_id)) + ); + } + exec( format!( - "{:?} create --source-type stargz_index --parent-bootstrap {:?} --bootstrap {:?} --blob-id {} --log-level info {:?}", + "{:?} create --source-type stargz_index --chunk-size 0x400000 --bootstrap {:?} {} --blob-id {} --fs-version {} --log-level info {:?}", self.builder, - self.work_dir.join("bootstrap-empty"), - self.work_dir.join("bootstrap-lower"), - "lower.stargz", + self.work_dir.join(blob_id), + blob_meta, + blob_id, + rafs_version, self.work_dir.join("stargz.index-lower.json"), ) .as_str(), @@ -262,14 +289,25 @@ impl<'a> Builder<'a> { ).unwrap(); } - pub fn build_stargz_upper(&mut self) { + pub fn build_stargz_upper(&mut self, rafs_version: &str, blob_id: &str) { + let mut blob_meta = String::new(); + if rafs_version == "6" { + blob_meta = format!( + "--blob-meta {:?}", + self.work_dir + .join("cache") + .join(format!("{}.blob.meta", blob_id)) + ); + } + exec( format!( - "{:?} create --source-type stargz_index --parent-bootstrap {:?} --bootstrap {:?} --blob-id {} --log-level info {:?}", + "{:?} create --source-type stargz_index --chunk-size 0x400000 --bootstrap {:?} {} --blob-id {} --fs-version {} --log-level info {:?}", self.builder, - self.work_dir.join("bootstrap-lower"), - self.work_dir.join("bootstrap-overlay"), - "upper.stargz", + self.work_dir.join(blob_id), + blob_meta, + blob_id, + rafs_version, self.work_dir.join("stargz.index-upper.json"), ) .as_str(), @@ -278,6 +316,21 @@ impl<'a> Builder<'a> { ).unwrap(); } + pub fn merge(&mut self, target: &Path, bootstraps: Vec<&str>) { + exec( + format!( + "{:?} merge --bootstrap {:?} --log-level info {}", + self.builder, + target, + bootstraps.join(" "), + ) + .as_str(), + false, + b"", + ) + .unwrap(); + } + pub fn build_special_files(&mut self) { let dir = self.work_dir.join("special_files"); self.create_dir(&dir); diff --git a/tests/smoke.rs b/tests/smoke.rs index d2ba5ff7b30..e3ca8753a30 100644 --- a/tests/smoke.rs +++ b/tests/smoke.rs @@ -249,8 +249,7 @@ fn integration_test_special_files() { } } -#[test] -fn integration_test_stargz() { +fn test_stargz(rafs_version: &str) { info!("\n\n==================== testing run: stargz test"); let tmp_dir = TempDir::new().unwrap(); @@ -263,11 +262,25 @@ fn integration_test_stargz() { ) .unwrap(); + let empty_blob_id = "db30bb2870067ed3e0e73c7448d9f0b529169da8295b5b5155b417624d861d81"; + let lower_blob_id = "e81a6dbddf425e3082f158ff1d80f2adab1dde8dad0b8c4ca43c1bad339f4ec5"; + let upper_blob_id = "5aabe268d8139e952bfff14b2dd0f43fcfef4f2c38b63ea782486ce7268d4954"; + let mut builder = builder::new(&work_dir, "oci"); - builder.build_stargz_empty(); - builder.build_stargz_lower(); - builder.build_stargz_upper(); + builder.build_stargz_empty(rafs_version, empty_blob_id); + builder.build_stargz_lower(rafs_version, lower_blob_id); + builder.build_stargz_upper(rafs_version, upper_blob_id); + + // Merge assumes the bootstrap name as the hash of nydus blob. + builder.merge( + &work_dir.join("bootstrap-overlay"), + vec![ + work_dir.join(empty_blob_id).to_str().unwrap(), + work_dir.join(lower_blob_id).to_str().unwrap(), + work_dir.join(upper_blob_id).to_str().unwrap(), + ], + ); let nydusd = nydusd::new( &work_dir, @@ -282,3 +295,9 @@ fn integration_test_stargz() { nydusd.check("directory/overlay.result", "mnt"); nydusd.umount("mnt"); } + +#[test] +fn integration_test_stargz() { + test_stargz("5"); + test_stargz("6"); +} diff --git a/tests/texture/stargz/blobs/upper.stargz b/tests/texture/stargz/blobs/5aabe268d8139e952bfff14b2dd0f43fcfef4f2c38b63ea782486ce7268d4954 similarity index 100% rename from tests/texture/stargz/blobs/upper.stargz rename to tests/texture/stargz/blobs/5aabe268d8139e952bfff14b2dd0f43fcfef4f2c38b63ea782486ce7268d4954 diff --git a/tests/texture/stargz/blobs/lower.stargz b/tests/texture/stargz/blobs/e81a6dbddf425e3082f158ff1d80f2adab1dde8dad0b8c4ca43c1bad339f4ec5 similarity index 100% rename from tests/texture/stargz/blobs/lower.stargz rename to tests/texture/stargz/blobs/e81a6dbddf425e3082f158ff1d80f2adab1dde8dad0b8c4ca43c1bad339f4ec5 diff --git a/tests/texture/stargz/stargz.index-lower.json b/tests/texture/stargz/stargz.index-lower.json index 27a0e467615..7799918bbc7 100644 --- a/tests/texture/stargz/stargz.index-lower.json +++ b/tests/texture/stargz/stargz.index-lower.json @@ -1,15 +1,6 @@ { "version": 1, "entries": [ - { - "name": "", - "type": "dir", - "modtime": "2020-09-10T07:40:29Z", - "mode": 493, - "userName": "root", - "groupName": "root", - "NumLink": 0 - }, { "name": "root-large", "type": "reg", diff --git a/tests/texture/stargz/stargz.index-upper.json b/tests/texture/stargz/stargz.index-upper.json index d1eb07d7861..44247f5b6f0 100644 --- a/tests/texture/stargz/stargz.index-upper.json +++ b/tests/texture/stargz/stargz.index-upper.json @@ -1,15 +1,6 @@ { "version": 1, "entries": [ - { - "name": "", - "type": "dir", - "modtime": "2020-09-10T06:43:07Z", - "mode": 493, - "userName": "root", - "groupName": "root", - "NumLink": 0 - }, { "name": "sub/", "type": "dir",