diff --git a/rafs/src/builder/core/blob.rs b/rafs/src/builder/core/blob.rs index 128bb758b7e..5623aaeac78 100644 --- a/rafs/src/builder/core/blob.rs +++ b/rafs/src/builder/core/blob.rs @@ -55,6 +55,7 @@ impl Blob { Self::finalize_blob_data(ctx, blob_mgr, blob_writer)?; } ConversionType::TarToRef + | ConversionType::TarToTarfs | ConversionType::TargzToRef | ConversionType::EStargzToRef => { // Use `sha256(tarball)` as `blob_id` for ref-type conversions. @@ -68,6 +69,9 @@ impl Blob { } } else if let Some(tar_reader) = &ctx.blob_tar_reader { blob_ctx.compressed_blob_size = tar_reader.position(); + if ctx.conversion_type == ConversionType::TarToTarfs { + blob_ctx.uncompressed_blob_size = blob_ctx.compressed_blob_size; + } if blob_ctx.blob_id.is_empty() { let hash = tar_reader.get_hash_object(); blob_ctx.blob_id = format!("{:x}", hash.finalize()); diff --git a/rafs/src/builder/core/bootstrap.rs b/rafs/src/builder/core/bootstrap.rs index 27ece03a62e..f889a812fc7 100644 --- a/rafs/src/builder/core/bootstrap.rs +++ b/rafs/src/builder/core/bootstrap.rs @@ -152,6 +152,7 @@ impl Bootstrap { let index = nodes.len() as u32 + 1; let parent = &mut nodes[tree.node.index as usize - 1]; let parent_ino = parent.inode.ino(); + let block_size = ctx.v6_block_size(); // Maybe the parent is not a directory in multi-layers build scenario, so we check here. if parent.is_dir() { @@ -162,7 +163,8 @@ impl Bootstrap { parent.inode.set_child_index(index); parent.inode.set_child_count(tree.children.len() as u32); if ctx.fs_version.is_v6() { - parent.v6_set_dir_offset(bootstrap_ctx, tree.node.v6_dirent_size(tree)?)?; + let d_size = tree.node.v6_dirent_size(ctx, tree)?; + parent.v6_set_dir_offset(bootstrap_ctx, d_size, block_size)?; } } @@ -216,7 +218,7 @@ impl Bootstrap { if !child.node.is_dir() && ctx.fs_version.is_v6() { child .node - .v6_set_offset(bootstrap_ctx, v6_hardlink_offset)?; + .v6_set_offset(bootstrap_ctx, v6_hardlink_offset, block_size)?; } // Store node for bootstrap & blob dump. diff --git a/rafs/src/builder/core/context.rs b/rafs/src/builder/core/context.rs index 2681e3a6c8e..a9c26da1b89 100644 --- a/rafs/src/builder/core/context.rs +++ b/rafs/src/builder/core/context.rs @@ -30,7 +30,7 @@ use nydus_storage::meta::{ BlobMetaChunkInfo, ZranContextGenerator, }; use nydus_utils::digest::DigestData; -use nydus_utils::{compress, digest, div_round_up, round_down_4k, BufReaderInfo}; +use nydus_utils::{compress, digest, div_round_up, round_down, BufReaderInfo}; use super::node::{ChunkSource, Node}; use crate::builder::{ @@ -62,6 +62,7 @@ pub enum ConversionType { TarToStargz, TarToRafs, TarToRef, + TarToTarfs, } impl Default for ConversionType { @@ -85,6 +86,7 @@ impl FromStr for ConversionType { "targz-ref" => Ok(Self::TargzToRef), "tar-rafs" => Ok(Self::TarToRafs), "tar-stargz" => Ok(Self::TarToStargz), + "tar-tarfs" => Ok(Self::TarToTarfs), // kept for backward compatibility "directory" => Ok(Self::DirectoryToRafs), "stargz_index" => Ok(Self::EStargzIndexToRef), @@ -106,8 +108,9 @@ impl fmt::Display for ConversionType { ConversionType::TargzToStargz => write!(f, "targz-ref"), ConversionType::TargzToRef => write!(f, "targz-ref"), ConversionType::TarToRafs => write!(f, "tar-rafs"), - ConversionType::TarToStargz => write!(f, "tar-stargz"), ConversionType::TarToRef => write!(f, "tar-ref"), + ConversionType::TarToStargz => write!(f, "tar-stargz"), + ConversionType::TarToTarfs => write!(f, "tar-tarfs"), } } } @@ -120,6 +123,7 @@ impl ConversionType { | ConversionType::EStargzIndexToRef | ConversionType::TargzToRef | ConversionType::TarToRef + | ConversionType::TarToTarfs ) } } @@ -478,6 +482,9 @@ impl BlobContext { blob_ctx .blob_meta_header .set_cap_tar_toc(features.contains(BlobFeatures::CAP_TAR_TOC)); + blob_ctx + .blob_meta_header + .set_tarfs(features.contains(BlobFeatures::TARFS)); blob_ctx } @@ -703,6 +710,7 @@ impl BlobContext { /// Get offset of compressed blob, since current_compressed_offset /// is always >= compressed_blob_size, we can safely subtract here. pub fn compressed_offset(&self) -> u64 { + assert!(self.current_compressed_offset >= self.compressed_blob_size); self.current_compressed_offset - self.compressed_blob_size } } @@ -745,7 +753,9 @@ impl BlobManager { ctx.digester, ); blob_ctx.set_chunk_size(ctx.chunk_size); - blob_ctx.set_meta_info_enabled(ctx.fs_version == RafsVersion::V6); + blob_ctx.set_meta_info_enabled( + ctx.fs_version == RafsVersion::V6 && ctx.conversion_type != ConversionType::TarToTarfs, + ); Ok(blob_ctx) } @@ -990,19 +1000,22 @@ impl BootstrapContext { // Try to find an used block with no less than `size` space left. // If found it, return the offset where we can store data. // If not, return 0. - pub(crate) fn allocate_available_block(&mut self, size: u64) -> u64 { - if size >= EROFS_BLOCK_SIZE_4096 { + pub(crate) fn allocate_available_block(&mut self, size: u64, block_size: u64) -> u64 { + if size >= block_size { return 0; } let min_idx = div_round_up(size, EROFS_INODE_SLOT_SIZE as u64) as usize; - let max_idx = div_round_up(EROFS_BLOCK_SIZE_4096, EROFS_INODE_SLOT_SIZE as u64) as usize; + let max_idx = div_round_up(block_size, EROFS_INODE_SLOT_SIZE as u64) as usize; for idx in min_idx..max_idx { let blocks = &mut self.v6_available_blocks[idx]; if let Some(mut offset) = blocks.pop_front() { - offset += EROFS_BLOCK_SIZE_4096 - (idx * EROFS_INODE_SLOT_SIZE) as u64; - self.append_available_block(offset + (min_idx * EROFS_INODE_SLOT_SIZE) as u64); + offset += block_size - (idx * EROFS_INODE_SLOT_SIZE) as u64; + self.append_available_block( + offset + (min_idx * EROFS_INODE_SLOT_SIZE) as u64, + block_size, + ); return offset; } } @@ -1011,11 +1024,11 @@ impl BootstrapContext { } // Append the block that `offset` belongs to corresponding deque. - pub(crate) fn append_available_block(&mut self, offset: u64) { - if offset % EROFS_BLOCK_SIZE_4096 != 0 { - let avail = EROFS_BLOCK_SIZE_4096 - offset % EROFS_BLOCK_SIZE_4096; + pub(crate) fn append_available_block(&mut self, offset: u64, block_size: u64) { + if offset % block_size != 0 { + let avail = block_size - offset % block_size; let idx = avail as usize / EROFS_INODE_SLOT_SIZE; - self.v6_available_blocks[idx].push_back(round_down_4k(offset)); + self.v6_available_blocks[idx].push_back(round_down(offset, block_size)); } } } @@ -1114,6 +1127,9 @@ impl BuildContext { blob_features |= BlobFeatures::HAS_TOC; blob_features |= BlobFeatures::HAS_TAR_HEADER; } + if conversion_type == ConversionType::TarToTarfs { + blob_features |= BlobFeatures::TARFS; + } BuildContext { blob_id, diff --git a/rafs/src/builder/core/node.rs b/rafs/src/builder/core/node.rs index bed39fe57d2..c9450fad890 100644 --- a/rafs/src/builder/core/node.rs +++ b/rafs/src/builder/core/node.rs @@ -24,7 +24,9 @@ use nydus_utils::digest::{DigestHasher, RafsDigest}; use nydus_utils::{div_round_up, event_tracer, root_tracer, try_round_up_4k, ByteSize}; use sha2::digest::Digest; -use crate::builder::{ArtifactWriter, BlobContext, BlobManager, BuildContext, ChunkDict, Overlay}; +use crate::builder::{ + ArtifactWriter, BlobContext, BlobManager, BuildContext, ChunkDict, ConversionType, Overlay, +}; use crate::metadata::chunk::ChunkWrapper; use crate::metadata::inode::InodeWrapper; use crate::metadata::layout::v6::EROFS_INODE_FLAT_PLAIN; @@ -270,35 +272,45 @@ impl Node { }; let chunk_data = &mut data_buf[0..uncompressed_size as usize]; - let (chunk, chunk_info) = self.read_file_chunk(ctx, reader, chunk_data)?; + let (mut chunk, chunk_info) = self.read_file_chunk(ctx, reader, chunk_data)?; if let Some(h) = inode_hasher.as_mut() { h.digest_update(chunk.id().as_ref()); } - let mut chunk = match self.deduplicate_chunk( - ctx, - blob_mgr, - file_offset, - uncompressed_size, - chunk, - )? { - None => continue, - Some(c) => c, - }; + // No need to perform chunk deduplication for tar-tarfs case. + if ctx.conversion_type != ConversionType::TarToTarfs { + chunk = match self.deduplicate_chunk( + ctx, + blob_mgr, + file_offset, + uncompressed_size, + chunk, + )? { + None => continue, + Some(c) => c, + }; + } let (blob_index, blob_ctx) = blob_mgr.get_or_create_current_blob(ctx)?; let chunk_index = blob_ctx.alloc_chunk_index()?; chunk.set_blob_index(blob_index); chunk.set_index(chunk_index); chunk.set_file_offset(file_offset); - self.dump_file_chunk(ctx, blob_ctx, blob_writer, chunk_data, &mut chunk)?; + if ctx.conversion_type == ConversionType::TarToTarfs { + chunk.set_uncompressed_offset(chunk.compressed_offset()); + chunk.set_uncompressed_size(chunk.compressed_size()); + } else { + self.dump_file_chunk(ctx, blob_ctx, blob_writer, chunk_data, &mut chunk)?; + } let chunk = Arc::new(chunk); blob_size += chunk.compressed_size() as u64; - blob_ctx.add_chunk_meta_info(&chunk, chunk_info)?; - blob_mgr - .layered_chunk_dict - .add_chunk(chunk.clone(), ctx.digester); + if ctx.conversion_type != ConversionType::TarToTarfs { + blob_ctx.add_chunk_meta_info(&chunk, chunk_info)?; + blob_mgr + .layered_chunk_dict + .add_chunk(chunk.clone(), ctx.digester); + } self.chunks.push(NodeChunk { source: ChunkSource::Build, inner: chunk, @@ -347,7 +359,11 @@ impl Node { .with_context(|| format!("failed to read node file {:?}", self.path()))?; } - chunk.set_id(RafsDigest::from_buf(buf, ctx.digester)); + // For tar-tarfs case, no need to compute chunk id. + if ctx.conversion_type != ConversionType::TarToTarfs { + chunk.set_id(RafsDigest::from_buf(buf, ctx.digester)); + } + Ok((chunk, chunk_info)) } diff --git a/rafs/src/builder/core/v6.rs b/rafs/src/builder/core/v6.rs index bbec79309e6..6cc8a921e6a 100644 --- a/rafs/src/builder/core/v6.rs +++ b/rafs/src/builder/core/v6.rs @@ -11,19 +11,20 @@ use std::os::unix::ffi::OsStrExt; use std::sync::Arc; use anyhow::{bail, ensure, Context, Result}; -use nydus_utils::{div_round_up, root_tracer, round_down_4k, round_up, timing_tracer}; +use nydus_utils::{root_tracer, round_down, round_up, timing_tracer}; use storage::device::BlobFeatures; use super::chunk_dict::DigestWithBlobIndex; use super::node::Node; -use crate::builder::{Bootstrap, BootstrapContext, BuildContext, Tree}; +use crate::builder::{Bootstrap, BootstrapContext, BuildContext, ConversionType, Tree}; use crate::metadata::chunk::ChunkWrapper; use crate::metadata::inode::new_v6_inode; use crate::metadata::layout::v6::{ align_offset, calculate_nid, RafsV6BlobTable, RafsV6Device, RafsV6Dirent, RafsV6InodeChunkAddr, RafsV6InodeChunkHeader, RafsV6OndiskInode, RafsV6SuperBlock, RafsV6SuperBlockExt, - EROFS_BLOCK_SIZE_4096, EROFS_DEVTABLE_OFFSET, EROFS_INODE_CHUNK_BASED, EROFS_INODE_FLAT_INLINE, - EROFS_INODE_FLAT_PLAIN, EROFS_INODE_SLOT_SIZE, EROFS_SUPER_BLOCK_SIZE, EROFS_SUPER_OFFSET, + EROFS_BLOCK_BITS_9, EROFS_BLOCK_SIZE_4096, EROFS_BLOCK_SIZE_512, EROFS_DEVTABLE_OFFSET, + EROFS_INODE_CHUNK_BASED, EROFS_INODE_FLAT_INLINE, EROFS_INODE_FLAT_PLAIN, + EROFS_INODE_SLOT_SIZE, EROFS_SUPER_BLOCK_SIZE, EROFS_SUPER_OFFSET, }; use crate::metadata::RafsStore; use crate::RafsIoWrite; @@ -102,6 +103,7 @@ impl Node { &mut self, bootstrap_ctx: &mut BootstrapContext, v6_hardlink_offset: Option, + block_size: u64, ) -> Result<()> { ensure!(!self.is_dir(), "{} is a directory", self.path().display()); if self.is_reg() { @@ -112,7 +114,7 @@ impl Node { let unit = size_of::() as u64; let total_size = round_up(size, unit) + self.inode.child_count() as u64 * unit; // First try to allocate from fragments of dirent pages. - self.v6_offset = bootstrap_ctx.allocate_available_block(total_size); + self.v6_offset = bootstrap_ctx.allocate_available_block(total_size, block_size); if self.v6_offset == 0 { self.v6_offset = bootstrap_ctx.offset; bootstrap_ctx.offset += total_size; @@ -120,7 +122,7 @@ impl Node { } self.v6_datalayout = EROFS_INODE_CHUNK_BASED; } else if self.is_symlink() { - self.v6_set_offset_with_tail(bootstrap_ctx, self.inode.size()); + self.v6_set_offset_with_tail(bootstrap_ctx, self.inode.size(), block_size); } else { self.v6_offset = bootstrap_ctx.offset; bootstrap_ctx.offset += self.v6_size_with_xattr(); @@ -135,6 +137,7 @@ impl Node { &mut self, bootstrap_ctx: &mut BootstrapContext, d_size: u64, + block_size: u64, ) -> Result<()> { ensure!( self.is_dir(), @@ -144,26 +147,27 @@ impl Node { // Dir isize is the total bytes of 'dirents + names'. self.inode.set_size(d_size); - self.v6_set_offset_with_tail(bootstrap_ctx, d_size); + self.v6_set_offset_with_tail(bootstrap_ctx, d_size, block_size); bootstrap_ctx.align_offset(EROFS_INODE_SLOT_SIZE as u64); Ok(()) } /// Calculate space needed to store dirents of the directory inode. - pub fn v6_dirent_size(&self, tree: &Tree) -> Result { + pub fn v6_dirent_size(&self, ctx: &mut BuildContext, tree: &Tree) -> Result { ensure!(self.is_dir(), "{} is not a directory", self); // Use length in byte, instead of length in character. let mut d_size: u64 = (".".as_bytes().len() + size_of::() + "..".as_bytes().len() + size_of::()) as u64; + let block_size = ctx.v6_block_size(); for child in tree.children.iter() { let len = child.node.name().as_bytes().len() + size_of::(); - // erofs disk format requires dirent to be aligned with 4096. - if (d_size % EROFS_BLOCK_SIZE_4096) + len as u64 > EROFS_BLOCK_SIZE_4096 { - d_size = div_round_up(d_size as u64, EROFS_BLOCK_SIZE_4096) * EROFS_BLOCK_SIZE_4096; + // erofs disk format requires dirent to be aligned to block size. + if (d_size % block_size) + len as u64 > block_size { + d_size = round_up(d_size as u64, block_size); } d_size += len as u64; } @@ -180,7 +184,12 @@ impl Node { // // For DIR inode, size is the total bytes of 'dirents + names'. // For symlink, size is the length of symlink name. - fn v6_set_offset_with_tail(&mut self, bootstrap_ctx: &mut BootstrapContext, d_size: u64) { + fn v6_set_offset_with_tail( + &mut self, + bootstrap_ctx: &mut BootstrapContext, + d_size: u64, + block_size: u64, + ) { // | avail | // +--------+-----------+----+ +-----------------------+ // | |inode+tail | free | dirents+names | @@ -217,7 +226,7 @@ impl Node { // // let inode_size = self.v6_size_with_xattr(); - let tail: u64 = d_size % EROFS_BLOCK_SIZE_4096; + let tail: u64 = d_size % block_size; // We use a simple inline strategy here: // If the inode size with xattr + tail data size <= EROFS_BLOCK_SIZE, @@ -229,7 +238,7 @@ impl Node { // since it contain only single blocks with some unused space, the available space can only // be smaller than EROFS_BLOCK_SIZE, therefore we can't use our used blocks to store the // inode plus the tail data bigger than EROFS_BLOCK_SIZE. - let should_inline = tail != 0 && (inode_size + tail) <= EROFS_BLOCK_SIZE_4096; + let should_inline = tail != 0 && (inode_size + tail) <= block_size; // If should inline, we first try to allocate space for the inode together with tail data // using used blocks. @@ -238,13 +247,12 @@ impl Node { // and we allocate space from the next block. // For the remaining data, we allocate space for it sequentially. self.v6_datalayout = if should_inline { - self.v6_offset = bootstrap_ctx.allocate_available_block(inode_size + tail); + self.v6_offset = bootstrap_ctx.allocate_available_block(inode_size + tail, block_size); if self.v6_offset == 0 { - let available = - EROFS_BLOCK_SIZE_4096 - bootstrap_ctx.offset % EROFS_BLOCK_SIZE_4096; + let available = block_size - bootstrap_ctx.offset % block_size; if available < inode_size + tail { - bootstrap_ctx.append_available_block(bootstrap_ctx.offset); - bootstrap_ctx.align_offset(EROFS_BLOCK_SIZE_4096); + bootstrap_ctx.append_available_block(bootstrap_ctx.offset, block_size); + bootstrap_ctx.align_offset(block_size); } self.v6_offset = bootstrap_ctx.offset; @@ -252,28 +260,28 @@ impl Node { } if d_size != tail { - bootstrap_ctx.append_available_block(bootstrap_ctx.offset); - bootstrap_ctx.align_offset(EROFS_BLOCK_SIZE_4096); + bootstrap_ctx.append_available_block(bootstrap_ctx.offset, block_size); + bootstrap_ctx.align_offset(block_size); } self.v6_dirents_offset = bootstrap_ctx.offset; - bootstrap_ctx.offset += round_down_4k(d_size); + bootstrap_ctx.offset += round_down(d_size, block_size); EROFS_INODE_FLAT_INLINE } else { // Otherwise, we first try to allocate space for the inode from used blocks. // If no available used block exists, we allocate space sequentially. // Then we allocate space for all data. - self.v6_offset = bootstrap_ctx.allocate_available_block(inode_size); + self.v6_offset = bootstrap_ctx.allocate_available_block(inode_size, block_size); if self.v6_offset == 0 { self.v6_offset = bootstrap_ctx.offset; bootstrap_ctx.offset += inode_size; } - bootstrap_ctx.append_available_block(bootstrap_ctx.offset); - bootstrap_ctx.align_offset(EROFS_BLOCK_SIZE_4096); + bootstrap_ctx.append_available_block(bootstrap_ctx.offset, block_size); + bootstrap_ctx.align_offset(block_size); self.v6_dirents_offset = bootstrap_ctx.offset; bootstrap_ctx.offset += d_size; - bootstrap_ctx.align_offset(EROFS_BLOCK_SIZE_4096); + bootstrap_ctx.align_offset(block_size); EROFS_INODE_FLAT_PLAIN }; @@ -327,6 +335,7 @@ impl Node { let mut dirents: Vec<(RafsV6Dirent, &OsString)> = Vec::new(); let mut nameoff: u64 = 0; let mut used: u64 = 0; + let block_size = ctx.v6_block_size(); trace!( "{:?} self.dirents.len {}", @@ -337,7 +346,7 @@ impl Node { for (offset, name, file_type) in self.v6_dirents.iter() { let len = name.len() + size_of::(); // write to bootstrap when it will exceed EROFS_BLOCK_SIZE - if used + len as u64 > EROFS_BLOCK_SIZE_4096 { + if used + len as u64 > block_size { for (entry, name) in dirents.iter_mut() { trace!("{:?} nameoff {}", name, nameoff); entry.set_name_offset(nameoff as u16); @@ -358,13 +367,13 @@ impl Node { .write(dir_data.as_slice()) .context("failed to write dirent data to meta blob")?; + // track where we're going to write. + dirent_off += round_up(used, block_size); + used = 0; + nameoff = 0; dir_data.clear(); entry_names.clear(); dirents.clear(); - nameoff = 0; - used = 0; - // track where we're going to write. - dirent_off += EROFS_BLOCK_SIZE_4096; } trace!( @@ -464,7 +473,7 @@ impl Node { } else { ctx.chunk_size as u64 }; - let info = RafsV6InodeChunkHeader::new(chunk_size); + let info = RafsV6InodeChunkHeader::new(chunk_size, ctx.v6_block_size()); inode.set_u(info.to_u32()); self.v6_dump_inode(ctx, f_bootstrap, inode) .context("failed to dump inode for file")?; @@ -528,7 +537,12 @@ impl Node { impl BuildContext { pub fn v6_block_size(&self) -> u64 { - EROFS_BLOCK_SIZE_4096 + if self.conversion_type == ConversionType::TarToTarfs { + // Tar stream is 512-byte aligned. + EROFS_BLOCK_SIZE_512 + } else { + EROFS_BLOCK_SIZE_4096 + } } pub fn v6_block_addr(&self, offset: u64) -> Result { @@ -605,12 +619,13 @@ impl Bootstrap { // | | |devslot | | | | | // +---+---------+------------+-------------+----------------------------------------------+ + let block_size = ctx.v6_block_size(); let blobs = blob_table.get_all(); let devtable_len = blobs.len() * size_of::(); let blob_table_size = blob_table.size() as u64; let blob_table_offset = align_offset( (EROFS_DEVTABLE_OFFSET as u64) + devtable_len as u64, - EROFS_BLOCK_SIZE_4096 as u64, + EROFS_BLOCK_SIZE_4096, ); let blob_table_entries = blobs.len(); assert!(blob_table_entries < u8::MAX as usize); @@ -644,7 +659,7 @@ impl Bootstrap { let meta_addr = if blob_table_size > 0 { align_offset( blob_table_offset + blob_table_size + prefetch_table_size as u64, - EROFS_BLOCK_SIZE_4096 as u64, + EROFS_BLOCK_SIZE_4096, ) } else { orig_meta_addr @@ -691,7 +706,7 @@ impl Bootstrap { "dump_bootstrap", Result<()> )?; - Self::v6_align_to_block(bootstrap_ctx)?; + Self::v6_align_to_4k(bootstrap_ctx)?; // `Node` offset might be updated during above inodes dumping. So `get_prefetch_table` after it. let prefetch_table = ctx @@ -727,14 +742,14 @@ impl Bootstrap { "chunk_table offset {} size {}", chunk_table_offset, chunk_table_size ); - Self::v6_align_to_block(bootstrap_ctx)?; + Self::v6_align_to_4k(bootstrap_ctx)?; // Prepare device slots. let mut pos = bootstrap_ctx .writer .seek_to_end() .context("failed to seek to bootstrap's end for chunk table")?; - assert_eq!(pos % EROFS_BLOCK_SIZE_4096, 0); + assert_eq!(pos % block_size, 0); let mut devtable: Vec = Vec::new(); let mut block_count = 0u32; let mut inlined_chunk_digest = true; @@ -761,7 +776,7 @@ impl Bootstrap { if block_count.checked_add(cnt).is_none() { bail!("Too many data blocks in RAFS filesystem, block size 0x{:x}, block count 0x{:x}", ctx.v6_block_size(), block_count as u64 + cnt as u64); } - let mapped_blkaddr = Self::v6_align_mapped_blkaddr(ctx, pos)?; + let mapped_blkaddr = Self::v6_align_mapped_blkaddr(block_size, pos)?; pos += cnt as u64 * ctx.v6_block_size(); block_count += cnt; @@ -777,6 +792,9 @@ impl Bootstrap { // Dump super block let mut sb = RafsV6SuperBlock::new(); + if ctx.conversion_type == ConversionType::TarToTarfs { + sb.set_block_bits(EROFS_BLOCK_BITS_9); + } sb.set_inos(bootstrap_ctx.nodes.len() as u64); sb.set_blocks(block_count); sb.set_root_nid(root_nid as u16); @@ -796,6 +814,9 @@ impl Bootstrap { if inlined_chunk_digest { ext_sb.set_inlined_chunk_digest(); } + if ctx.conversion_type == ConversionType::TarToTarfs { + ext_sb.set_tarfs_mode(); + } bootstrap_ctx .writer .seek_offset((EROFS_SUPER_OFFSET + EROFS_SUPER_BLOCK_SIZE) as u64) @@ -826,7 +847,7 @@ impl Bootstrap { Ok(()) } - fn v6_align_to_block(bootstrap_ctx: &mut BootstrapContext) -> Result<()> { + fn v6_align_to_4k(bootstrap_ctx: &mut BootstrapContext) -> Result<()> { bootstrap_ctx .writer .flush() @@ -835,7 +856,7 @@ impl Bootstrap { .writer .seek_to_end() .context("failed to seek to bootstrap's end for chunk table")?; - let padding = align_offset(pos, EROFS_BLOCK_SIZE_4096 as u64) - pos; + let padding = align_offset(pos, EROFS_BLOCK_SIZE_4096) - pos; bootstrap_ctx .writer .write_all(&WRITE_PADDING_DATA[0..padding as usize]) @@ -847,11 +868,11 @@ impl Bootstrap { Ok(()) } - fn v6_align_mapped_blkaddr(ctx: &BuildContext, addr: u64) -> Result { + fn v6_align_mapped_blkaddr(block_size: u64, addr: u64) -> Result { match addr.checked_add(V6_BLOCK_SEG_ALIGNMENT - 1) { None => bail!("address 0x{:x} is too big", addr), Some(v) => { - let v = (v & !(V6_BLOCK_SEG_ALIGNMENT - 1)) / ctx.v6_block_size(); + let v = (v & !(V6_BLOCK_SEG_ALIGNMENT - 1)) / block_size; if v > u32::MAX as u64 { bail!("address 0x{:x} is too big", addr); } else { @@ -894,7 +915,8 @@ mod tests { // reg file. // "1" is used only for testing purpose, in practice // it's always aligned to 32 bytes. - node.v6_set_offset(&mut bootstrap_ctx, None).unwrap(); + node.v6_set_offset(&mut bootstrap_ctx, None, EROFS_BLOCK_SIZE_4096) + .unwrap(); assert_eq!(node.v6_offset, 0); assert_eq!(node.v6_datalayout, EROFS_INODE_CHUNK_BASED); assert!(node.v6_compact_inode); @@ -913,14 +935,14 @@ mod tests { .unwrap(); dir_node - .v6_set_dir_offset(&mut bootstrap_ctx, 4064) + .v6_set_dir_offset(&mut bootstrap_ctx, 4064, EROFS_BLOCK_SIZE_4096) .unwrap(); assert_eq!(dir_node.v6_datalayout, EROFS_INODE_FLAT_INLINE); assert_eq!(dir_node.v6_offset, 4096); assert_eq!(bootstrap_ctx.offset, 8192); dir_node - .v6_set_dir_offset(&mut bootstrap_ctx, 4096) + .v6_set_dir_offset(&mut bootstrap_ctx, 4096, EROFS_BLOCK_SIZE_4096) .unwrap(); assert_eq!(dir_node.v6_datalayout, EROFS_INODE_FLAT_PLAIN); assert_eq!(dir_node.v6_offset, 32); @@ -928,7 +950,7 @@ mod tests { assert_eq!(bootstrap_ctx.offset, 8192 + 4096); dir_node - .v6_set_dir_offset(&mut bootstrap_ctx, 8160) + .v6_set_dir_offset(&mut bootstrap_ctx, 8160, EROFS_BLOCK_SIZE_4096) .unwrap(); assert_eq!(dir_node.v6_datalayout, EROFS_INODE_FLAT_INLINE); assert_eq!(dir_node.v6_offset, 8192 + 4096); @@ -936,7 +958,7 @@ mod tests { assert_eq!(bootstrap_ctx.offset, 8192 + 4096 + 8192); dir_node - .v6_set_dir_offset(&mut bootstrap_ctx, 8161) + .v6_set_dir_offset(&mut bootstrap_ctx, 8161, EROFS_BLOCK_SIZE_4096) .unwrap(); assert_eq!(dir_node.v6_datalayout, EROFS_INODE_FLAT_PLAIN); assert_eq!(dir_node.v6_offset, 64); @@ -944,7 +966,7 @@ mod tests { assert_eq!(bootstrap_ctx.offset, 8192 + 4096 + 8192 + 8192); dir_node - .v6_set_dir_offset(&mut bootstrap_ctx, 4096 + 3968) + .v6_set_dir_offset(&mut bootstrap_ctx, 4096 + 3968, EROFS_BLOCK_SIZE_4096) .unwrap(); assert_eq!(dir_node.v6_datalayout, EROFS_INODE_FLAT_INLINE); assert_eq!(dir_node.v6_offset, 96); @@ -952,7 +974,7 @@ mod tests { assert_eq!(bootstrap_ctx.offset, 8192 + 4096 + 8192 + 8192 + 4096); dir_node - .v6_set_dir_offset(&mut bootstrap_ctx, 4096 + 2048) + .v6_set_dir_offset(&mut bootstrap_ctx, 4096 + 2048, EROFS_BLOCK_SIZE_4096) .unwrap(); assert_eq!(dir_node.v6_datalayout, EROFS_INODE_FLAT_INLINE); assert_eq!(dir_node.v6_offset, 8192 + 4096 + 8192 + 8192 + 4096); @@ -966,7 +988,7 @@ mod tests { ); dir_node - .v6_set_dir_offset(&mut bootstrap_ctx, 1985) + .v6_set_dir_offset(&mut bootstrap_ctx, 1985, EROFS_BLOCK_SIZE_4096) .unwrap(); assert_eq!(dir_node.v6_datalayout, EROFS_INODE_FLAT_INLINE); assert_eq!(dir_node.v6_offset, 8192 + 4096 + 8192 + 8192 + 4096 + 8192); @@ -977,7 +999,7 @@ mod tests { bootstrap_ctx.align_offset(EROFS_INODE_SLOT_SIZE as u64); dir_node - .v6_set_dir_offset(&mut bootstrap_ctx, 1984) + .v6_set_dir_offset(&mut bootstrap_ctx, 1984, EROFS_BLOCK_SIZE_4096) .unwrap(); assert_eq!(dir_node.v6_datalayout, EROFS_INODE_FLAT_INLINE); assert_eq!( diff --git a/rafs/src/builder/mod.rs b/rafs/src/builder/mod.rs index b2f6eb93bf2..d6b901187f8 100644 --- a/rafs/src/builder/mod.rs +++ b/rafs/src/builder/mod.rs @@ -105,6 +105,7 @@ fn dump_bootstrap( )?; if ctx.blob_inline_meta { + assert_ne!(ctx.conversion_type, ConversionType::TarToTarfs); // Ensure the blob object is created in case of no chunks generated for the blob. let (_, blob_ctx) = blob_mgr .get_or_create_current_blob(ctx) @@ -159,6 +160,7 @@ fn dump_toc( blob_writer: &mut ArtifactWriter, ) -> Result<()> { if ctx.features.is_enabled(Feature::BlobToc) { + assert_ne!(ctx.conversion_type, ConversionType::TarToTarfs); let mut hasher = RafsDigest::hasher(digest::Algorithm::Sha256); let data = blob_ctx.entry_list.as_bytes().to_vec(); let toc_size = data.len() as u64; @@ -178,8 +180,11 @@ fn finalize_blob( blob_writer: &mut ArtifactWriter, ) -> Result<()> { if let Some((_, blob_ctx)) = blob_mgr.get_current_blob() { - dump_toc(ctx, blob_ctx, blob_writer)?; + let is_tarfs = ctx.conversion_type == ConversionType::TarToTarfs; + if !is_tarfs { + dump_toc(ctx, blob_ctx, blob_writer)?; + } if !ctx.conversion_type.is_to_ref() { blob_ctx.compressed_blob_size = blob_writer.pos()?; } @@ -191,7 +196,7 @@ fn finalize_blob( let blob_meta_id = if ctx.blob_id.is_empty() { format!("{:x}", hash) } else { - assert!(!ctx.conversion_type.is_to_ref()); + assert!(!ctx.conversion_type.is_to_ref() || is_tarfs); ctx.blob_id.clone() }; @@ -214,7 +219,8 @@ fn finalize_blob( } } } - if !ctx.blob_inline_meta { + // Tarfs mode only has tar stream and meta blob, there's no data blob. + if !ctx.blob_inline_meta && !is_tarfs { blob_ctx.blob_meta_digest = hash.into(); blob_ctx.blob_meta_size = blob_writer.pos()?; } @@ -223,7 +229,11 @@ fn finalize_blob( blob_ctx.blob_id = blob_meta_id.clone(); } - blob_writer.finalize(Some(blob_meta_id))?; + // Tarfs mode directly use the tar file as RAFS data blob, so no need to generate the data + // blob file. + if !is_tarfs { + blob_writer.finalize(Some(blob_meta_id))?; + } } Ok(()) diff --git a/rafs/src/builder/tarball.rs b/rafs/src/builder/tarball.rs index de72514dad2..a9558740426 100644 --- a/rafs/src/builder/tarball.rs +++ b/rafs/src/builder/tarball.rs @@ -103,7 +103,6 @@ impl<'a> TarballTreeBuilder<'a> { .context("tarball: can not open source file for conversion")?; let reader = match self.ty { - ConversionType::TarToRafs => TarReader::File(file), ConversionType::EStargzToRafs | ConversionType::TargzToRafs => { TarReader::TarGz(Box::new(ZlibDecoder::new(file))) } @@ -130,12 +129,21 @@ impl<'a> TarballTreeBuilder<'a> { TarReader::Buf(reader) } } + ConversionType::TarToRafs => TarReader::File(file), ConversionType::TarToRef => { let reader = BufReaderInfo::from_buf_reader(BufReader::new(file)); self.ctx.blob_tar_reader = Some(reader.clone()); TarReader::Buf(reader) } - _ => return Err(anyhow!("unsupported image conversion type")), + ConversionType::TarToTarfs => { + let mut reader = BufReaderInfo::from_buf_reader(BufReader::new(file)); + self.ctx.blob_tar_reader = Some(reader.clone()); + if !self.ctx.blob_id.is_empty() { + reader.enable_digest_calculation(false); + } + TarReader::Buf(reader) + } + _ => return Err(anyhow!("tarball: unsupported image conversion type")), }; let mut tar = Archive::new(reader); tar.set_ignore_zeros(true); @@ -551,7 +559,6 @@ impl<'a> TarballTreeBuilder<'a> { // The Landmark file MUST be a regular file entry with 4 bits contents 0xf in eStargz. // It MUST be recorded to TOC as a TOCEntry. Prefetch landmark MUST be named .prefetch.landmark. // No-prefetch landmark MUST be named .no.prefetch.landmark. - // TODO: check "a regular file entry with 4 bits contents 0xf" fn is_special_files(&self, path: &Path) -> bool { (self.ty == ConversionType::EStargzToRafs || self.ty == ConversionType::EStargzToRef) && (path == Path::new("/stargz.index.json") @@ -588,7 +595,8 @@ impl Builder for TarballBuilder { | ConversionType::EStargzToRef | ConversionType::TargzToRafs | ConversionType::TargzToRef - | ConversionType::TarToRafs => { + | ConversionType::TarToRafs + | ConversionType::TarToTarfs => { if let Some(blob_stor) = ctx.blob_storage.clone() { ArtifactWriter::new(blob_stor)? } else { diff --git a/rafs/src/fs.rs b/rafs/src/fs.rs index 247f1e0996a..d4ccf3b9619 100644 --- a/rafs/src/fs.rs +++ b/rafs/src/fs.rs @@ -39,7 +39,7 @@ use nydus_utils::{ }; use crate::metadata::{ - Inode, RafsInode, RafsInodeWalkAction, RafsSuper, RafsSuperMeta, DOT, DOTDOT, + Inode, RafsInode, RafsInodeWalkAction, RafsSuper, RafsSuperFlags, RafsSuperMeta, DOT, DOTDOT, }; use crate::{RafsError, RafsIoReader, RafsResult}; @@ -616,6 +616,10 @@ impl FileSystem for Rafs { } let real_size = cmp::min(size as u64, inode_size - offset); + if self.sb.meta.flags.contains(RafsSuperFlags::TARTFS_MODE) { + return Err(enosys!("rafs: `TARFS` mode is not supported by FUSE yet")); + } + let mut result = 0; let mut descs = inode.alloc_bio_vecs(&self.device, offset, real_size as usize, true)?; assert!(!descs.is_empty() && !descs[0].is_empty()); diff --git a/rafs/src/metadata/direct_v6.rs b/rafs/src/metadata/direct_v6.rs index 04f3c417444..72cb2eff836 100644 --- a/rafs/src/metadata/direct_v6.rs +++ b/rafs/src/metadata/direct_v6.rs @@ -40,14 +40,15 @@ use crate::metadata::layout::v5::RafsV5ChunkInfo; use crate::metadata::layout::v6::{ rafsv6_load_blob_extra_info, recover_namespace, RafsV6BlobTable, RafsV6Dirent, RafsV6InodeChunkAddr, RafsV6InodeCompact, RafsV6InodeExtended, RafsV6OndiskInode, - RafsV6XattrEntry, RafsV6XattrIbodyHeader, EROFS_BLOCK_SIZE_4096, EROFS_INODE_CHUNK_BASED, - EROFS_INODE_FLAT_INLINE, EROFS_INODE_FLAT_PLAIN, EROFS_INODE_SLOT_SIZE, - EROFS_I_DATALAYOUT_BITS, EROFS_I_VERSION_BIT, EROFS_I_VERSION_BITS, + RafsV6XattrEntry, RafsV6XattrIbodyHeader, EROFS_BLOCK_SIZE_4096, EROFS_BLOCK_SIZE_512, + EROFS_INODE_CHUNK_BASED, EROFS_INODE_FLAT_INLINE, EROFS_INODE_FLAT_PLAIN, + EROFS_INODE_SLOT_SIZE, EROFS_I_DATALAYOUT_BITS, EROFS_I_VERSION_BIT, EROFS_I_VERSION_BITS, }; use crate::metadata::layout::{bytes_to_os_str, MetaRange, XattrName, XattrValue}; use crate::metadata::{ Attr, Entry, Inode, RafsBlobExtraInfo, RafsInode, RafsInodeWalkAction, RafsInodeWalkHandler, - RafsSuperBlock, RafsSuperInodes, RafsSuperMeta, RAFS_ATTR_BLOCK_SIZE, RAFS_MAX_NAME, + RafsSuperBlock, RafsSuperFlags, RafsSuperInodes, RafsSuperMeta, RAFS_ATTR_BLOCK_SIZE, + RAFS_MAX_NAME, }; use crate::{MetaType, RafsError, RafsInodeExt, RafsIoReader, RafsResult}; @@ -77,6 +78,18 @@ impl DirectMappingState { map: FileMapState::default(), } } + + fn is_tarfs(&self) -> bool { + self.meta.flags.contains(RafsSuperFlags::TARTFS_MODE) + } + + fn block_size(&self) -> u64 { + if self.is_tarfs() { + EROFS_BLOCK_SIZE_512 + } else { + EROFS_BLOCK_SIZE_4096 + } + } } struct DirectCachedInfo { @@ -100,7 +113,8 @@ impl DirectSuperBlockV6 { /// Create a new instance of `DirectSuperBlockV6`. pub fn new(meta: &RafsSuperMeta) -> Self { let state = DirectMappingState::new(meta); - let meta_offset = meta.meta_blkaddr as usize * EROFS_BLOCK_SIZE_4096 as usize; + let block_size = state.block_size(); + let meta_offset = meta.meta_blkaddr as usize * block_size as usize; let info = DirectCachedInfo { meta_offset, root_ino: meta.root_nid as Inode, @@ -220,6 +234,7 @@ impl DirectSuperBlockV6 { return Ok(chunk_map); } + let block_size = state.block_size(); let unit_size = size_of::(); if size % unit_size != 0 { return Err(std::io::Error::from_raw_os_error(libc::EINVAL)); @@ -230,7 +245,7 @@ impl DirectSuperBlockV6 { let mut v6_chunk = RafsV6InodeChunkAddr::new(); v6_chunk.set_blob_index(chunk.blob_index()); v6_chunk.set_blob_ci_index(chunk.id()); - v6_chunk.set_block_addr((chunk.uncompressed_offset() / EROFS_BLOCK_SIZE_4096) as u32); + v6_chunk.set_block_addr((chunk.uncompressed_offset() / block_size) as u32); chunk_map.insert(v6_chunk, idx); } @@ -240,8 +255,9 @@ impl DirectSuperBlockV6 { impl RafsSuperInodes for DirectSuperBlockV6 { fn get_max_ino(&self) -> Inode { + let state = self.state.load(); // The maximum inode number supported by RAFSv6 is smaller than limit of fuse-backend-rs. - (0xffff_ffffu64) * EROFS_BLOCK_SIZE_4096 / EROFS_INODE_SLOT_SIZE as u64 + (0xffff_ffffu64) * state.block_size() / EROFS_INODE_SLOT_SIZE as u64 } /// Find inode offset by ino from inode table and mmap to OndiskInode. @@ -326,7 +342,7 @@ impl OndiskInodeWrapper { offset: usize, ) -> Result { let inode = DirectSuperBlockV6::disk_inode(state, offset)?; - let blocks_count = div_round_up(inode.size(), EROFS_BLOCK_SIZE_4096); + let blocks_count = div_round_up(inode.size(), state.block_size()); Ok(OndiskInodeWrapper { mapping, @@ -360,8 +376,8 @@ impl OndiskInodeWrapper { block_index: usize, index: usize, ) -> RafsResult<&'a RafsV6Dirent> { - let offset = self.data_block_offset(inode, block_index)?; - if size_of::() * (index + 1) >= EROFS_BLOCK_SIZE_4096 as usize { + let offset = self.data_block_offset(state, inode, block_index)?; + if size_of::() * (index + 1) >= state.block_size() as usize { Err(RafsError::InvalidImageData) } else if let Some(offset) = offset.checked_add(size_of::() * index) { state @@ -384,12 +400,13 @@ impl OndiskInodeWrapper { max_entries: usize, ) -> RafsResult<&'a OsStr> { assert!(max_entries > 0); - let offset = self.data_block_offset(inode, block_index)?; + let block_size = state.block_size(); + let offset = self.data_block_offset(state, inode, block_index)?; let de = self.get_entry(state, inode, block_index, index)?; let buf: &[u8] = match index.cmp(&(max_entries - 1)) { Ordering::Less => { let next_de = self.get_entry(state, inode, block_index, index + 1)?; - if next_de.e_nameoff as u64 >= EROFS_BLOCK_SIZE_4096 { + if next_de.e_nameoff as u64 >= block_size { return Err(RafsError::InvalidImageData); } let len = next_de.e_nameoff.checked_sub(de.e_nameoff).ok_or_else(|| { @@ -410,7 +427,7 @@ impl OndiskInodeWrapper { } Ordering::Equal => { let base = de.e_nameoff as u64; - if base >= EROFS_BLOCK_SIZE_4096 { + if base >= block_size { return Err(RafsError::InvalidImageData); } @@ -419,12 +436,12 @@ impl OndiskInodeWrapper { // Because the other blocks should be fully used, while the last may not. let block_count = self.blocks_count() as usize; let len = match block_count.cmp(&(block_index + 1)) { - Ordering::Greater => (EROFS_BLOCK_SIZE_4096 - base) as usize, + Ordering::Greater => (block_size - base) as usize, Ordering::Equal => { - if self.size() % EROFS_BLOCK_SIZE_4096 == 0 { - EROFS_BLOCK_SIZE_4096 as usize + if self.size() % block_size == 0 { + block_size as usize } else { - (self.size() % EROFS_BLOCK_SIZE_4096 - base) as usize + (self.size() % block_size - base) as usize } } Ordering::Less => return Err(RafsError::InvalidImageData), @@ -462,7 +479,12 @@ impl OndiskInodeWrapper { // 3 - inode compression D: inode, [xattrs], map_header, extents ... | ... // 4 - inode chunk-based E: inode, [xattrs], chunk indexes ... | ... // 5~7 - reserved - fn data_block_offset(&self, inode: &dyn RafsV6OndiskInode, index: usize) -> RafsResult { + fn data_block_offset<'a>( + &self, + state: &'a Guard>, + inode: &dyn RafsV6OndiskInode, + index: usize, + ) -> RafsResult { const VALID_MODE_BITS: u16 = ((1 << EROFS_I_DATALAYOUT_BITS) - 1) << EROFS_I_VERSION_BITS | ((1 << EROFS_I_VERSION_BITS) - 1); if inode.format() & !VALID_MODE_BITS != 0 || index > u32::MAX as usize { @@ -471,9 +493,9 @@ impl OndiskInodeWrapper { let layout = inode.format() >> EROFS_I_VERSION_BITS; match layout { - EROFS_INODE_FLAT_PLAIN => Self::flat_data_block_offset(inode, index), + EROFS_INODE_FLAT_PLAIN => Self::flat_data_block_offset(state, inode, index), EROFS_INODE_FLAT_INLINE => match self.blocks_count().cmp(&(index as u64 + 1)) { - Ordering::Greater => Self::flat_data_block_offset(inode, index), + Ordering::Greater => Self::flat_data_block_offset(state, inode, index), Ordering::Equal => { Ok(self.offset as usize + Self::inode_size(inode) + Self::xattr_size(inode)) } @@ -483,13 +505,17 @@ impl OndiskInodeWrapper { } } - fn flat_data_block_offset(inode: &dyn RafsV6OndiskInode, index: usize) -> RafsResult { + fn flat_data_block_offset( + state: &Guard>, + inode: &dyn RafsV6OndiskInode, + index: usize, + ) -> RafsResult { // `i_u` points to the Nth block let base = inode.union() as usize; if base.checked_add(index).is_none() || base + index > u32::MAX as usize { Err(RafsError::InvalidImageData) } else { - Ok((base + index) * EROFS_BLOCK_SIZE_4096 as usize) + Ok((base + index) * state.block_size() as usize) } } @@ -1001,7 +1027,7 @@ impl RafsInode for OndiskInodeWrapper { ))); } let offset = self - .data_block_offset(inode, 0) + .data_block_offset(&state, inode, 0) .map_err(err_invalidate_data)?; let buf: &[u8] = state.map.get_slice(offset, inode.size() as usize)?; Ok(bytes_to_os_str(buf).to_os_string()) @@ -1257,6 +1283,17 @@ impl RafsInodeExt for OndiskInodeWrapper { blob_index, chunk_index )) }) + } else if state.is_tarfs() { + let blob_index = chunk_addr.blob_index(); + let chunk_index = chunk_addr.blob_ci_index(); + let offset = (chunk_addr.block_addr() as u64) << 9; + let size = if idx == self.get_chunk_count() - 1 { + (self.size() % self.chunk_size() as u64) as u32 + } else { + self.chunk_size() + }; + let chunk = TarfsChunkInfo::new(blob_index, chunk_index, offset, size); + Ok(Arc::new(chunk)) } else { let mut chunk_map = self.mapping.info.chunk_map.lock().unwrap(); if chunk_map.is_none() { @@ -1369,3 +1406,63 @@ impl BlobV5ChunkInfo for DirectChunkInfoV6 { impl_chunkinfo_getter!(file_offset, u64); impl_chunkinfo_getter!(flags, BlobChunkFlags); } + +/// Rafs v6 fake ChunkInfo for Tarfs. +pub(crate) struct TarfsChunkInfo { + blob_index: u32, + chunk_index: u32, + offset: u64, + size: u32, +} + +impl TarfsChunkInfo { + /// Create a new instance of [TarfsChunkInfo]. + pub fn new(blob_index: u32, chunk_index: u32, offset: u64, size: u32) -> Self { + TarfsChunkInfo { + blob_index, + chunk_index, + offset, + size, + } + } +} + +const TARFS_DIGEST: RafsDigest = RafsDigest { data: [0u8; 32] }; + +impl BlobChunkInfo for TarfsChunkInfo { + fn chunk_id(&self) -> &RafsDigest { + &TARFS_DIGEST + } + + fn id(&self) -> u32 { + self.chunk_index + } + + fn blob_index(&self) -> u32 { + self.blob_index + } + + fn compressed_offset(&self) -> u64 { + self.offset + } + + fn compressed_size(&self) -> u32 { + self.size + } + + fn uncompressed_offset(&self) -> u64 { + self.offset + } + + fn uncompressed_size(&self) -> u32 { + self.size + } + + fn is_compressed(&self) -> bool { + false + } + + fn as_any(&self) -> &dyn Any { + self + } +} diff --git a/rafs/src/metadata/layout/v6.rs b/rafs/src/metadata/layout/v6.rs index fd3c60edff7..e2b54afaecd 100644 --- a/rafs/src/metadata/layout/v6.rs +++ b/rafs/src/metadata/layout/v6.rs @@ -32,6 +32,9 @@ pub const EROFS_INODE_SLOT_SIZE: usize = 1 << EROFS_INODE_SLOT_BITS; pub const EROFS_BLOCK_BITS_12: u8 = 12; /// EROFS logical block size. pub const EROFS_BLOCK_SIZE_4096: u64 = 1u64 << EROFS_BLOCK_BITS_12; +pub const EROFS_BLOCK_BITS_9: u8 = 9; +/// EROFS logical block size. +pub const EROFS_BLOCK_SIZE_512: u64 = 1u64 << EROFS_BLOCK_BITS_9; /// Offset of EROFS super block. pub const EROFS_SUPER_OFFSET: u16 = 1024; @@ -97,7 +100,7 @@ pub struct RafsV6SuperBlock { s_checksum: u32, /// Compatible filesystem features. s_feature_compat: u32, - /// Bits of block size. Only 12 is supported, thus block_size == PAGE_SIZE(4096). + /// Bits of block size, 4K or 512 bytes. s_blkszbits: u8, /// Number of extended superblock slots, ignored by Rafs v6. /// `superblock size = 128(size of RafsV6SuperBlock) + s_extslots * 16`. @@ -163,7 +166,11 @@ impl RafsV6SuperBlock { meta_size ))); } - let meta_addr = u32::from_le(self.s_meta_blkaddr) as u64 * EROFS_BLOCK_SIZE_4096; + let meta_addr = if self.s_blkszbits == EROFS_BLOCK_BITS_9 { + u32::from_le(self.s_meta_blkaddr) as u64 * EROFS_BLOCK_SIZE_512 + } else { + u32::from_le(self.s_meta_blkaddr) as u64 * EROFS_BLOCK_SIZE_4096 + }; if meta_addr > meta_size { return Err(einval!(format!( "invalid Rafs v6 meta block address 0x{:x}, meta file size 0x{:x}", @@ -185,7 +192,7 @@ impl RafsV6SuperBlock { ))); } - if self.s_blkszbits != EROFS_BLOCK_BITS_12 { + if self.s_blkszbits != EROFS_BLOCK_BITS_12 && self.s_blkszbits != EROFS_BLOCK_BITS_9 { return Err(einval!(format!( "invalid block size bits {} in Rafsv6 superblock", self.s_blkszbits @@ -296,8 +303,15 @@ impl RafsV6SuperBlock { /// Set EROFS meta block address. pub fn set_meta_addr(&mut self, meta_addr: u64) { - assert!((meta_addr / EROFS_BLOCK_SIZE_4096) <= u32::MAX as u64); - self.s_meta_blkaddr = u32::to_le((meta_addr / EROFS_BLOCK_SIZE_4096) as u32); + if self.s_blkszbits == EROFS_BLOCK_BITS_12 { + assert!((meta_addr / EROFS_BLOCK_SIZE_4096) <= u32::MAX as u64); + self.s_meta_blkaddr = u32::to_le((meta_addr / EROFS_BLOCK_SIZE_4096) as u32); + } else if self.s_blkszbits == EROFS_BLOCK_BITS_9 { + assert!((meta_addr / EROFS_BLOCK_SIZE_512) <= u32::MAX as u64); + self.s_meta_blkaddr = u32::to_le((meta_addr / EROFS_BLOCK_SIZE_512) as u32); + } else { + error!("v6: unsupported block bits {}", self.s_blkszbits); + } } /// Get device table offset. @@ -305,6 +319,12 @@ impl RafsV6SuperBlock { u16::from_le(self.s_devt_slotoff) as u64 * size_of::() as u64 } + /// Set bits of block size. + pub fn set_block_bits(&mut self, block_bits: u8) { + assert!(block_bits == EROFS_BLOCK_BITS_12 || block_bits == EROFS_BLOCK_BITS_9); + self.s_blkszbits = block_bits; + } + impl_pub_getter_setter!(magic, set_magic, s_magic, u32); impl_pub_getter_setter!(extra_devices, set_extra_devices, s_extra_devices, u16); } @@ -530,6 +550,12 @@ impl RafsV6SuperBlockExt { self.s_flags |= RafsSuperFlags::INLINED_CHUNK_DIGEST.bits(); } + /// Enable `tarfs` mode, which directly use a tar stream/file as RAFS data blob and do not + /// generate any blob meta data. + pub fn set_tarfs_mode(&mut self) { + self.s_flags |= RafsSuperFlags::TARTFS_MODE.bits(); + } + /// Set message digest algorithm to handle chunk of the Rafs filesystem. pub fn set_digester(&mut self, digester: digest::Algorithm) { let c: RafsSuperFlags = digester.into(); @@ -1077,11 +1103,16 @@ impl RafsV6InodeChunkHeader { /// If all chunks are continous in uncompressed cache file, the `chunk_size` will set to /// `inode.size().next_power_of_two()`, so EROFS can optimize page cache in this case. /// Otherwise `chunk_size` is set to RAFS filesystem's chunk size. - pub fn new(chunk_size: u64) -> Self { + pub fn new(chunk_size: u64, block_size: u64) -> Self { assert!(chunk_size.is_power_of_two()); + assert!(block_size == EROFS_BLOCK_SIZE_4096 || block_size == EROFS_BLOCK_SIZE_512); let chunk_bits = chunk_size.trailing_zeros() as u16; assert!(chunk_bits >= EROFS_BLOCK_BITS_12 as u16); - let chunk_bits = chunk_bits - EROFS_BLOCK_BITS_12 as u16; + let chunk_bits = if block_size == EROFS_BLOCK_SIZE_4096 { + chunk_bits - EROFS_BLOCK_BITS_12 as u16 + } else { + chunk_bits - EROFS_BLOCK_BITS_9 as u16 + }; assert!(chunk_bits <= EROFS_CHUNK_FORMAT_SIZE_MASK); let format = EROFS_CHUNK_FORMAT_INDEXES_FLAG | chunk_bits; @@ -1462,7 +1493,7 @@ impl RafsV6Blob { }) } - fn validate(&self, blob_index: u32, chunk_size: u32, _flags: RafsSuperFlags) -> bool { + fn validate(&self, blob_index: u32, chunk_size: u32, flags: RafsSuperFlags) -> bool { match String::from_utf8(self.blob_id.to_vec()) { Ok(v) => { if v.len() != BLOB_SHA256_LEN { @@ -1545,12 +1576,20 @@ impl RafsV6Blob { Ok(v) => v, Err(_) => return false, }; - if !blob_features.contains(BlobFeatures::ALIGNED) { - error!( - "RafsV6Blob: idx {} should have 4K-aligned feature bit", - blob_index - ); - return false; + let tarfs_mode = flags.contains(RafsSuperFlags::TARTFS_MODE); + match (blob_features.contains(BlobFeatures::ALIGNED), tarfs_mode) { + (false, false) => { + error!( + "RafsV6Blob: idx {} should have `ALIGNED` feature bit set", + blob_index + ); + return false; + } + (true, true) => { + error!("RafsV6Blob: `ALIGNED` flag should not be set for `TARFS` mode"); + return false; + } + _ => {} } let ci_offset = u64::from_le(self.ci_offset); @@ -1590,7 +1629,9 @@ impl RafsV6Blob { blob_features.bits() ); return false; - } else if ci_uncompr_size != count * size_of::() as u64 { + } else if !tarfs_mode + && ci_uncompr_size != count * size_of::() as u64 + { error!( "RafsV6Blob: idx {} invalid fields, ci_d_size {:x}, chunk_count {:x}", blob_index, ci_uncompr_size, chunk_count @@ -2125,7 +2166,7 @@ mod tests { #[test] fn test_rafs_v6_chunk_header() { let chunk_size: u32 = 1024 * 1024; - let header = RafsV6InodeChunkHeader::new(chunk_size as u64); + let header = RafsV6InodeChunkHeader::new(chunk_size as u64, EROFS_BLOCK_SIZE_4096); let target = EROFS_CHUNK_FORMAT_INDEXES_FLAG | (20 - 12) as u16; assert_eq!(u16::from_le(header.format), target); } diff --git a/rafs/src/metadata/mod.rs b/rafs/src/metadata/mod.rs index 99dccee5a3a..a8ffb02bba1 100644 --- a/rafs/src/metadata/mod.rs +++ b/rafs/src/metadata/mod.rs @@ -286,6 +286,8 @@ bitflags! { const COMPRESSION_ZSTD = 0x0000_0080; /// Chunk digests are inlined in RAFS v6 data blob. const INLINED_CHUNK_DIGEST = 0x0000_0100; + /// RAFS works in Tarfs mode, which directly uses tar streams as data blobs. + const TARTFS_MODE = 0x0000_0200; // Reserved for future compatible changes. const PRESERVED_COMPAT_7 = 0x0100_0000; diff --git a/service/src/blob_cache.rs b/service/src/blob_cache.rs index 2afa936fbe9..d38a59b00e5 100644 --- a/service/src/blob_cache.rs +++ b/service/src/blob_cache.rs @@ -17,7 +17,7 @@ use nydus_api::{ BLOB_CACHE_TYPE_META_BLOB, }; use nydus_rafs::metadata::layout::v6::{EROFS_BLOCK_BITS_12, EROFS_BLOCK_SIZE_4096}; -use nydus_rafs::metadata::{RafsBlobExtraInfo, RafsSuper}; +use nydus_rafs::metadata::{RafsBlobExtraInfo, RafsSuper, RafsSuperFlags}; use nydus_storage::cache::BlobCache; use nydus_storage::device::BlobInfo; use nydus_storage::factory::BLOB_FACTORY; @@ -43,6 +43,7 @@ pub struct MetaBlobConfig { config: Arc, blobs: Mutex>>, blob_extra_infos: HashMap, + is_tarfs_mode: bool, } impl MetaBlobConfig { @@ -70,6 +71,11 @@ impl MetaBlobConfig { self.blob_extra_infos.get(blob_id) } + /// Check whether the filesystem is in `TARFS` mode. + pub fn is_tarfs_mode(&self) -> bool { + self.is_tarfs_mode + } + fn add_data_blob(&self, blob: Arc) { self.blobs.lock().unwrap().push(blob); } @@ -130,6 +136,7 @@ impl BlobConfig { path: PathBuf, config: Arc, blob_extra_infos: HashMap, + is_tarfs_mode: bool, ) -> Self { let scoped_blob_id = generate_blob_key(&domain_id, &blob_id); @@ -140,6 +147,7 @@ impl BlobConfig { config, blobs: Mutex::new(Vec::new()), blob_extra_infos, + is_tarfs_mode, })) } @@ -394,6 +402,7 @@ impl BlobCacheMgr { path, config, blob_extra_infos, + rs.meta.flags.contains(RafsSuperFlags::TARTFS_MODE), ); // Safe to unwrap because it's a meta blob object. let meta_obj = meta.meta_config().unwrap(); @@ -612,6 +621,7 @@ mod tests { config, blobs: Mutex::new(Vec::new()), blob_extra_infos: HashMap::new(), + is_tarfs_mode: false, }; assert_eq!(blob.path(), &path); } diff --git a/service/src/block_device.rs b/service/src/block_device.rs index 3b3cc261a4a..a751ef9cbf9 100644 --- a/service/src/block_device.rs +++ b/service/src/block_device.rs @@ -16,7 +16,9 @@ use std::io::Result; use std::sync::Arc; use dbs_allocator::{Constraint, IntervalTree, NodeState, Range}; -use nydus_rafs::metadata::layout::v6::EROFS_BLOCK_BITS_12; +use nydus_rafs::metadata::layout::v6::{ + EROFS_BLOCK_BITS_12, EROFS_BLOCK_BITS_9, EROFS_BLOCK_SIZE_4096, EROFS_BLOCK_SIZE_512, +}; use tokio_uring::buf::IoBufMut; use crate::blob_cache::{BlobCacheMgr, BlobConfig, DataBlob, MetaBlob}; @@ -40,6 +42,7 @@ pub struct BlockDevice { blob_id: String, cache_mgr: Arc, ranges: IntervalTree, + is_tarfs_mode: bool, } impl BlockDevice { @@ -63,11 +66,15 @@ impl BlockDevice { } Some(BlobConfig::MetaBlob(v)) => v, }; + let is_tarfs_mode = meta_blob_config.is_tarfs_mode(); let meta_blob = MetaBlob::new(meta_blob_config.path())?; let meta_blob = Arc::new(meta_blob); - let constraint = Constraint::new(meta_blob.blocks()) - .min(0u32) - .max(meta_blob.blocks() - 1); + let blocks = if is_tarfs_mode { + meta_blob.blocks() * 8 + } else { + meta_blob.blocks() + }; + let constraint = Constraint::new(blocks).min(0u32).max(blocks - 1); let range = ranges.allocate(&constraint).ok_or_else(|| { enoent!(format!( "block_device: failed to allocate address range for meta blob {}", @@ -76,7 +83,7 @@ impl BlockDevice { })?; ranges.update(&range, BlockRange::MetaBlob(meta_blob.clone())); - let mut pos = meta_blob.blocks(); + let mut pos = blocks; let data_blobs = meta_blob_config.get_blobs(); for blob in data_blobs.iter() { let blob_info = blob.blob_info(); @@ -97,6 +104,13 @@ impl BlockDevice { ); return Err(einval!(msg)); } + if is_tarfs_mode != blob_info.features().is_tarfs() { + let msg = format!( + "block_device: inconsistent `TARFS` mode from meta and data blob {}", + blob_id + ); + return Err(einval!(msg)); + } if pos < extra_info.mapped_blkaddr { let constraint = Constraint::new(extra_info.mapped_blkaddr - pos) @@ -108,7 +122,11 @@ impl BlockDevice { ranges.update(&range, BlockRange::Hole); } - let blocks = blob_info.uncompressed_size() >> EROFS_BLOCK_BITS_12; + let blocks = if is_tarfs_mode { + blob_info.uncompressed_size() >> EROFS_BLOCK_BITS_9 + } else { + blob_info.uncompressed_size() >> EROFS_BLOCK_BITS_12 + }; if blocks > u32::MAX as u64 || blocks + extra_info.mapped_blkaddr as u64 > u32::MAX as u64 { @@ -137,6 +155,7 @@ impl BlockDevice { blob_id, cache_mgr, ranges, + is_tarfs_mode, }) } @@ -155,6 +174,33 @@ impl BlockDevice { self.blocks } + /// Get block size of block device. + pub fn block_size(&self) -> u64 { + if self.is_tarfs_mode { + EROFS_BLOCK_SIZE_512 + } else { + EROFS_BLOCK_SIZE_4096 + } + } + + /// Convert data size to number of blocks. + pub fn size_to_blocks(&self, sz: u64) -> u64 { + if self.is_tarfs_mode { + sz >> EROFS_BLOCK_BITS_9 + } else { + sz >> EROFS_BLOCK_BITS_12 + } + } + + /// Convert number of blocks to data size. + pub fn blocks_to_size(&self, blocks: u32) -> u64 { + if self.is_tarfs_mode { + (blocks as u64) << EROFS_BLOCK_BITS_9 + } else { + (blocks as u64) << EROFS_BLOCK_BITS_12 + } + } + /// Read block range [start, start + blocks) from the block device. pub async fn async_read( &self, @@ -162,16 +208,15 @@ impl BlockDevice { mut blocks: u32, mut buf: T, ) -> (Result, T) { - if start.checked_add(blocks).is_none() - || (blocks as u64) << EROFS_BLOCK_BITS_12 > buf.bytes_total() as u64 - { + let sz = self.blocks_to_size(blocks); + if start.checked_add(blocks).is_none() || sz > buf.bytes_total() as u64 { return ( Err(einval!("block_device: invalid parameters to read()")), buf, ); } - let total_size = (blocks as usize) << EROFS_BLOCK_BITS_12; + let total_size = sz as usize; let mut pos = 0; while blocks > 0 { let (range, node) = match self.ranges.get_superset(&Range::new_point(start)) { @@ -189,7 +234,7 @@ impl BlockDevice { if let NodeState::Valued(r) = node { let count = min(range.max as u32 - start + 1, blocks); - let sz = (count as usize) << EROFS_BLOCK_BITS_12 as usize; + let sz = self.blocks_to_size(count) as usize; let mut s = buf.slice(pos..pos + sz); let (res, s) = match r { BlockRange::Hole => { @@ -197,11 +242,13 @@ impl BlockDevice { (Ok(sz), s) } BlockRange::MetaBlob(m) => { - m.async_read((start as u64) << EROFS_BLOCK_BITS_12, s).await + let offset = self.blocks_to_size(start); + m.async_read(offset, s).await } BlockRange::DataBlob(d) => { - let offset = start as u64 - range.min; - d.async_read(offset << EROFS_BLOCK_BITS_12, s).await + let offset = start - range.min as u32; + let offset = self.blocks_to_size(offset); + d.async_read(offset, s).await } }; diff --git a/service/src/block_nbd.rs b/service/src/block_nbd.rs index f8ba8be4610..f4b6d3e7ad0 100644 --- a/service/src/block_nbd.rs +++ b/service/src/block_nbd.rs @@ -21,7 +21,6 @@ use std::thread::JoinHandle; use bytes::{Buf, BufMut}; use mio::Waker; use nydus_api::{BlobCacheEntry, BuildTimeInfo}; -use nydus_rafs::metadata::layout::v6::{EROFS_BLOCK_BITS_12, EROFS_BLOCK_SIZE_4096}; use nydus_storage::utils::alloc_buf; use tokio::sync::broadcast::{channel, Sender}; use tokio_uring::buf::IoBuf; @@ -85,11 +84,7 @@ impl NbdService { error!("block_nbd: failed to open NBD device {}", nbd_path); e })?; - nbd_ioctl( - nbd_dev.as_raw_fd(), - NBD_SET_BLOCK_SIZE, - EROFS_BLOCK_SIZE_4096, - )?; + nbd_ioctl(nbd_dev.as_raw_fd(), NBD_SET_BLOCK_SIZE, device.block_size())?; nbd_ioctl(nbd_dev.as_raw_fd(), NBD_SET_BLOCKS, device.blocks() as u64)?; nbd_ioctl(nbd_dev.as_raw_fd(), NBD_SET_TIMEOUT, 60)?; nbd_ioctl(nbd_dev.as_raw_fd(), NBD_CLEAR_SOCK, 0)?; @@ -226,20 +221,18 @@ impl NbdWorker { let pos = request.get_u64(); let len = request.get_u32(); + let block_size = device.block_size(); let mut code = NBD_OK; let mut data_buf = alloc_buf(len as usize); - if magic != NBD_REQUEST_MAGIC - || pos % EROFS_BLOCK_SIZE_4096 != 0 - || len as u64 % EROFS_BLOCK_SIZE_4096 != 0 - { + if magic != NBD_REQUEST_MAGIC || pos % block_size != 0 || len as u64 % block_size != 0 { warn!( "block_nbd: invalid request magic 0x{:x}, type {}, pos 0x{:x}, len 0x{:x}", magic, ty, pos, len ); code = NBD_EINVAL; } else if ty == NBD_CMD_READ { - let start = (pos >> EROFS_BLOCK_BITS_12) as u32; - let count = len >> EROFS_BLOCK_BITS_12; + let start = (pos / block_size) as u32; + let count = len / block_size as u32; let (res, buf) = device.async_read(start, count, data_buf).await; data_buf = buf; match res { diff --git a/src/bin/nydus-image/main.rs b/src/bin/nydus-image/main.rs index 952fa1a701e..053fad79277 100644 --- a/src/bin/nydus-image/main.rs +++ b/src/bin/nydus-image/main.rs @@ -182,6 +182,7 @@ fn prepare_cmd_args(bti_string: &'static str) -> App { "estargz-ref", "estargztoc-ref", "tar-rafs", + "tar-tarfs", "targz-rafs", "targz-ref", "stargz_index", @@ -649,7 +650,7 @@ impl Command { let repeatable = matches.get_flag("repeatable"); let version = Self::get_fs_version(matches)?; let chunk_size = Self::get_chunk_size(matches, conversion_type)?; - let aligned_chunk = if version.is_v6() { + let aligned_chunk = if version.is_v6() && conversion_type != ConversionType::TarToTarfs { true } else { // get_fs_version makes sure it's either v6 or v5. @@ -736,6 +737,71 @@ impl Command { ); } } + ConversionType::TarToTarfs => { + Self::ensure_file(&source_path)?; + if matches.value_source("compressor") != Some(ValueSource::DefaultValue) + && compressor != compress::Algorithm::None + { + info!( + "only compressor `None` is supported for conversion type {}, use `None` instead of {}", + conversion_type, compressor + ); + } + if matches.value_source("digester") != Some(ValueSource::DefaultValue) + && digester != digest::Algorithm::Sha256 + { + info!( + "only SHA256 is supported for conversion type {}, use SHA256 instead of {}", + conversion_type, compressor + ); + } + compressor = compress::Algorithm::None; + digester = digest::Algorithm::Sha256; + if blob_storage.is_none() { + bail!("both --blob and --blob-dir are missing"); + } else if !prefetch.disabled && prefetch.policy == PrefetchPolicy::Blob { + bail!( + "conversion type {} conflicts with '--prefetch-policy blob'", + conversion_type + ); + } + if version != RafsVersion::V6 { + bail!( + "'--fs-version 5' conflicts with conversion type '{}', only V6 is supported", + conversion_type + ); + } + if matches.get_one::("chunk-dict").is_some() { + bail!( + "conversion type '{}' conflicts with '--chunk-dict'", + conversion_type + ); + } + if parent_path.is_some() { + bail!( + "conversion type '{}' conflicts with '--parent-bootstrap'", + conversion_type + ); + } + if blob_inline_meta { + bail!( + "conversion type '{}' conflicts with '--blob-inline-meta'", + conversion_type + ); + } + if features.is_enabled(Feature::BlobToc) { + bail!( + "conversion type '{}' conflicts with '--features blob-toc'", + conversion_type + ); + } + if aligned_chunk { + bail!( + "conversion type '{}' conflicts with '--aligned-chunk'", + conversion_type + ); + } + } ConversionType::EStargzIndexToRef => { Self::ensure_file(&source_path)?; if matches.value_source("compressor") != Some(ValueSource::DefaultValue) @@ -852,6 +918,12 @@ impl Command { build_ctx.blob_features.insert(BlobFeatures::SEPARATE); Box::new(TarballBuilder::new(conversion_type)) } + ConversionType::TarToTarfs => { + if version.is_v5() { + bail!("conversion type {} conflicts with RAFS v5", conversion_type); + } + Box::new(TarballBuilder::new(conversion_type)) + } ConversionType::DirectoryToStargz | ConversionType::DirectoryToTargz | ConversionType::TarToStargz @@ -1181,15 +1253,21 @@ impl Command { .get_one::("blob") .map(|b| ArtifactStorage::SingleFile(b.into())) { + if conversion_type == ConversionType::TarToTarfs { + bail!( + "conversion type `{}` conflicts with `--blob`", + conversion_type + ); + } Ok(Some(p)) } else if let Some(d) = matches.get_one::("blob-dir").map(PathBuf::from) { if !d.exists() { - bail!("Directory to store blobs does not exist") + bail!("directory to store blobs does not exist") } Ok(Some(ArtifactStorage::FileDir(d))) } else if let Some(config_json) = matches.get_one::("backend-config") { let config: serde_json::Value = serde_json::from_str(config_json).unwrap(); - warn!("Using --backend-type=localfs is DEPRECATED. Use --blob-dir instead."); + warn!("using --backend-type=localfs is DEPRECATED. Use --blob-dir instead."); if let Some(bf) = config.get("blob_file") { // Even unwrap, it is caused by invalid json. Image creation just can't start. let b: PathBuf = bf diff --git a/storage/src/cache/cachedfile.rs b/storage/src/cache/cachedfile.rs index 14f09905384..be59aea9e65 100644 --- a/storage/src/cache/cachedfile.rs +++ b/storage/src/cache/cachedfile.rs @@ -148,6 +148,8 @@ pub(crate) struct FileCacheEntry { pub(crate) is_direct_chunkmap: bool, // The blob is for an stargz image. pub(crate) is_legacy_stargz: bool, + // The blob is for an RAFS filesystem in `TARFS` mode. + pub(crate) is_tarfs: bool, // The blob is based on ZRan decompression algorithm. pub(crate) is_zran: bool, // True if direct IO is enabled for the `self.file`, supported for fscache only. @@ -624,7 +626,10 @@ impl BlobObject for FileCacheEntry { } fn is_all_data_ready(&self) -> bool { - if let Some(b) = self.chunk_map.as_range_map() { + // Assume data from tar file is always ready. + if self.is_tarfs { + true + } else if let Some(b) = self.chunk_map.as_range_map() { b.is_range_all_ready() } else { false @@ -632,6 +637,11 @@ impl BlobObject for FileCacheEntry { } fn fetch_range_compressed(&self, offset: u64, size: u64, prefetch: bool) -> Result<()> { + // Assume data from tar file is always ready. + if self.is_tarfs { + return Ok(()); + } + let meta = self.meta.as_ref().ok_or_else(|| enoent!())?; let meta = meta.get_blob_meta().ok_or_else(|| einval!())?; let mut chunks = @@ -654,6 +664,11 @@ impl BlobObject for FileCacheEntry { } fn fetch_range_uncompressed(&self, offset: u64, size: u64) -> Result<()> { + // Assume data from tar file is always ready. + if self.is_tarfs { + return Ok(()); + } + let meta = self.meta.as_ref().ok_or_else(|| einval!())?; let meta = meta.get_blob_meta().ok_or_else(|| einval!())?; let mut chunks = meta.get_chunks_uncompressed(offset, size, self.ondemand_batch_size())?; @@ -668,6 +683,11 @@ impl BlobObject for FileCacheEntry { } fn prefetch_chunks(&self, range: &BlobIoRange) -> Result<()> { + // Assume data from tar file is always ready. + if self.is_tarfs { + return Ok(()); + } + let chunks_extended; let mut chunks = &range.chunks; if let Some(v) = self.extend_pending_chunks(chunks, self.prefetch_batch_size())? { diff --git a/storage/src/cache/filecache/mod.rs b/storage/src/cache/filecache/mod.rs index 6c7cb8f0b77..ce4a60c358e 100644 --- a/storage/src/cache/filecache/mod.rs +++ b/storage/src/cache/filecache/mod.rs @@ -16,7 +16,9 @@ use nydus_utils::metrics::BlobcacheMetrics; use crate::backend::BlobBackend; use crate::cache::cachedfile::{FileCacheEntry, FileCacheMeta}; -use crate::cache::state::{BlobStateMap, ChunkMap, DigestedChunkMap, IndexedChunkMap}; +use crate::cache::state::{ + BlobStateMap, ChunkMap, DigestedChunkMap, IndexedChunkMap, NoopChunkMap, +}; use crate::cache::worker::{AsyncPrefetchConfig, AsyncWorkerMgr}; use crate::cache::{BlobCache, BlobCacheMgr}; use crate::device::{BlobFeatures, BlobInfo}; @@ -171,6 +173,7 @@ impl FileCacheEntry { workers: Arc, ) -> Result { let is_separate_meta = blob_info.has_feature(BlobFeatures::SEPARATE); + let is_tarfs = blob_info.features().is_tarfs(); let is_zran = blob_info.has_feature(BlobFeatures::ZRAN); let blob_id = blob_info.blob_id(); let blob_meta_id = if is_separate_meta { @@ -193,69 +196,96 @@ impl FileCacheEntry { reader.clone() }; - let blob_file_path = format!("{}/{}", mgr.work_dir, blob_meta_id); - let (chunk_map, is_direct_chunkmap) = - Self::create_chunk_map(mgr, &blob_info, &blob_file_path)?; - let blob_compressed_size = Self::get_blob_size(&reader, &blob_info)?; let blob_uncompressed_size = blob_info.uncompressed_size(); let is_legacy_stargz = blob_info.is_legacy_stargz(); - // Validation is supported by RAFS v5 (which has no meta_ci) or v6 with chunk digest array. - let validation_supported = !blob_info.meta_ci_is_valid() - || blob_info.has_feature(BlobFeatures::INLINED_CHUNK_DIGEST); - let need_validation = - ((mgr.validate && validation_supported) || !is_direct_chunkmap) && !is_legacy_stargz; + + let ( + file, + meta, + chunk_map, + is_direct_chunkmap, + is_get_blob_object_supported, + need_validation, + ) = if is_tarfs { + let blob_file_path = format!("{}/{}", mgr.work_dir, blob_id); + let file = OpenOptions::new() + .create(false) + .write(false) + .read(true) + .open(&blob_file_path)?; + let chunk_map = + Arc::new(BlobStateMap::from(NoopChunkMap::new(true))) as Arc; + (file, None, chunk_map, true, true, false) + } else { + let blob_file_path = format!("{}/{}", mgr.work_dir, blob_meta_id); + let (chunk_map, is_direct_chunkmap) = + Self::create_chunk_map(mgr, &blob_info, &blob_file_path)?; + // Validation is supported by RAFS v5 (which has no meta_ci) or v6 with chunk digest array. + let validation_supported = !blob_info.meta_ci_is_valid() + || blob_info.has_feature(BlobFeatures::INLINED_CHUNK_DIGEST); + let need_validation = ((mgr.validate && validation_supported) || !is_direct_chunkmap) + && !is_legacy_stargz; + // Set cache file to its expected size. + let suffix = if mgr.cache_raw_data { + ".blob.raw" + } else { + ".blob.data" + }; + let blob_data_file_path = blob_file_path.clone() + suffix; + let file = OpenOptions::new() + .create(true) + .write(true) + .read(true) + .open(&blob_data_file_path)?; + let file_size = file.metadata()?.len(); + let cached_file_size = if mgr.cache_raw_data { + blob_info.compressed_data_size() + } else { + blob_info.uncompressed_size() + }; + if file_size == 0 { + file.set_len(cached_file_size)?; + } else if cached_file_size != 0 && file_size != cached_file_size { + let msg = format!( + "blob data file size doesn't match: got 0x{:x}, expect 0x{:x}", + file_size, cached_file_size + ); + return Err(einval!(msg)); + } + let meta = if blob_info.meta_ci_is_valid() { + let meta = FileCacheMeta::new( + blob_file_path, + blob_info.clone(), + Some(blob_meta_reader), + Some(runtime.clone()), + false, + need_validation, + )?; + Some(meta) + } else { + None + }; + let is_get_blob_object_supported = meta.is_some() && is_direct_chunkmap; + ( + file, + meta, + chunk_map, + is_direct_chunkmap, + is_get_blob_object_supported, + need_validation, + ) + }; + trace!( - "filecache entry: is_raw_data {}, direct {}, legacy_stargz {}, separate_meta {}, zran {}", + "filecache entry: is_raw_data {}, direct {}, legacy_stargz {}, separate_meta {}, tarfs {}, zran {}", mgr.cache_raw_data, is_direct_chunkmap, is_legacy_stargz, is_separate_meta, + is_tarfs, is_zran, ); - - // Set cache file to its expected size. - let suffix = if mgr.cache_raw_data { - ".blob.raw" - } else { - ".blob.data" - }; - let blob_data_file_path = blob_file_path.clone() + suffix; - let file = OpenOptions::new() - .create(true) - .write(true) - .read(true) - .open(&blob_data_file_path)?; - let file_size = file.metadata()?.len(); - let cached_file_size = if mgr.cache_raw_data { - blob_info.compressed_data_size() - } else { - blob_info.uncompressed_size() - }; - if file_size == 0 { - file.set_len(cached_file_size)?; - } else if cached_file_size != 0 && file_size != cached_file_size { - let msg = format!( - "blob data file size doesn't match: got 0x{:x}, expect 0x{:x}", - file_size, cached_file_size - ); - return Err(einval!(msg)); - } - let meta = if blob_info.meta_ci_is_valid() { - let meta = FileCacheMeta::new( - blob_file_path, - blob_info.clone(), - Some(blob_meta_reader), - Some(runtime.clone()), - false, - need_validation, - )?; - Some(meta) - } else { - None - }; - let is_get_blob_object_supported = meta.is_some() && is_direct_chunkmap; - Ok(FileCacheEntry { blob_id, blob_info, @@ -274,6 +304,7 @@ impl FileCacheEntry { is_raw_data: mgr.cache_raw_data, is_direct_chunkmap, is_legacy_stargz, + is_tarfs, is_zran, dio_enabled: false, need_validation, diff --git a/storage/src/cache/fscache/mod.rs b/storage/src/cache/fscache/mod.rs index 7ec6767948e..f8cca7ed108 100644 --- a/storage/src/cache/fscache/mod.rs +++ b/storage/src/cache/fscache/mod.rs @@ -201,6 +201,10 @@ impl FileCacheEntry { if blob_info.has_feature(BlobFeatures::_V5_NO_EXT_BLOB_TABLE) { return Err(einval!("fscache does not support Rafs v5 blobs")); } + let is_tarfs = blob_info.features().is_tarfs(); + if is_tarfs { + return Err(einval!("fscache does not support RAFS in tarfs mode")); + } let file = blob_info .get_fscache_file() @@ -273,6 +277,7 @@ impl FileCacheEntry { is_raw_data: false, is_direct_chunkmap: true, is_legacy_stargz: blob_info.is_legacy_stargz(), + is_tarfs, is_zran, dio_enabled: true, need_validation, diff --git a/storage/src/device.rs b/storage/src/device.rs index aa81174a97d..31a603f3713 100644 --- a/storage/src/device.rs +++ b/storage/src/device.rs @@ -44,7 +44,7 @@ use crate::cache::BlobCache; use crate::factory::BLOB_FACTORY; pub(crate) const BLOB_FEATURE_INCOMPAT_MASK: u32 = 0x0000_ffff; -pub(crate) const BLOB_FEATURE_INCOMPAT_VALUE: u32 = 0x0000_003f; +pub(crate) const BLOB_FEATURE_INCOMPAT_VALUE: u32 = 0x0000_007f; bitflags! { /// Features bits for blob management. @@ -61,6 +61,8 @@ bitflags! { const SEPARATE = 0x0000_0010; /// Chunk digest array is inlined in the data blob. const INLINED_CHUNK_DIGEST = 0x0000_0020; + /// Blob is for RAFS filesystems in TARFS mode. + const TARFS = 0x0000_0040; /// Blob has TAR headers to separate contents. const HAS_TAR_HEADER = 0x1000_0000; /// Blob has Table of Content (ToC) at the tail. @@ -79,6 +81,13 @@ impl Default for BlobFeatures { } } +impl BlobFeatures { + /// Check whether the blob is for RAFS filesystems in TARFS mode. + pub fn is_tarfs(&self) -> bool { + self.contains(BlobFeatures::CAP_TAR_TOC) && self.contains(BlobFeatures::TARFS) + } +} + impl TryFrom for BlobFeatures { type Error = Error; diff --git a/storage/src/meta/mod.rs b/storage/src/meta/mod.rs index 2ef43b4f5c6..7ecafee30f5 100644 --- a/storage/src/meta/mod.rs +++ b/storage/src/meta/mod.rs @@ -309,6 +309,15 @@ impl BlobCompressionContextHeader { } } + /// Set flag indicating the blob is for RAFS filesystem in TARFS mode. + pub fn set_tarfs(&mut self, enable: bool) { + if enable { + self.s_features |= BlobFeatures::TARFS.bits(); + } else { + self.s_features &= !BlobFeatures::TARFS.bits(); + } + } + /// Get blob meta feature flags. pub fn features(&self) -> u32 { self.s_features diff --git a/utils/src/lib.rs b/utils/src/lib.rs index 852d071e320..9294cf76951 100644 --- a/utils/src/lib.rs +++ b/utils/src/lib.rs @@ -63,6 +63,13 @@ pub fn round_down_4k(x: u64) -> u64 { x & (!4095u64) } +/// Round down the value `n` to by `d`. +pub fn round_down(n: u64, d: u64) -> u64 { + debug_assert!(d != 0); + debug_assert!(d.is_power_of_two()); + n / d * d +} + pub enum DelayType { Fixed, // an exponential delay between each attempts diff --git a/utils/src/reader.rs b/utils/src/reader.rs index aef8cf0d6d6..7b09a4044f5 100644 --- a/utils/src/reader.rs +++ b/utils/src/reader.rs @@ -51,6 +51,7 @@ struct BufReaderState { /// A wrapper over `BufReader` to track current position. pub struct BufReaderInfo { + calc_digest: bool, state: Arc>>, } @@ -63,6 +64,7 @@ impl BufReaderInfo { hash: Sha256::default(), }; Self { + calc_digest: true, state: Arc::new(Mutex::new(state)), } } @@ -76,6 +78,11 @@ impl BufReaderInfo { pub fn get_hash_object(&self) -> Sha256 { self.state.lock().unwrap().hash.clone() } + + /// Enable or disable blob digest calculation. + pub fn enable_digest_calculation(&mut self, enable: bool) { + self.calc_digest = enable; + } } impl Read for BufReaderInfo { @@ -83,7 +90,7 @@ impl Read for BufReaderInfo { let mut state = self.state.lock().unwrap(); state.reader.read(buf).map(|v| { state.pos += v as u64; - if v > 0 { + if v > 0 && self.calc_digest { state.hash.digest_update(&buf[..v]); } v @@ -94,6 +101,7 @@ impl Read for BufReaderInfo { impl Clone for BufReaderInfo { fn clone(&self) -> Self { Self { + calc_digest: self.calc_digest, state: self.state.clone(), } }