diff options
Diffstat (limited to 'vendor/gix-pack/src/index')
-rw-r--r-- | vendor/gix-pack/src/index/access.rs | 11 | ||||
-rw-r--r-- | vendor/gix-pack/src/index/encode.rs | 158 | ||||
-rw-r--r-- | vendor/gix-pack/src/index/mod.rs | 2 | ||||
-rw-r--r-- | vendor/gix-pack/src/index/traverse/mod.rs | 35 | ||||
-rw-r--r-- | vendor/gix-pack/src/index/traverse/with_index.rs | 38 | ||||
-rw-r--r-- | vendor/gix-pack/src/index/traverse/with_lookup.rs | 32 | ||||
-rw-r--r-- | vendor/gix-pack/src/index/util.rs | 30 | ||||
-rw-r--r-- | vendor/gix-pack/src/index/verify.rs | 28 | ||||
-rw-r--r-- | vendor/gix-pack/src/index/write/encode.rs | 124 | ||||
-rw-r--r-- | vendor/gix-pack/src/index/write/mod.rs | 31 |
10 files changed, 252 insertions, 237 deletions
diff --git a/vendor/gix-pack/src/index/access.rs b/vendor/gix-pack/src/index/access.rs index 18fb70e2a..3b748e110 100644 --- a/vendor/gix-pack/src/index/access.rs +++ b/vendor/gix-pack/src/index/access.rs @@ -119,7 +119,7 @@ impl index::File { // NOTE: pretty much the same things as in `multi_index::File::lookup`, change things there // as well. pub fn lookup(&self, id: impl AsRef<gix_hash::oid>) -> Option<EntryIndex> { - lookup(id, &self.fan, |idx| self.oid_at_index(idx)) + lookup(id.as_ref(), &self.fan, &|idx| self.oid_at_index(idx)) } /// Given a `prefix`, find an object that matches it uniquely within this index and return `Some(Ok(entry_index))`. @@ -141,7 +141,7 @@ impl index::File { prefix, candidates, &self.fan, - |idx| self.oid_at_index(idx), + &|idx| self.oid_at_index(idx), self.num_objects, ) } @@ -206,7 +206,7 @@ pub(crate) fn lookup_prefix<'a>( prefix: gix_hash::Prefix, candidates: Option<&mut Range<EntryIndex>>, fan: &[u32; FAN_LEN], - oid_at_index: impl Fn(EntryIndex) -> &'a gix_hash::oid, + oid_at_index: &dyn Fn(EntryIndex) -> &'a gix_hash::oid, num_objects: u32, ) -> Option<PrefixLookupResult> { let first_byte = prefix.as_oid().first_byte() as usize; @@ -266,11 +266,10 @@ pub(crate) fn lookup_prefix<'a>( } pub(crate) fn lookup<'a>( - id: impl AsRef<gix_hash::oid>, + id: &gix_hash::oid, fan: &[u32; FAN_LEN], - oid_at_index: impl Fn(EntryIndex) -> &'a gix_hash::oid, + oid_at_index: &dyn Fn(EntryIndex) -> &'a gix_hash::oid, ) -> Option<EntryIndex> { - let id = id.as_ref(); let first_byte = id.first_byte() as usize; let mut upper_bound = fan[first_byte]; let mut lower_bound = if first_byte != 0 { fan[first_byte - 1] } else { 0 }; diff --git a/vendor/gix-pack/src/index/encode.rs b/vendor/gix-pack/src/index/encode.rs new file mode 100644 index 000000000..d9dad68ce --- /dev/null +++ b/vendor/gix-pack/src/index/encode.rs @@ -0,0 +1,158 @@ +use std::cmp::Ordering; + +pub(crate) const LARGE_OFFSET_THRESHOLD: u64 = 0x7fff_ffff; +pub(crate) const HIGH_BIT: u32 = 0x8000_0000; + +pub(crate) fn fanout(iter: &mut dyn ExactSizeIterator<Item = u8>) -> [u32; 256] { + let mut fan_out = [0u32; 256]; + let entries_len = iter.len() as u32; + let mut iter = iter.enumerate(); + let mut idx_and_entry = iter.next(); + let mut upper_bound = 0; + + for (offset_be, byte) in fan_out.iter_mut().zip(0u8..=255) { + *offset_be = match idx_and_entry.as_ref() { + Some((_idx, first_byte)) => match first_byte.cmp(&byte) { + Ordering::Less => unreachable!("ids should be ordered, and we make sure to keep ahead with them"), + Ordering::Greater => upper_bound, + Ordering::Equal => { + if byte == 255 { + entries_len + } else { + idx_and_entry = iter.find(|(_, first_byte)| *first_byte != byte); + upper_bound = idx_and_entry.as_ref().map_or(entries_len, |(idx, _)| *idx as u32); + upper_bound + } + } + }, + None => entries_len, + }; + } + + fan_out +} + +#[cfg(feature = "streaming-input")] +mod function { + use gix_features::{ + hash, + progress::{self, DynNestedProgress}, + }; + use std::io; + + use super::{fanout, HIGH_BIT, LARGE_OFFSET_THRESHOLD}; + + use crate::index::V2_SIGNATURE; + + struct Count<W> { + bytes: u64, + inner: W, + } + + impl<W> Count<W> { + fn new(inner: W) -> Self { + Count { bytes: 0, inner } + } + } + + impl<W> io::Write for Count<W> + where + W: io::Write, + { + fn write(&mut self, buf: &[u8]) -> io::Result<usize> { + let written = self.inner.write(buf)?; + self.bytes += written as u64; + Ok(written) + } + + fn flush(&mut self) -> io::Result<()> { + self.inner.flush() + } + } + + pub(crate) fn write_to( + out: &mut dyn io::Write, + entries_sorted_by_oid: Vec<crate::cache::delta::Item<crate::index::write::TreeEntry>>, + pack_hash: &gix_hash::ObjectId, + kind: crate::index::Version, + progress: &mut dyn DynNestedProgress, + ) -> io::Result<gix_hash::ObjectId> { + use io::Write; + assert_eq!(kind, crate::index::Version::V2, "Can only write V2 packs right now"); + assert!( + entries_sorted_by_oid.len() <= u32::MAX as usize, + "a pack cannot have more than u32::MAX objects" + ); + + // Write header + let mut out = Count::new(std::io::BufWriter::with_capacity( + 8 * 4096, + hash::Write::new(out, kind.hash()), + )); + out.write_all(V2_SIGNATURE)?; + out.write_all(&(kind as u32).to_be_bytes())?; + + progress.init(Some(4), progress::steps()); + let start = std::time::Instant::now(); + let _info = progress.add_child_with_id("writing fan-out table".into(), gix_features::progress::UNKNOWN); + let fan_out = fanout(&mut entries_sorted_by_oid.iter().map(|e| e.data.id.first_byte())); + + for value in fan_out.iter() { + out.write_all(&value.to_be_bytes())?; + } + + progress.inc(); + let _info = progress.add_child_with_id("writing ids".into(), gix_features::progress::UNKNOWN); + for entry in &entries_sorted_by_oid { + out.write_all(entry.data.id.as_slice())?; + } + + progress.inc(); + let _info = progress.add_child_with_id("writing crc32".into(), gix_features::progress::UNKNOWN); + for entry in &entries_sorted_by_oid { + out.write_all(&entry.data.crc32.to_be_bytes())?; + } + + progress.inc(); + let _info = progress.add_child_with_id("writing offsets".into(), gix_features::progress::UNKNOWN); + { + let mut offsets64 = Vec::<u64>::new(); + for entry in &entries_sorted_by_oid { + let offset: u32 = if entry.offset > LARGE_OFFSET_THRESHOLD { + assert!( + offsets64.len() < LARGE_OFFSET_THRESHOLD as usize, + "Encoding breakdown - way too many 64bit offsets" + ); + offsets64.push(entry.offset); + ((offsets64.len() - 1) as u32) | HIGH_BIT + } else { + entry.offset as u32 + }; + out.write_all(&offset.to_be_bytes())?; + } + for value in offsets64 { + out.write_all(&value.to_be_bytes())?; + } + } + + out.write_all(pack_hash.as_slice())?; + + let bytes_written_without_trailer = out.bytes; + let out = out.inner.into_inner()?; + let index_hash: gix_hash::ObjectId = out.hash.digest().into(); + out.inner.write_all(index_hash.as_slice())?; + out.inner.flush()?; + + progress.inc(); + progress.show_throughput_with( + start, + (bytes_written_without_trailer + 20) as usize, + progress::bytes().expect("unit always set"), + progress::MessageLevel::Success, + ); + + Ok(index_hash) + } +} +#[cfg(feature = "streaming-input")] +pub(crate) use function::write_to; diff --git a/vendor/gix-pack/src/index/mod.rs b/vendor/gix-pack/src/index/mod.rs index 36be2d429..8d8807442 100644 --- a/vendor/gix-pack/src/index/mod.rs +++ b/vendor/gix-pack/src/index/mod.rs @@ -141,10 +141,12 @@ pub mod init; pub(crate) mod access; pub use access::Entry; +pub(crate) mod encode; /// pub mod traverse; mod util; /// pub mod verify; /// +#[cfg(feature = "streaming-input")] pub mod write; diff --git a/vendor/gix-pack/src/index/traverse/mod.rs b/vendor/gix-pack/src/index/traverse/mod.rs index 83173f904..1edf0b1d5 100644 --- a/vendor/gix-pack/src/index/traverse/mod.rs +++ b/vendor/gix-pack/src/index/traverse/mod.rs @@ -1,9 +1,6 @@ use std::sync::atomic::AtomicBool; -use gix_features::{ - parallel, - progress::{Progress, RawProgress}, -}; +use gix_features::{parallel, progress::Progress, zlib}; use crate::index; @@ -16,6 +13,7 @@ use reduce::Reducer; mod error; pub use error::Error; +use gix_features::progress::DynNestedProgress; mod types; pub use types::{Algorithm, ProgressId, SafetyCheck, Statistics}; @@ -46,13 +44,11 @@ impl Default for Options<fn() -> crate::cache::Never> { } /// The outcome of the [`traverse()`][index::File::traverse()] method. -pub struct Outcome<P> { +pub struct Outcome { /// The checksum obtained when hashing the file, which matched the checksum contained within the file. pub actual_index_checksum: gix_hash::ObjectId, /// The statistics obtained during traversal. pub statistics: Statistics, - /// The input progress to allow reuse. - pub progress: P, } /// Traversal of pack data files using an index file @@ -77,10 +73,10 @@ impl index::File { /// /// Use [`thread_limit`][Options::thread_limit] to further control parallelism and [`check`][SafetyCheck] to define how much the passed /// objects shall be verified beforehand. - pub fn traverse<P, C, Processor, E, F>( + pub fn traverse<C, Processor, E, F>( &self, pack: &crate::data::File, - progress: P, + progress: &mut dyn DynNestedProgress, should_interrupt: &AtomicBool, processor: Processor, Options { @@ -89,12 +85,11 @@ impl index::File { check, make_pack_lookup_cache, }: Options<F>, - ) -> Result<Outcome<P>, Error<E>> + ) -> Result<Outcome, Error<E>> where - P: Progress, C: crate::cache::DecodeEntry, E: std::error::Error + Send + Sync + 'static, - Processor: FnMut(gix_object::Kind, &[u8], &index::Entry, &dyn RawProgress) -> Result<(), E> + Send + Clone, + Processor: FnMut(gix_object::Kind, &[u8], &index::Entry, &dyn Progress) -> Result<(), E> + Send + Clone, F: Fn() -> C + Send + Clone, { match traversal { @@ -123,8 +118,8 @@ impl index::File { &self, pack: &crate::data::File, check: SafetyCheck, - pack_progress: impl Progress, - index_progress: impl Progress, + pack_progress: &mut dyn Progress, + index_progress: &mut dyn Progress, should_interrupt: &AtomicBool, ) -> Result<gix_hash::ObjectId, Error<E>> where @@ -155,9 +150,10 @@ impl index::File { pack: &crate::data::File, cache: &mut C, buf: &mut Vec<u8>, - progress: &mut dyn RawProgress, + inflate: &mut zlib::Inflate, + progress: &mut dyn Progress, index_entry: &index::Entry, - processor: &mut impl FnMut(gix_object::Kind, &[u8], &index::Entry, &dyn RawProgress) -> Result<(), E>, + processor: &mut impl FnMut(gix_object::Kind, &[u8], &index::Entry, &dyn Progress) -> Result<(), E>, ) -> Result<crate::data::decode::entry::Outcome, Error<E>> where C: crate::cache::DecodeEntry, @@ -169,7 +165,8 @@ impl index::File { .decode_entry( pack_entry, buf, - |id, _| { + inflate, + &|id, _| { self.lookup(id).map(|index| { crate::data::decode::entry::ResolvedBase::InPack(pack.entry(self.pack_offset_at_index(index))) }) @@ -205,8 +202,8 @@ fn process_entry<E>( decompressed: &[u8], index_entry: &index::Entry, pack_entry_crc32: impl FnOnce() -> u32, - progress: &dyn RawProgress, - processor: &mut impl FnMut(gix_object::Kind, &[u8], &index::Entry, &dyn RawProgress) -> Result<(), E>, + progress: &dyn Progress, + processor: &mut impl FnMut(gix_object::Kind, &[u8], &index::Entry, &dyn Progress) -> Result<(), E>, ) -> Result<(), Error<E>> where E: std::error::Error + Send + Sync + 'static, diff --git a/vendor/gix-pack/src/index/traverse/with_index.rs b/vendor/gix-pack/src/index/traverse/with_index.rs index 884277c9d..91382034c 100644 --- a/vendor/gix-pack/src/index/traverse/with_index.rs +++ b/vendor/gix-pack/src/index/traverse/with_index.rs @@ -1,6 +1,7 @@ use std::sync::atomic::{AtomicBool, Ordering}; -use gix_features::{parallel, progress::Progress}; +use gix_features::parallel; +use gix_features::progress::DynNestedProgress; use super::Error; use crate::{ @@ -56,31 +57,30 @@ impl index::File { /// at the cost of memory. /// /// For more details, see the documentation on the [`traverse()`][index::File::traverse()] method. - pub fn traverse_with_index<P, Processor, E>( + pub fn traverse_with_index<Processor, E>( &self, pack: &crate::data::File, mut processor: Processor, - mut progress: P, + progress: &mut dyn DynNestedProgress, should_interrupt: &AtomicBool, Options { check, thread_limit }: Options, - ) -> Result<Outcome<P>, Error<E>> + ) -> Result<Outcome, Error<E>> where - P: Progress, - Processor: FnMut(gix_object::Kind, &[u8], &index::Entry, &dyn gix_features::progress::RawProgress) -> Result<(), E> + Processor: FnMut(gix_object::Kind, &[u8], &index::Entry, &dyn gix_features::progress::Progress) -> Result<(), E> + Send + Clone, E: std::error::Error + Send + Sync + 'static, { let (verify_result, traversal_result) = parallel::join( { - let pack_progress = progress.add_child_with_id( + let mut pack_progress = progress.add_child_with_id( format!( "Hash of pack '{}'", pack.path().file_name().expect("pack has filename").to_string_lossy() ), ProgressId::HashPackDataBytes.into(), ); - let index_progress = progress.add_child_with_id( + let mut index_progress = progress.add_child_with_id( format!( "Hash of index '{}'", self.path.file_name().expect("index has filename").to_string_lossy() @@ -88,7 +88,8 @@ impl index::File { ProgressId::HashPackIndexBytes.into(), ); move || { - let res = self.possibly_verify(pack, check, pack_progress, index_progress, should_interrupt); + let res = + self.possibly_verify(pack, check, &mut pack_progress, &mut index_progress, should_interrupt); if res.is_err() { should_interrupt.store(true, Ordering::SeqCst); } @@ -98,14 +99,17 @@ impl index::File { || -> Result<_, Error<_>> { let sorted_entries = index_entries_sorted_by_offset_ascending( self, - progress.add_child_with_id("collecting sorted index", ProgressId::CollectSortedIndexEntries.into()), + &mut progress.add_child_with_id( + "collecting sorted index".into(), + ProgressId::CollectSortedIndexEntries.into(), + ), ); /* Pack Traverse Collect sorted Entries */ let tree = crate::cache::delta::Tree::from_offsets_in_pack( pack.path(), sorted_entries.into_iter().map(Entry::from), - |e| e.index_entry.pack_offset, - |id| self.lookup(id).map(|idx| self.pack_offset_at_index(idx)), - progress.add_child_with_id("indexing", ProgressId::TreeFromOffsetsObjects.into()), + &|e| e.index_entry.pack_offset, + &|id| self.lookup(id).map(|idx| self.pack_offset_at_index(idx)), + &mut progress.add_child_with_id("indexing".into(), ProgressId::TreeFromOffsetsObjects.into()), should_interrupt, self.object_hash, )?; @@ -153,8 +157,11 @@ impl index::File { } }, traverse::Options { - object_progress: progress.add_child_with_id("Resolving", ProgressId::DecodedObjects.into()), - size_progress: progress.add_child_with_id("Decoding", ProgressId::DecodedBytes.into()), + object_progress: Box::new( + progress.add_child_with_id("Resolving".into(), ProgressId::DecodedObjects.into()), + ), + size_progress: + &mut progress.add_child_with_id("Decoding".into(), ProgressId::DecodedBytes.into()), thread_limit, should_interrupt, object_hash: self.object_hash, @@ -167,7 +174,6 @@ impl index::File { Ok(Outcome { actual_index_checksum: verify_result?, statistics: traversal_result?, - progress, }) } } diff --git a/vendor/gix-pack/src/index/traverse/with_lookup.rs b/vendor/gix-pack/src/index/traverse/with_lookup.rs index 0165e4e01..3759dae5e 100644 --- a/vendor/gix-pack/src/index/traverse/with_lookup.rs +++ b/vendor/gix-pack/src/index/traverse/with_lookup.rs @@ -1,9 +1,11 @@ use std::sync::atomic::{AtomicBool, Ordering}; +use gix_features::progress::{Count, DynNestedProgress}; use gix_features::{ parallel::{self, in_parallel_if}, progress::{self, Progress}, threading::{lock, Mutable, OwnShared}, + zlib, }; use super::{Error, Reducer}; @@ -65,37 +67,34 @@ impl index::File { /// waste while decoding objects. /// /// For more details, see the documentation on the [`traverse()`][index::File::traverse()] method. - pub fn traverse_with_lookup<P, C, Processor, E, F>( + pub fn traverse_with_lookup<C, Processor, E, F>( &self, mut processor: Processor, pack: &data::File, - mut progress: P, + progress: &mut dyn DynNestedProgress, should_interrupt: &AtomicBool, Options { thread_limit, check, make_pack_lookup_cache, }: Options<F>, - ) -> Result<Outcome<P>, Error<E>> + ) -> Result<Outcome, Error<E>> where - P: Progress, C: crate::cache::DecodeEntry, E: std::error::Error + Send + Sync + 'static, - Processor: FnMut(gix_object::Kind, &[u8], &index::Entry, &dyn gix_features::progress::RawProgress) -> Result<(), E> - + Send - + Clone, + Processor: FnMut(gix_object::Kind, &[u8], &index::Entry, &dyn Progress) -> Result<(), E> + Send + Clone, F: Fn() -> C + Send + Clone, { let (verify_result, traversal_result) = parallel::join( { - let pack_progress = progress.add_child_with_id( + let mut pack_progress = progress.add_child_with_id( format!( "Hash of pack '{}'", pack.path().file_name().expect("pack has filename").to_string_lossy() ), ProgressId::HashPackDataBytes.into(), ); - let index_progress = progress.add_child_with_id( + let mut index_progress = progress.add_child_with_id( format!( "Hash of index '{}'", self.path.file_name().expect("index has filename").to_string_lossy() @@ -103,7 +102,8 @@ impl index::File { ProgressId::HashPackIndexBytes.into(), ); move || { - let res = self.possibly_verify(pack, check, pack_progress, index_progress, should_interrupt); + let res = + self.possibly_verify(pack, check, &mut pack_progress, &mut index_progress, should_interrupt); if res.is_err() { should_interrupt.store(true, Ordering::SeqCst); } @@ -113,7 +113,10 @@ impl index::File { || { let index_entries = util::index_entries_sorted_by_offset_ascending( self, - progress.add_child_with_id("collecting sorted index", ProgressId::CollectSortedIndexEntries.into()), + &mut progress.add_child_with_id( + "collecting sorted index".into(), + ProgressId::CollectSortedIndexEntries.into(), + ), ); let (chunk_size, thread_limit, available_cores) = @@ -121,7 +124,7 @@ impl index::File { let there_are_enough_entries_to_process = || index_entries.len() > chunk_size * available_cores; let input_chunks = index_entries.chunks(chunk_size.max(chunk_size)); let reduce_progress = OwnShared::new(Mutable::new({ - let mut p = progress.add_child_with_id("Traversing", ProgressId::DecodedObjects.into()); + let mut p = progress.add_child_with_id("Traversing".into(), ProgressId::DecodedObjects.into()); p.init(Some(self.num_objects() as usize), progress::count("objects")); p })); @@ -131,6 +134,7 @@ impl index::File { ( make_pack_lookup_cache(), Vec::with_capacity(2048), // decode buffer + zlib::Inflate::default(), lock(&reduce_progress) .add_child_with_id(format!("thread {index}"), gix_features::progress::UNKNOWN), // per thread progress ) @@ -143,7 +147,7 @@ impl index::File { thread_limit, state_per_thread, move |entries: &[index::Entry], - (cache, buf, progress)| + (cache, buf, inflate, progress)| -> Result<Vec<data::decode::entry::Outcome>, Error<_>> { progress.init( Some(entries.len()), @@ -157,6 +161,7 @@ impl index::File { pack, cache, buf, + inflate, progress, index_entry, &mut processor, @@ -183,7 +188,6 @@ impl index::File { Ok(Outcome { actual_index_checksum: verify_result?, statistics: traversal_result?, - progress, }) } } diff --git a/vendor/gix-pack/src/index/util.rs b/vendor/gix-pack/src/index/util.rs index 284ee6158..2549429f9 100644 --- a/vendor/gix-pack/src/index/util.rs +++ b/vendor/gix-pack/src/index/util.rs @@ -1,10 +1,10 @@ -use std::{io, time::Instant}; +use std::time::Instant; use gix_features::progress::{self, Progress}; pub(crate) fn index_entries_sorted_by_offset_ascending( idx: &crate::index::File, - mut progress: impl Progress, + progress: &mut dyn Progress, ) -> Vec<crate::index::Entry> { progress.init(Some(idx.num_objects as usize), progress::count("entries")); let start = Instant::now(); @@ -19,29 +19,3 @@ pub(crate) fn index_entries_sorted_by_offset_ascending( progress.show_throughput(start); v } - -pub(crate) struct Count<W> { - pub bytes: u64, - pub inner: W, -} - -impl<W> Count<W> { - pub fn new(inner: W) -> Self { - Count { bytes: 0, inner } - } -} - -impl<W> io::Write for Count<W> -where - W: io::Write, -{ - fn write(&mut self, buf: &[u8]) -> io::Result<usize> { - let written = self.inner.write(buf)?; - self.bytes += written as u64; - Ok(written) - } - - fn flush(&mut self) -> io::Result<()> { - self.inner.flush() - } -} diff --git a/vendor/gix-pack/src/index/verify.rs b/vendor/gix-pack/src/index/verify.rs index 6af352ac9..d259a6a38 100644 --- a/vendor/gix-pack/src/index/verify.rs +++ b/vendor/gix-pack/src/index/verify.rs @@ -1,6 +1,6 @@ use std::sync::atomic::AtomicBool; -use gix_features::progress::Progress; +use gix_features::progress::{DynNestedProgress, Progress}; use gix_object::{bstr::ByteSlice, WriteTo}; use crate::index; @@ -15,6 +15,8 @@ pub mod integrity { #[derive(thiserror::Error, Debug)] #[allow(missing_docs)] pub enum Error { + #[error("Reserialization of an object failed")] + Io(#[from] std::io::Error), #[error("The fan at index {index} is out of order as it's larger then the following value.")] Fan { index: usize }, #[error("{kind} object {id} could not be decoded")] @@ -33,13 +35,11 @@ pub mod integrity { } /// Returned by [`index::File::verify_integrity()`][crate::index::File::verify_integrity()]. - pub struct Outcome<P> { + pub struct Outcome { /// The computed checksum of the index which matched the stored one. pub actual_index_checksum: gix_hash::ObjectId, /// The packs traversal outcome, if one was provided pub pack_traverse_statistics: Option<crate::index::traverse::Statistics>, - /// The provided progress instance. - pub progress: P, } /// Additional options to define how the integrity should be verified. @@ -136,7 +136,7 @@ impl index::File { /// of this index file, and return it if it does. pub fn verify_checksum( &self, - progress: impl Progress, + progress: &mut dyn Progress, should_interrupt: &AtomicBool, ) -> Result<gix_hash::ObjectId, checksum::Error> { crate::verify::checksum_on_disk_or_mmap( @@ -166,14 +166,13 @@ impl index::File { /// /// The given `progress` is inevitably consumed if there is an error, which is a tradeoff chosen to easily allow using `?` in the /// error case. - pub fn verify_integrity<P, C, F>( + pub fn verify_integrity<C, F>( &self, pack: Option<PackContext<'_, F>>, - mut progress: P, + progress: &mut dyn DynNestedProgress, should_interrupt: &AtomicBool, - ) -> Result<integrity::Outcome<P>, index::traverse::Error<index::verify::integrity::Error>> + ) -> Result<integrity::Outcome, index::traverse::Error<index::verify::integrity::Error>> where - P: Progress, C: crate::cache::DecodeEntry, F: Fn() -> C + Send + Clone, { @@ -214,18 +213,17 @@ impl index::File { .map(|o| integrity::Outcome { actual_index_checksum: o.actual_index_checksum, pack_traverse_statistics: Some(o.statistics), - progress: o.progress, }), None => self .verify_checksum( - progress.add_child_with_id("Sha1 of index", integrity::ProgressId::ChecksumBytes.into()), + &mut progress + .add_child_with_id("Sha1 of index".into(), integrity::ProgressId::ChecksumBytes.into()), should_interrupt, ) .map_err(Into::into) .map(|id| integrity::Outcome { actual_index_checksum: id, pack_traverse_statistics: None, - progress, }), } } @@ -237,7 +235,7 @@ impl index::File { object_kind: gix_object::Kind, buf: &[u8], index_entry: &index::Entry, - progress: &dyn gix_features::progress::RawProgress, + progress: &dyn gix_features::progress::Progress, ) -> Result<(), integrity::Error> { if let Mode::HashCrc32Decode | Mode::HashCrc32DecodeEncode = verify_mode { use gix_object::Kind::*; @@ -252,9 +250,7 @@ impl index::File { })?; if let Mode::HashCrc32DecodeEncode = verify_mode { encode_buf.clear(); - object - .write_to(&mut *encode_buf) - .expect("writing to a memory buffer never fails"); + object.write_to(&mut *encode_buf)?; if encode_buf.as_slice() != buf { let mut should_return_error = true; if let Tree = object_kind { diff --git a/vendor/gix-pack/src/index/write/encode.rs b/vendor/gix-pack/src/index/write/encode.rs deleted file mode 100644 index f1195875c..000000000 --- a/vendor/gix-pack/src/index/write/encode.rs +++ /dev/null @@ -1,124 +0,0 @@ -use std::{cmp::Ordering, io}; - -pub(crate) const LARGE_OFFSET_THRESHOLD: u64 = 0x7fff_ffff; -pub(crate) const HIGH_BIT: u32 = 0x8000_0000; - -use gix_features::{ - hash, - progress::{self, Progress}, -}; - -use crate::index::{util::Count, V2_SIGNATURE}; - -pub(crate) fn write_to( - out: impl io::Write, - entries_sorted_by_oid: Vec<crate::cache::delta::Item<crate::index::write::TreeEntry>>, - pack_hash: &gix_hash::ObjectId, - kind: crate::index::Version, - mut progress: impl Progress, -) -> io::Result<gix_hash::ObjectId> { - use io::Write; - assert_eq!(kind, crate::index::Version::V2, "Can only write V2 packs right now"); - assert!( - entries_sorted_by_oid.len() <= u32::MAX as usize, - "a pack cannot have more than u32::MAX objects" - ); - - // Write header - let mut out = Count::new(std::io::BufWriter::with_capacity( - 8 * 4096, - hash::Write::new(out, kind.hash()), - )); - out.write_all(V2_SIGNATURE)?; - out.write_all(&(kind as u32).to_be_bytes())?; - - progress.init(Some(4), progress::steps()); - let start = std::time::Instant::now(); - let _info = progress.add_child_with_id("writing fan-out table", gix_features::progress::UNKNOWN); - let fan_out = fanout(entries_sorted_by_oid.iter().map(|e| e.data.id.first_byte())); - - for value in fan_out.iter() { - out.write_all(&value.to_be_bytes())?; - } - - progress.inc(); - let _info = progress.add_child_with_id("writing ids", gix_features::progress::UNKNOWN); - for entry in &entries_sorted_by_oid { - out.write_all(entry.data.id.as_slice())?; - } - - progress.inc(); - let _info = progress.add_child_with_id("writing crc32", gix_features::progress::UNKNOWN); - for entry in &entries_sorted_by_oid { - out.write_all(&entry.data.crc32.to_be_bytes())?; - } - - progress.inc(); - let _info = progress.add_child_with_id("writing offsets", gix_features::progress::UNKNOWN); - { - let mut offsets64 = Vec::<u64>::new(); - for entry in &entries_sorted_by_oid { - let offset: u32 = if entry.offset > LARGE_OFFSET_THRESHOLD { - assert!( - offsets64.len() < LARGE_OFFSET_THRESHOLD as usize, - "Encoding breakdown - way too many 64bit offsets" - ); - offsets64.push(entry.offset); - ((offsets64.len() - 1) as u32) | HIGH_BIT - } else { - entry.offset as u32 - }; - out.write_all(&offset.to_be_bytes())?; - } - for value in offsets64 { - out.write_all(&value.to_be_bytes())?; - } - } - - out.write_all(pack_hash.as_slice())?; - - let bytes_written_without_trailer = out.bytes; - let mut out = out.inner.into_inner()?; - let index_hash: gix_hash::ObjectId = out.hash.digest().into(); - out.inner.write_all(index_hash.as_slice())?; - out.inner.flush()?; - - progress.inc(); - progress.show_throughput_with( - start, - (bytes_written_without_trailer + 20) as usize, - progress::bytes().expect("unit always set"), - progress::MessageLevel::Success, - ); - - Ok(index_hash) -} - -pub(crate) fn fanout(iter: impl ExactSizeIterator<Item = u8>) -> [u32; 256] { - let mut fan_out = [0u32; 256]; - let entries_len = iter.len() as u32; - let mut iter = iter.enumerate(); - let mut idx_and_entry = iter.next(); - let mut upper_bound = 0; - - for (offset_be, byte) in fan_out.iter_mut().zip(0u8..=255) { - *offset_be = match idx_and_entry.as_ref() { - Some((_idx, first_byte)) => match first_byte.cmp(&byte) { - Ordering::Less => unreachable!("ids should be ordered, and we make sure to keep ahead with them"), - Ordering::Greater => upper_bound, - Ordering::Equal => { - if byte == 255 { - entries_len - } else { - idx_and_entry = iter.find(|(_, first_byte)| *first_byte != byte); - upper_bound = idx_and_entry.as_ref().map_or(entries_len, |(idx, _)| *idx as u32); - upper_bound - } - } - }, - None => entries_len, - }; - } - - fan_out -} diff --git a/vendor/gix-pack/src/index/write/mod.rs b/vendor/gix-pack/src/index/write/mod.rs index 72a076a85..d1402fa86 100644 --- a/vendor/gix-pack/src/index/write/mod.rs +++ b/vendor/gix-pack/src/index/write/mod.rs @@ -1,11 +1,11 @@ use std::{convert::TryInto, io, sync::atomic::AtomicBool}; pub use error::Error; -use gix_features::progress::{self, Progress}; +use gix_features::progress::prodash::DynNestedProgress; +use gix_features::progress::{self, Count, Progress}; use crate::cache::delta::{traverse, Tree}; -pub(crate) mod encode; mod error; pub(crate) struct TreeEntry { @@ -83,13 +83,13 @@ impl crate::index::File { /// It should return `None` if the entry cannot be resolved from the pack that produced the `entries` iterator, causing /// the write operation to fail. #[allow(clippy::too_many_arguments)] - pub fn write_data_iter_to_stream<F, F2, R, P>( + pub fn write_data_iter_to_stream<F, F2, R>( version: crate::index::Version, make_resolver: F, - entries: impl Iterator<Item = Result<crate::data::input::Entry, crate::data::input::Error>>, + entries: &mut dyn Iterator<Item = Result<crate::data::input::Entry, crate::data::input::Error>>, thread_limit: Option<usize>, - mut root_progress: P, - out: impl io::Write, + root_progress: &mut dyn DynNestedProgress, + out: &mut dyn io::Write, should_interrupt: &AtomicBool, object_hash: gix_hash::Kind, pack_version: crate::data::Version, @@ -98,7 +98,6 @@ impl crate::index::File { F: FnOnce() -> io::Result<(F2, R)>, R: Send + Sync, F2: for<'r> Fn(crate::data::EntryRange, &'r R) -> Option<&'r [u8]> + Send + Clone, - P: Progress, { if version != crate::index::Version::default() { return Err(Error::Unsupported(version)); @@ -111,10 +110,10 @@ impl crate::index::File { let indexing_start = std::time::Instant::now(); root_progress.init(Some(4), progress::steps()); - let mut objects_progress = root_progress.add_child_with_id("indexing", ProgressId::IndexObjects.into()); + let mut objects_progress = root_progress.add_child_with_id("indexing".into(), ProgressId::IndexObjects.into()); objects_progress.init(Some(anticipated_num_objects), progress::count("objects")); let mut decompressed_progress = - root_progress.add_child_with_id("decompressing", ProgressId::DecompressedBytes.into()); + root_progress.add_child_with_id("decompressing".into(), ProgressId::DecompressedBytes.into()); decompressed_progress.init(None, progress::bytes()); let mut pack_entries_end: u64 = 0; @@ -199,8 +198,11 @@ impl crate::index::File { Ok::<_, Error>(()) }, traverse::Options { - object_progress: root_progress.add_child_with_id("Resolving", ProgressId::ResolveObjects.into()), - size_progress: root_progress.add_child_with_id("Decoding", ProgressId::DecodedBytes.into()), + object_progress: Box::new( + root_progress.add_child_with_id("Resolving".into(), ProgressId::ResolveObjects.into()), + ), + size_progress: &mut root_progress + .add_child_with_id("Decoding".into(), ProgressId::DecodedBytes.into()), thread_limit, should_interrupt, object_hash, @@ -211,7 +213,8 @@ impl crate::index::File { let mut items = roots; items.extend(children); { - let _progress = root_progress.add_child_with_id("sorting by id", gix_features::progress::UNKNOWN); + let _progress = + root_progress.add_child_with_id("sorting by id".into(), gix_features::progress::UNKNOWN); items.sort_by_key(|e| e.data.id); } @@ -229,12 +232,12 @@ impl crate::index::File { } None => return Err(Error::IteratorInvariantTrailer), }; - let index_hash = encode::write_to( + let index_hash = crate::index::encode::write_to( out, sorted_pack_offsets_by_oid, &pack_hash, version, - root_progress.add_child_with_id("writing index file", ProgressId::IndexBytesWritten.into()), + &mut root_progress.add_child_with_id("writing index file".into(), ProgressId::IndexBytesWritten.into()), )?; root_progress.show_throughput_with( indexing_start, |