diff --git a/builder/src/lib.rs b/builder/src/lib.rs index 185e719e144..185faabe3e5 100644 --- a/builder/src/lib.rs +++ b/builder/src/lib.rs @@ -41,6 +41,7 @@ pub use self::core::prefetch::{Prefetch, PrefetchPolicy}; pub use self::core::tree::{MetadataTreeBuilder, Tree, TreeNode}; pub use self::directory::DirectoryBuilder; pub use self::merge::Merger; +pub use self::optimize_prefetch::generate_prefetch_file_info; pub use self::optimize_prefetch::update_ctx_from_bootstrap; pub use self::optimize_prefetch::OptimizePrefetch; pub use self::stargz::StargzBuilder; diff --git a/builder/src/optimize_prefetch.rs b/builder/src/optimize_prefetch.rs index b8a7b10583d..35404ece666 100644 --- a/builder/src/optimize_prefetch.rs +++ b/builder/src/optimize_prefetch.rs @@ -15,20 +15,18 @@ use crate::NodeChunk; use crate::Path; use crate::PathBuf; use crate::Tree; -use crate::TreeNode; -use anyhow::Context; -use anyhow::{Ok, Result}; +use anyhow::{bail, Context, Result}; use nydus_api::ConfigV2; use nydus_rafs::metadata::layout::RafsBlobTable; use nydus_rafs::metadata::RafsSuper; use nydus_rafs::metadata::RafsVersion; +use nydus_storage::backend::BlobBackend; use nydus_storage::device::BlobInfo; use nydus_storage::meta::BatchContextGenerator; -use nydus_storage::meta::BlobChunkInfoV1Ondisk; +use nydus_storage::meta::BlobChunkInfoV2Ondisk; use nydus_utils::compress; use sha2::Digest; -use std::fs::File; -use std::io::{Read, Seek, Write}; +use std::cmp::{max, min}; use std::mem::size_of; use std::sync::Arc; pub struct OptimizePrefetch {} @@ -39,8 +37,63 @@ struct PrefetchBlobState { blob_writer: Box, } +#[derive(Clone)] +struct PrefetchFileRange { + offset: u64, + size: usize, +} + +#[derive(Clone)] +pub struct PrefetchFileInfo { + file: PathBuf, + ranges: Option>, +} + +impl PrefetchFileInfo { + fn from_input(input: &str) -> Result { + let parts: Vec<&str> = input.split_whitespace().collect(); + let file = PathBuf::from(parts[0]); + if !file.is_absolute() { + bail!("prefetch file path is not absolute: {}", file.display()); + } + + if parts.len() != 2 { + return Ok(PrefetchFileInfo { file, ranges: None }); + } + let range_strs = parts[1]; + let mut ranges = Vec::new(); + for range_s in range_strs.split(',') { + let range_parts: Vec<&str> = range_s.split('-').collect(); + if range_parts.len() != 2 { + return Err(anyhow!(format!( + "PrefetchFileInfo Range format is incorrect" + ))); + } + + let offset = range_parts[0] + .parse::() + .map_err(|_| anyhow!("parse offset failed"))?; + + let end = range_parts[1] + .parse::() + .map_err(|_| anyhow!("parse size failed"))?; + + let range = PrefetchFileRange { + offset, + size: (end - offset) as usize, + }; + + ranges.push(range); + } + Ok(PrefetchFileInfo { + file, + ranges: Some(ranges), + }) + } +} + impl PrefetchBlobState { - fn new(ctx: &BuildContext, blob_layer_num: u32, blobs_dir_path: &Path) -> Result { + fn new(ctx: &BuildContext, blob_layer_num: u32, output_blob_dir_path: &Path) -> Result { let mut blob_info = BlobInfo::new( blob_layer_num, String::from("prefetch-blob"), @@ -55,7 +108,7 @@ impl PrefetchBlobState { let mut blob_ctx = BlobContext::from(ctx, &blob_info, ChunkSource::Build)?; blob_ctx.blob_meta_info_enabled = true; let blob_writer = ArtifactWriter::new(crate::ArtifactStorage::FileDir( - blobs_dir_path.to_path_buf(), + output_blob_dir_path.to_path_buf(), )) .map(|writer| Box::new(writer) as Box)?; Ok(Self { @@ -73,8 +126,9 @@ impl OptimizePrefetch { ctx: &mut BuildContext, bootstrap_mgr: &mut BootstrapManager, blob_table: &mut RafsBlobTable, - blobs_dir_path: PathBuf, - prefetch_nodes: Vec, + output_blob_dir_path: PathBuf, + prefetch_files: Vec, + backend: Arc, ) -> Result { // create a new blob for prefetch layer @@ -82,24 +136,26 @@ impl OptimizePrefetch { RafsBlobTable::V5(table) => table.get_all().len(), RafsBlobTable::V6(table) => table.get_all().len(), }; - let mut blob_state = PrefetchBlobState::new(&ctx, blob_layer_num as u32, &blobs_dir_path)?; + let mut blob_state = + PrefetchBlobState::new(&ctx, blob_layer_num as u32, &output_blob_dir_path)?; let mut batch = BatchContextGenerator::new(0)?; - for node in &prefetch_nodes { + for node in prefetch_files.clone() { Self::process_prefetch_node( tree, - &node, + node, &mut blob_state, &mut batch, blob_table, - &blobs_dir_path, + backend.clone(), )?; } - let blob_mgr = Self::dump_blob(ctx, blob_table, &mut blob_state)?; + Self::dump_blob(ctx, blob_table, &mut blob_state)?; debug!("prefetch blob id: {}", ctx.blob_id); - Self::build_dump_bootstrap(tree, ctx, bootstrap_mgr, blob_table)?; + let blob_mgr = + Self::build_dump_bootstrap(tree, ctx, bootstrap_mgr, blob_table, prefetch_files)?; BuildOutput::new(&blob_mgr, &bootstrap_mgr.bootstrap_storage) } @@ -108,31 +164,69 @@ impl OptimizePrefetch { ctx: &mut BuildContext, bootstrap_mgr: &mut BootstrapManager, blob_table: &mut RafsBlobTable, - ) -> Result<()> { + prefetch_files: Vec, + ) -> Result { let mut bootstrap_ctx = bootstrap_mgr.create_ctx()?; let mut bootstrap = Bootstrap::new(tree.clone())?; // Build bootstrap bootstrap.build(ctx, &mut bootstrap_ctx)?; - let blob_table_withprefetch = match blob_table { - RafsBlobTable::V5(table) => RafsBlobTable::V5(table.clone()), - RafsBlobTable::V6(table) => RafsBlobTable::V6(table.clone()), + // Fix hardlink + for node in prefetch_files.clone() { + let file = &node.file; + if tree.get_node(&file).is_none() { + warn!( + "prefetch file {} is skipped, no need to fixing hardlink", + file.display() + ); + continue; + } + + let tree_node = tree + .get_node(&file) + .ok_or(anyhow!("failed to get node"))? + .node + .as_ref(); + let child_node = tree_node.borrow(); + let key = ( + child_node.layer_idx, + child_node.info.src_ino, + child_node.info.src_dev, + ); + let chunks = child_node.chunks.clone(); + drop(child_node); + + if let Some(indexes) = bootstrap_ctx.inode_map.get_mut(&key) { + for n in indexes.iter() { + // Rewrite blob chunks to the prefetch blob's chunks + n.borrow_mut().chunks = chunks.clone(); + } + } + } + // generate blob table with extended table + let mut blob_mgr = BlobManager::new(ctx.digester); + let blob_info = match blob_table { + RafsBlobTable::V5(table) => table.get_all(), + RafsBlobTable::V6(table) => table.get_all(), }; + blob_mgr.extend_from_blob_table(ctx, blob_info)?; + let blob_table_withprefetch = blob_mgr.to_blob_table(&ctx)?; + bootstrap.dump( ctx, &mut bootstrap_mgr.bootstrap_storage, &mut bootstrap_ctx, &blob_table_withprefetch, )?; - Ok(()) + Ok(blob_mgr) } fn dump_blob( ctx: &mut BuildContext, blob_table: &mut RafsBlobTable, blob_state: &mut PrefetchBlobState, - ) -> Result { + ) -> Result<()> { match blob_table { RafsBlobTable::V5(table) => { table.entries.push(blob_state.blob_info.clone().into()); @@ -146,9 +240,11 @@ impl OptimizePrefetch { blob_mgr.add_blob(blob_state.blob_ctx.clone()); blob_mgr.set_current_blob_index(0); Blob::finalize_blob_data(&ctx, &mut blob_mgr, blob_state.blob_writer.as_mut())?; - if let Some((_, blob_ctx)) = blob_mgr.get_current_blob() { - Blob::dump_meta_data(&ctx, blob_ctx, blob_state.blob_writer.as_mut()).unwrap(); - }; + if let RafsBlobTable::V6(_) = blob_table { + if let Some((_, blob_ctx)) = blob_mgr.get_current_blob() { + Blob::dump_meta_data(&ctx, blob_ctx, blob_state.blob_writer.as_mut()).unwrap(); + }; + } ctx.blob_id = String::from(""); blob_mgr.get_current_blob().unwrap().1.blob_id = String::from(""); finalize_blob(ctx, &mut blob_mgr, blob_state.blob_writer.as_mut())?; @@ -182,19 +278,25 @@ impl OptimizePrefetch { rewrite_blob_id(&mut table.entries, "prefetch-blob", ctx.blob_id.clone()) } } - Ok(blob_mgr) + Ok(()) } fn process_prefetch_node( tree: &mut Tree, - node: &TreeNode, + prefetch_file_info: PrefetchFileInfo, prefetch_state: &mut PrefetchBlobState, batch: &mut BatchContextGenerator, blob_table: &RafsBlobTable, - blobs_dir_path: &Path, + backend: Arc, ) -> Result<()> { + let file = prefetch_file_info.file.clone(); + if tree.get_node_mut(&file).is_none() { + warn!("prefetch file {} is bad, skip it", file.display()); + return Ok(()); + } + let tree_node = tree - .get_node_mut(&node.borrow().path()) + .get_node_mut(&file) .ok_or(anyhow!("failed to get node"))? .node .as_ref(); @@ -202,16 +304,6 @@ impl OptimizePrefetch { RafsBlobTable::V5(table) => table.get_all(), RafsBlobTable::V6(table) => table.get_all(), }; - let blob_id = tree_node - .borrow() - .chunks - .first() - .and_then(|chunk| entries.get(chunk.inner.blob_index() as usize).cloned()) - .map(|entry| entry.blob_id()) - .ok_or(anyhow!("failed to get blob id"))?; - let mut blob_file = Arc::new(File::open(blobs_dir_path.join(blob_id))?); - - tree_node.borrow_mut().layer_idx = prefetch_state.blob_info.blob_index() as u16; let mut child = tree_node.borrow_mut(); let chunks: &mut Vec = child.chunks.as_mut(); @@ -220,18 +312,36 @@ impl OptimizePrefetch { let encrypted = blob_ctx.blob_compressor != compress::Algorithm::None; for chunk in chunks { + // check the file range + if let Some(ref ranges) = prefetch_file_info.ranges { + let mut should_skip = true; + for range in ranges { + if range_overlap(chunk, range) { + should_skip = false; + break; + } + } + if should_skip { + continue; + } + } + + let blob_id = entries + .get(chunk.inner.blob_index() as usize) + .map(|entry| entry.blob_id()) + .ok_or(anyhow!("failed to get blob id"))?; + let inner = Arc::make_mut(&mut chunk.inner); + let reader = backend + .clone() + .get_reader(&blob_id.clone()) + .expect("get blob err"); let mut buf = vec![0u8; inner.compressed_size() as usize]; - blob_file.seek(std::io::SeekFrom::Start(inner.compressed_offset()))?; - blob_file.read_exact(&mut buf)?; + reader + .read(&mut buf, inner.compressed_offset()) + .expect("read blob err"); prefetch_state.blob_writer.write_all(&buf)?; - let info = batch.generate_chunk_info( - blob_ctx.current_compressed_offset, - blob_ctx.current_uncompressed_offset, - inner.uncompressed_size(), - encrypted, - )?; inner.set_blob_index(blob_info.blob_index()); if blob_ctx.chunk_count == u32::MAX { blob_ctx.chunk_count = 0; @@ -240,24 +350,36 @@ impl OptimizePrefetch { blob_ctx.chunk_count += 1; inner.set_compressed_offset(blob_ctx.current_compressed_offset); inner.set_uncompressed_offset(blob_ctx.current_uncompressed_offset); - let aligned_d_size: u64 = nydus_utils::try_round_up_4k(inner.uncompressed_size()) - .ok_or_else(|| anyhow!("invalid size"))?; + let mut aligned_d_size: u64 = inner.uncompressed_size() as u64; + if let RafsBlobTable::V6(_) = blob_table { + aligned_d_size = nydus_utils::try_round_up_4k(inner.uncompressed_size()) + .ok_or_else(|| anyhow!("invalid size"))?; + let info = batch.generate_chunk_info( + blob_ctx.current_compressed_offset, + blob_ctx.current_uncompressed_offset, + inner.uncompressed_size(), + encrypted, + )?; + blob_info.set_meta_ci_compressed_size( + (blob_info.meta_ci_compressed_size() + + size_of::() as u64) as usize, + ); + + blob_info.set_meta_ci_uncompressed_size( + (blob_info.meta_ci_uncompressed_size() + + size_of::() as u64) as usize, + ); + blob_ctx.add_chunk_meta_info(&inner, Some(info))?; + } blob_ctx.compressed_blob_size += inner.compressed_size() as u64; blob_ctx.uncompressed_blob_size += aligned_d_size; blob_ctx.current_compressed_offset += inner.compressed_size() as u64; blob_ctx.current_uncompressed_offset += aligned_d_size; - blob_ctx.add_chunk_meta_info(&inner, Some(info))?; blob_ctx.blob_hash.update(&buf); - blob_info.set_meta_ci_compressed_size( - (blob_info.meta_ci_compressed_size() + size_of::() as u64) - as usize, - ); - - blob_info.set_meta_ci_uncompressed_size( - (blob_info.meta_ci_uncompressed_size() + size_of::() as u64) - as usize, - ); + blob_info.set_compressed_size(blob_ctx.compressed_blob_size as usize); + blob_info.set_uncompressed_size(blob_ctx.uncompressed_blob_size as usize); + blob_info.set_chunk_count(blob_ctx.chunk_count as usize); } Ok(()) @@ -299,3 +421,32 @@ pub fn update_ctx_from_bootstrap( ctx.compressor = config.compressor; Ok(sb) } + +pub fn generate_prefetch_file_info(prefetch_file: &Path) -> Result> { + let content = std::fs::read_to_string(prefetch_file) + .map_err(|e| anyhow!("failed to read prefetch files from {}", e))?; + + let mut prefetch_nodes: Vec = Vec::new(); + for line in content.lines() { + if line.is_empty() || line.trim().is_empty() { + continue; + } + match PrefetchFileInfo::from_input(line) { + Ok(node) => prefetch_nodes.push(node), + Err(e) => warn!("parse prefetch node failed {}", e), + } + } + Ok(prefetch_nodes) +} + +fn range_overlap(chunk: &mut NodeChunk, range: &PrefetchFileRange) -> bool { + if max(range.offset, chunk.inner.file_offset()) + <= min( + range.offset + range.size as u64, + chunk.inner.file_offset() + chunk.inner.uncompressed_size() as u64, + ) + { + return true; + } + false +} diff --git a/contrib/nydusify/cmd/nydusify.go b/contrib/nydusify/cmd/nydusify.go index 109dd65c4ff..178718b08c2 100644 --- a/contrib/nydusify/cmd/nydusify.go +++ b/contrib/nydusify/cmd/nydusify.go @@ -200,7 +200,8 @@ func main() { Required: false, Value: false, Usage: "Enable debug log level, overwrites the 'log-level' option", - EnvVars: []string{"DEBUG_LOG_LEVEL"}}, + EnvVars: []string{"DEBUG_LOG_LEVEL"}, + }, &cli.StringFlag{ Name: "log-level", Aliases: []string{"l"}, @@ -1223,10 +1224,52 @@ func main() { Value: "0MB", Usage: "Chunk size for pushing a blob layer in chunked", }, + + &cli.StringFlag{ + Name: "source-backend-type", + Value: "", + Usage: "Type of storage backend, enable verification of file data in Nydus image if specified, possible values: 'oss', 's3', 'localfs'", + EnvVars: []string{"BACKEND_TYPE"}, + }, + &cli.StringFlag{ + Name: "source-backend-config", + Value: "", + Usage: "Json string for storage backend configuration", + EnvVars: []string{"BACKEND_CONFIG"}, + }, + &cli.PathFlag{ + Name: "source-backend-config-file", + Value: "", + TakesFile: true, + Usage: "Json configuration file for storage backend", + EnvVars: []string{"BACKEND_CONFIG_FILE"}, + }, }, Action: func(c *cli.Context) error { setupLogLevel(c) + backendType, backendConfig, err := getBackendConfig(c, "source-", false) + if err != nil { + return err + } else if backendConfig == "" { + backendType = "registry" + parsed, err := reference.ParseNormalizedNamed(c.String("target")) + if err != nil { + return err + } + + backendConfigStruct, err := utils.NewRegistryBackendConfig(parsed, c.Bool("target-insecure")) + if err != nil { + return errors.Wrap(err, "parse registry backend configuration") + } + + bytes, err := json.Marshal(backendConfigStruct) + if err != nil { + return errors.Wrap(err, "marshal registry backend configuration") + } + backendConfig = string(bytes) + } + pushChunkSize, err := humanize.ParseBytes(c.String("push-chunk-size")) if err != nil { return errors.Wrap(err, "invalid --push-chunk-size option") @@ -1248,6 +1291,9 @@ func main() { PushChunkSize: int64(pushChunkSize), PrefetchFilesPath: c.String("prefetch-files"), + + BackendType: backendType, + BackendConfig: backendConfig, } return optimizer.Optimize(context.Background(), opt) diff --git a/contrib/nydusify/pkg/optimizer/builder.go b/contrib/nydusify/pkg/optimizer/builder.go index 66f26ac23d5..84b31b6bbcc 100644 --- a/contrib/nydusify/pkg/optimizer/builder.go +++ b/contrib/nydusify/pkg/optimizer/builder.go @@ -19,9 +19,13 @@ func isSignalKilled(err error) bool { } type BuildOption struct { - BuilderPath string - PrefetchFilesPath string - BootstrapPath string + BuilderPath string + PrefetchFilesPath string + BootstrapPath string + BackendType string + BackendConfig string + // `BlobDir` is used to store optimized blob, + // Beside, `BlobDir` is also used to store the original blobs when backend is localfs BlobDir string OutputBootstrapPath string OutputJSONPath string @@ -42,7 +46,7 @@ func Build(option BuildOption) (string, error) { option.PrefetchFilesPath, "--bootstrap", option.BootstrapPath, - "--blob-dir", + "--output-blob-dir", option.BlobDir, "--output-bootstrap", option.OutputBootstrapPath, @@ -50,6 +54,13 @@ func Build(option BuildOption) (string, error) { outputJSONPath, } + if option.BackendType == "localfs" { + args = append(args, "--blob-dir", option.BlobDir) + } else { + args = append(args, "--backend-type", option.BackendType) + args = append(args, "--backend-config", option.BackendConfig) + } + ctx := context.Background() var cancel context.CancelFunc if option.Timeout != nil { diff --git a/contrib/nydusify/pkg/optimizer/optimizer.go b/contrib/nydusify/pkg/optimizer/optimizer.go index 84d35614e5f..207e9cc8292 100644 --- a/contrib/nydusify/pkg/optimizer/optimizer.go +++ b/contrib/nydusify/pkg/optimizer/optimizer.go @@ -61,6 +61,9 @@ type Opt struct { Platforms string PushChunkSize int64 + + BackendType string + BackendConfig string } // the information generated during building @@ -269,8 +272,10 @@ func Optimize(ctx context.Context, opt Opt) error { } defer os.RemoveAll(buildDir) - if err := fetchBlobs(ctx, opt, buildDir); err != nil { - return errors.Wrap(err, "prepare nydus blobs") + if opt.BackendType == "localfs" { + if err := fetchBlobs(ctx, opt, buildDir); err != nil { + return errors.Wrap(err, "prepare nydus blobs") + } } originalBootstrap := filepath.Join(buildDir, "nydus_bootstrap") @@ -289,12 +294,17 @@ func Optimize(ctx context.Context, opt Opt) error { compressAlgo := bootstrapDesc.Digest.Algorithm().String() blobDir := filepath.Join(buildDir + "/content/blobs/" + compressAlgo) + if err := os.MkdirAll(blobDir, 0755); err != nil { + return errors.Wrap(err, "create blob directory") + } outPutJSONPath := filepath.Join(buildDir, "output.json") newBootstrapPath := filepath.Join(buildDir, "optimized_bootstrap") builderOpt := BuildOption{ BuilderPath: opt.NydusImagePath, PrefetchFilesPath: opt.PrefetchFilesPath, BootstrapPath: originalBootstrap, + BackendType: opt.BackendType, + BackendConfig: opt.BackendConfig, BlobDir: blobDir, OutputBootstrapPath: newBootstrapPath, OutputJSONPath: outPutJSONPath, diff --git a/docs/nydusify.md b/docs/nydusify.md index 32e2239871b..11d6201b7ca 100644 --- a/docs/nydusify.md +++ b/docs/nydusify.md @@ -262,6 +262,24 @@ nerdctl --snapshotter nydus run \ The original container ID need to be a full container ID rather than an abbreviation. +## Optimize nydus image from prefetch files + +The nydusify optimize command can optimize a nydus image from prefetch files, prefetch files are file access patterns during container startup. This will generate a new bootstrap and a new blob wich contains all data indicated by prefetch files. + +The content of prefetch files likes this: +``` +/path/to/file1 start_offset1-end_offset1, start_offset2-end_offset2, ... +/path/to/file2 start_offset1-end_offset1, start_offset2-end_offset2, ... +``` + +``` shell +nydusify optimize \ + --nydus-image /path/to/nydus-image \ + --source myregistry/repo:tag-nydus \ + --target myregistry/repo:tag-nydus-optimized \ + --prefetch-files /path/to/prefetch-files \ +``` + ## More Nydusify Options See `nydusify convert/check/mount --help` diff --git a/src/bin/nydus-image/main.rs b/src/bin/nydus-image/main.rs index 06bd168dd03..f395d871b93 100644 --- a/src/bin/nydus-image/main.rs +++ b/src/bin/nydus-image/main.rs @@ -31,11 +31,11 @@ use nix::unistd::{getegid, geteuid}; use nydus::{get_build_time_info, setup_logging}; use nydus_api::{BuildTimeInfo, ConfigV2, LocalFsConfig}; use nydus_builder::{ - parse_chunk_dict_arg, update_ctx_from_bootstrap, ArtifactStorage, BlobCacheGenerator, - BlobCompactor, BlobManager, BootstrapManager, BuildContext, BuildOutput, Builder, - ChunkdictBlobInfo, ChunkdictChunkInfo, ConversionType, DirectoryBuilder, Feature, Features, - Generator, HashChunkDict, Merger, OptimizePrefetch, Prefetch, PrefetchPolicy, StargzBuilder, - TarballBuilder, Tree, TreeNode, WhiteoutSpec, + generate_prefetch_file_info, parse_chunk_dict_arg, update_ctx_from_bootstrap, ArtifactStorage, + BlobCacheGenerator, BlobCompactor, BlobManager, BootstrapManager, BuildContext, BuildOutput, + Builder, ChunkdictBlobInfo, ChunkdictChunkInfo, ConversionType, DirectoryBuilder, Feature, + Features, Generator, HashChunkDict, Merger, OptimizePrefetch, Prefetch, PrefetchPolicy, + StargzBuilder, TarballBuilder, Tree, WhiteoutSpec, }; use nydus_rafs::metadata::{MergeError, RafsSuper, RafsSuperConfig, RafsVersion}; @@ -555,6 +555,32 @@ fn prepare_cmd_args(bti_string: &'static str) -> App { .num_args(1), ) .arg(arg_config.clone()) + .arg( + Arg::new("backend-type") + .long("backend-type") + .help(format!( + "Type of backend [possible values: {}]", + BlobFactory::supported_backends() + .into_iter() + .filter(|x| x != "localfs") + .collect::>() + .join(", ") + )) + .required(false) + ) + .arg( + Arg::new("backend-config") + .long("backend-config") + .help("Config string of backend") + .required(false), + ) + .arg( + Arg::new("backend-config-file") + .long("backend-config-file") + .help("Config file of backend") + .conflicts_with("backend-config") + .required(false), + ) .arg( Arg::new("blob-dir") .long("blob-dir") @@ -564,12 +590,23 @@ fn prepare_cmd_args(bti_string: &'static str) -> App { "Directory for localfs storage backend, hosting data blobs and cache files", ), ) + .arg( + Arg::new("blob") + .long("blob") + .short('b') + .help("Path to RAFS data blob file") + ) .arg( Arg::new("output-bootstrap") .long("output-bootstrap") .short('O') .help("Output path of optimized bootstrap"), ) + .arg( + Arg::new("output-blob-dir") + .long("output-blob-dir") + .help("Directroy path for storing optimized blob"), + ) .arg( arg_output_json.clone(), ) @@ -1683,9 +1720,8 @@ impl Command { } fn optimize(matches: &ArgMatches, build_info: &BuildTimeInfo) -> Result<()> { - let blobs_dir_path = Self::get_blobs_dir(matches)?; - let prefetch_files = Self::get_prefetch_files(matches)?; - prefetch_files.iter().for_each(|f| println!("{}", f)); + let output_blob_dir_path = Self::get_output_blob_dir(matches)?; + let prefetch_file = Self::get_prefetch_files(matches)?; let bootstrap_path = Self::get_bootstrap(matches)?; let dst_bootstrap = match matches.get_one::("output-bootstrap") { None => ArtifactStorage::SingleFile(PathBuf::from("optimized_bootstrap")), @@ -1700,19 +1736,17 @@ impl Command { ..Default::default() }; - let sb = update_ctx_from_bootstrap(&mut build_ctx, config, bootstrap_path)?; - let mut tree = Tree::from_bootstrap(&sb, &mut ())?; - - let mut prefetch_nodes: Vec = Vec::new(); - // Init prefetch nodes - for f in prefetch_files.iter() { - let path = PathBuf::from(f); - if let Some(tree) = tree.get_node(&path) { - prefetch_nodes.push(tree.node.clone()); + let (_c, backend) = match Self::get_backend(matches, "optimizer") { + Ok((c, b)) => (c, b), + Err(e) => { + bail!("{}, --blob-dir or --backend-type must be specified", e); } - } + }; + let sb = update_ctx_from_bootstrap(&mut build_ctx, config, bootstrap_path)?; + let mut tree = Tree::from_bootstrap(&sb, &mut ())?; let mut bootstrap_mgr = BootstrapManager::new(Some(dst_bootstrap), None); + let prefetch_nodes = generate_prefetch_file_info(prefetch_file)?; let blobs = sb.superblock.get_blob_infos(); let mut blob_table = match build_ctx.fs_version { @@ -1728,8 +1762,9 @@ impl Command { &mut build_ctx, &mut bootstrap_mgr, &mut blob_table, - blobs_dir_path.to_path_buf(), + output_blob_dir_path.to_path_buf(), prefetch_nodes, + backend, ) .with_context(|| "Failed to generate prefetch bootstrap")?; @@ -1842,28 +1877,16 @@ impl Command { } } - fn get_blobs_dir(matches: &ArgMatches) -> Result<&Path> { - match matches.get_one::("blob-dir") { + fn get_output_blob_dir(matches: &ArgMatches) -> Result<&Path> { + match matches.get_one::("output-blob-dir") { Some(s) => Ok(Path::new(s)), - None => bail!("missing parameter `blob-dir`"), + None => bail!("missing parameter `output-blob-dir`"), } } - fn get_prefetch_files(matches: &ArgMatches) -> Result> { + fn get_prefetch_files(matches: &ArgMatches) -> Result<&Path> { match matches.get_one::("prefetch-files") { - Some(v) => { - let content = std::fs::read_to_string(v) - .map_err(|e| anyhow!("failed to read prefetch files from {}: {}", v, e))?; - - let mut prefetch_files: Vec = Vec::new(); - for line in content.lines() { - if line.is_empty() || line.trim().is_empty() { - continue; - } - prefetch_files.push(line.trim().to_string()); - } - Ok(prefetch_files) - } + Some(s) => Ok(Path::new(s)), None => bail!("missing parameter `prefetch-files`"), } }