summaryrefslogtreecommitdiffstats
path: root/vendor/gix-pack/src/multi_index
diff options
context:
space:
mode:
Diffstat (limited to 'vendor/gix-pack/src/multi_index')
-rw-r--r--vendor/gix-pack/src/multi_index/access.rs143
-rw-r--r--vendor/gix-pack/src/multi_index/chunk.rs276
-rw-r--r--vendor/gix-pack/src/multi_index/init.rs157
-rw-r--r--vendor/gix-pack/src/multi_index/mod.rs57
-rw-r--r--vendor/gix-pack/src/multi_index/verify.rs337
-rw-r--r--vendor/gix-pack/src/multi_index/write.rs244
6 files changed, 1214 insertions, 0 deletions
diff --git a/vendor/gix-pack/src/multi_index/access.rs b/vendor/gix-pack/src/multi_index/access.rs
new file mode 100644
index 000000000..d209cd0b9
--- /dev/null
+++ b/vendor/gix-pack/src/multi_index/access.rs
@@ -0,0 +1,143 @@
+use std::{
+ ops::Range,
+ path::{Path, PathBuf},
+};
+
+use crate::{
+ data,
+ index::PrefixLookupResult,
+ multi_index::{EntryIndex, File, PackIndex, Version},
+};
+
+/// Represents an entry within a multi index file, effectively mapping object [`IDs`][gix_hash::ObjectId] to pack data
+/// files and the offset within.
+#[derive(PartialEq, Eq, Debug, Hash, Ord, PartialOrd, Clone)]
+#[cfg_attr(feature = "serde1", derive(serde::Serialize, serde::Deserialize))]
+pub struct Entry {
+ /// The ID of the object.
+ pub oid: gix_hash::ObjectId,
+ /// The offset to the object's header in the pack data file.
+ pub pack_offset: data::Offset,
+ /// The index of the pack matching our [`File::index_names()`] slice.
+ pub pack_index: PackIndex,
+}
+
+/// Access methods
+impl File {
+ /// Returns the version of the multi-index file.
+ pub fn version(&self) -> Version {
+ self.version
+ }
+ /// Returns the path from which the multi-index file was loaded.
+ ///
+ /// Note that it might have changed in the mean time, or might have been removed as well.
+ pub fn path(&self) -> &Path {
+ &self.path
+ }
+ /// Returns the amount of indices stored in this multi-index file. It's the same as [File::index_names().len()][File::index_names()],
+ /// and returned as one past the highest known index.
+ pub fn num_indices(&self) -> PackIndex {
+ self.num_indices
+ }
+ /// Returns the total amount of objects available for lookup, and returned as one past the highest known entry index
+ pub fn num_objects(&self) -> EntryIndex {
+ self.num_objects
+ }
+ /// Returns the kind of hash function used for object ids available in this index.
+ pub fn object_hash(&self) -> gix_hash::Kind {
+ self.object_hash
+ }
+ /// Returns the checksum over the entire content of the file (excluding the checksum itself).
+ ///
+ /// It can be used to validate it didn't change after creation.
+ pub fn checksum(&self) -> gix_hash::ObjectId {
+ gix_hash::ObjectId::from(&self.data[self.data.len() - self.hash_len..])
+ }
+ /// Return all names of index files (`*.idx`) whose objects we contain.
+ ///
+ /// The corresponding pack can be found by replacing the `.idx` extension with `.pack`.
+ pub fn index_names(&self) -> &[PathBuf] {
+ &self.index_names
+ }
+}
+
+impl File {
+ /// Return the object id at the given `index`, which ranges from 0 to [File::num_objects()].
+ pub fn oid_at_index(&self, index: EntryIndex) -> &gix_hash::oid {
+ debug_assert!(index < self.num_objects, "index out of bounds");
+ let index: usize = index as usize;
+ let start = self.lookup_ofs + index * self.hash_len;
+ gix_hash::oid::from_bytes_unchecked(&self.data[start..][..self.hash_len])
+ }
+
+ /// Given a `prefix`, find an object that matches it uniquely within this index and return `Some(Ok(entry_index))`.
+ /// If there is more than one object matching the object `Some(Err(())` is returned.
+ ///
+ /// Finally, if no object matches the index, the return value is `None`.
+ ///
+ /// Pass `candidates` to obtain the set of entry-indices matching `prefix`, with the same return value as
+ /// one would have received if it remained `None`. It will be empty if no object matched the `prefix`.
+ ///
+ // NOTE: pretty much the same things as in `index::File::lookup`, change things there
+ // as well.
+ pub fn lookup_prefix(
+ &self,
+ prefix: gix_hash::Prefix,
+ candidates: Option<&mut Range<EntryIndex>>,
+ ) -> Option<PrefixLookupResult> {
+ crate::index::access::lookup_prefix(
+ prefix,
+ candidates,
+ &self.fan,
+ |idx| self.oid_at_index(idx),
+ self.num_objects,
+ )
+ }
+
+ /// Find the index ranging from 0 to [File::num_objects()] that belongs to data associated with `id`, or `None` if it wasn't found.
+ ///
+ /// Use this index for finding additional information via [`File::pack_id_and_pack_offset_at_index()`].
+ pub fn lookup(&self, id: impl AsRef<gix_hash::oid>) -> Option<EntryIndex> {
+ crate::index::access::lookup(id, &self.fan, |idx| self.oid_at_index(idx))
+ }
+
+ /// Given the `index` ranging from 0 to [File::num_objects()], return the pack index and its absolute offset into the pack.
+ ///
+ /// The pack-index refers to an entry in the [`index_names`][File::index_names()] list, from which the pack can be derived.
+ pub fn pack_id_and_pack_offset_at_index(&self, index: EntryIndex) -> (PackIndex, data::Offset) {
+ const OFFSET_ENTRY_SIZE: usize = 4 + 4;
+ let index = index as usize;
+ let start = self.offsets_ofs + index * OFFSET_ENTRY_SIZE;
+
+ const HIGH_BIT: u32 = 1 << 31;
+
+ let pack_index = crate::read_u32(&self.data[start..][..4]);
+ let offset = &self.data[start + 4..][..4];
+ let ofs32 = crate::read_u32(offset);
+ let pack_offset = if (ofs32 & HIGH_BIT) == HIGH_BIT {
+ // We determine if large offsets are actually larger than 4GB and if not, we don't use the high-bit to signal anything
+ // but allow the presence of the large-offset chunk to signal what's happening.
+ if let Some(offsets_64) = self.large_offsets_ofs {
+ let from = offsets_64 + (ofs32 ^ HIGH_BIT) as usize * 8;
+ crate::read_u64(&self.data[from..][..8])
+ } else {
+ ofs32 as u64
+ }
+ } else {
+ ofs32 as u64
+ };
+ (pack_index, pack_offset)
+ }
+
+ /// Return an iterator over all entries within this file.
+ pub fn iter(&self) -> impl Iterator<Item = Entry> + '_ {
+ (0..self.num_objects).map(move |idx| {
+ let (pack_index, pack_offset) = self.pack_id_and_pack_offset_at_index(idx);
+ Entry {
+ oid: self.oid_at_index(idx).to_owned(),
+ pack_offset,
+ pack_index,
+ }
+ })
+ }
+}
diff --git a/vendor/gix-pack/src/multi_index/chunk.rs b/vendor/gix-pack/src/multi_index/chunk.rs
new file mode 100644
index 000000000..7ed8eebcb
--- /dev/null
+++ b/vendor/gix-pack/src/multi_index/chunk.rs
@@ -0,0 +1,276 @@
+/// Information for the chunk about index names
+pub mod index_names {
+ use std::path::{Path, PathBuf};
+
+ use gix_object::bstr::{BString, ByteSlice};
+
+ /// The ID used for the index-names chunk.
+ pub const ID: gix_chunk::Id = *b"PNAM";
+
+ ///
+ pub mod decode {
+ use gix_object::bstr::BString;
+
+ /// The error returned by [from_bytes()][super::from_bytes()].
+ #[derive(Debug, thiserror::Error)]
+ #[allow(missing_docs)]
+ pub enum Error {
+ #[error("The pack names were not ordered alphabetically.")]
+ NotOrderedAlphabetically,
+ #[error("Each pack path name must be terminated with a null byte")]
+ MissingNullByte,
+ #[error("Couldn't turn path '{path}' into OS path due to encoding issues")]
+ PathEncoding { path: BString },
+ #[error("non-padding bytes found after all paths were read.")]
+ UnknownTrailerBytes,
+ }
+ }
+
+ /// Parse null-separated index names from the given `chunk` of bytes and the expected number of packs and indices.
+ /// Ignore padding bytes which are typically \0.
+ pub fn from_bytes(mut chunk: &[u8], num_packs: u32) -> Result<Vec<PathBuf>, decode::Error> {
+ let mut out = Vec::new();
+ for _ in 0..num_packs {
+ let null_byte_pos = chunk.find_byte(b'\0').ok_or(decode::Error::MissingNullByte)?;
+
+ let path = &chunk[..null_byte_pos];
+ let path = gix_path::try_from_byte_slice(path)
+ .map_err(|_| decode::Error::PathEncoding {
+ path: BString::from(path),
+ })?
+ .to_owned();
+
+ if let Some(previous) = out.last() {
+ if previous >= &path {
+ return Err(decode::Error::NotOrderedAlphabetically);
+ }
+ }
+ out.push(path);
+
+ chunk = &chunk[null_byte_pos + 1..];
+ }
+
+ if !chunk.is_empty() && !chunk.iter().all(|b| *b == 0) {
+ return Err(decode::Error::UnknownTrailerBytes);
+ }
+ // NOTE: git writes garbage into this chunk, usually extra \0 bytes, which we simply ignore. If we were strict
+ // about it we couldn't read this chunk data at all.
+ Ok(out)
+ }
+
+ /// Calculate the size on disk for our chunk with the given index paths. Note that these are expected to have been processed already
+ /// to actually be file names.
+ pub fn storage_size(paths: impl IntoIterator<Item = impl AsRef<Path>>) -> u64 {
+ let mut count = 0u64;
+ for path in paths {
+ let path = path.as_ref();
+ let ascii_path = path.to_str().expect("UTF-8 compatible paths");
+ assert!(
+ ascii_path.is_ascii(),
+ "must use ascii bytes for correct size computation"
+ );
+ count += (ascii_path.as_bytes().len() + 1/* null byte */) as u64
+ }
+
+ let needed_alignment = CHUNK_ALIGNMENT - (count % CHUNK_ALIGNMENT);
+ if needed_alignment < CHUNK_ALIGNMENT {
+ count += needed_alignment;
+ }
+ count
+ }
+
+ /// Write all `paths` in order to `out`, including padding.
+ pub fn write(
+ paths: impl IntoIterator<Item = impl AsRef<Path>>,
+ mut out: impl std::io::Write,
+ ) -> std::io::Result<()> {
+ let mut written_bytes = 0;
+ for path in paths {
+ let path = path.as_ref().to_str().expect("UTF-8 path");
+ out.write_all(path.as_bytes())?;
+ out.write_all(&[0])?;
+ written_bytes += path.as_bytes().len() as u64 + 1;
+ }
+
+ let needed_alignment = CHUNK_ALIGNMENT - (written_bytes % CHUNK_ALIGNMENT);
+ if needed_alignment < CHUNK_ALIGNMENT {
+ let padding = [0u8; CHUNK_ALIGNMENT as usize];
+ out.write_all(&padding[..needed_alignment as usize])?;
+ }
+ Ok(())
+ }
+
+ const CHUNK_ALIGNMENT: u64 = 4;
+}
+
+/// Information for the chunk with the fanout table
+pub mod fanout {
+ use std::convert::TryInto;
+
+ use crate::multi_index;
+
+ /// The size of the fanout table
+ pub const SIZE: usize = 4 * 256;
+
+ /// The id uniquely identifying the fanout table.
+ pub const ID: gix_chunk::Id = *b"OIDF";
+
+ /// Decode the fanout table contained in `chunk`, or return `None` if it didn't have the expected size.
+ pub fn from_bytes(chunk: &[u8]) -> Option<[u32; 256]> {
+ if chunk.len() != SIZE {
+ return None;
+ }
+ let mut out = [0; 256];
+ for (c, f) in chunk.chunks(4).zip(out.iter_mut()) {
+ *f = u32::from_be_bytes(c.try_into().unwrap());
+ }
+ out.into()
+ }
+
+ /// Write the fanout for the given entries, which must be sorted by oid
+ pub(crate) fn write(
+ sorted_entries: &[multi_index::write::Entry],
+ mut out: impl std::io::Write,
+ ) -> std::io::Result<()> {
+ let fanout = crate::index::write::encode::fanout(sorted_entries.iter().map(|e| e.id.first_byte()));
+
+ for value in fanout.iter() {
+ out.write_all(&value.to_be_bytes())?;
+ }
+ Ok(())
+ }
+}
+
+/// Information about the oid lookup table.
+pub mod lookup {
+ use std::ops::Range;
+
+ use crate::multi_index;
+
+ /// The id uniquely identifying the oid lookup table.
+ pub const ID: gix_chunk::Id = *b"OIDL";
+
+ /// Return the amount of bytes needed to store the data on disk for the given amount of `entries`
+ pub fn storage_size(entries: usize, object_hash: gix_hash::Kind) -> u64 {
+ (entries * object_hash.len_in_bytes()) as u64
+ }
+
+ pub(crate) fn write(
+ sorted_entries: &[multi_index::write::Entry],
+ mut out: impl std::io::Write,
+ ) -> std::io::Result<()> {
+ for entry in sorted_entries {
+ out.write_all(entry.id.as_slice())?;
+ }
+ Ok(())
+ }
+
+ /// Return true if the size of the `offset` range seems to match for a `hash` of the given kind and the amount of objects.
+ pub fn is_valid(offset: &Range<usize>, hash: gix_hash::Kind, num_objects: u32) -> bool {
+ (offset.end - offset.start) / hash.len_in_bytes() == num_objects as usize
+ }
+}
+
+/// Information about the offsets table.
+pub mod offsets {
+ use std::{convert::TryInto, ops::Range};
+
+ use crate::multi_index;
+
+ /// The id uniquely identifying the offsets table.
+ pub const ID: gix_chunk::Id = *b"OOFF";
+
+ /// Return the amount of bytes needed to offset data for `entries`.
+ pub fn storage_size(entries: usize) -> u64 {
+ (entries * (4 /*pack-id*/ + 4/* pack offset */)) as u64
+ }
+
+ pub(crate) fn write(
+ sorted_entries: &[multi_index::write::Entry],
+ large_offsets_needed: bool,
+ mut out: impl std::io::Write,
+ ) -> std::io::Result<()> {
+ use crate::index::write::encode::{HIGH_BIT, LARGE_OFFSET_THRESHOLD};
+ let mut num_large_offsets = 0u32;
+
+ for entry in sorted_entries {
+ out.write_all(&entry.pack_index.to_be_bytes())?;
+
+ let offset: u32 = if large_offsets_needed {
+ if entry.pack_offset > LARGE_OFFSET_THRESHOLD {
+ let res = num_large_offsets | HIGH_BIT;
+ num_large_offsets += 1;
+ res
+ } else {
+ entry.pack_offset as u32
+ }
+ } else {
+ entry
+ .pack_offset
+ .try_into()
+ .expect("without large offsets, pack-offset fits u32")
+ };
+ out.write_all(&offset.to_be_bytes())?;
+ }
+ Ok(())
+ }
+
+ /// Returns true if the `offset` range seems to match the size required for `num_objects`.
+ pub fn is_valid(offset: &Range<usize>, num_objects: u32) -> bool {
+ let entry_size = 4 /* pack-id */ + 4 /* pack-offset */;
+ ((offset.end - offset.start) / num_objects as usize) == entry_size
+ }
+}
+
+/// Information about the large offsets table.
+pub mod large_offsets {
+ use std::ops::Range;
+
+ use crate::{index::write::encode::LARGE_OFFSET_THRESHOLD, multi_index};
+
+ /// The id uniquely identifying the large offsets table (with 64 bit offsets)
+ pub const ID: gix_chunk::Id = *b"LOFF";
+
+ /// Returns Some(num-large-offset) if there are offsets larger than u32.
+ pub(crate) fn num_large_offsets(entries: &[multi_index::write::Entry]) -> Option<usize> {
+ let mut num_large_offsets = 0;
+ let mut needs_large_offsets = false;
+ for entry in entries {
+ if entry.pack_offset > LARGE_OFFSET_THRESHOLD {
+ num_large_offsets += 1;
+ }
+ if entry.pack_offset > u32::MAX as crate::data::Offset {
+ needs_large_offsets = true;
+ }
+ }
+
+ needs_large_offsets.then_some(num_large_offsets)
+ }
+ /// Returns true if the `offsets` range seems to be properly aligned for the data we expect.
+ pub fn is_valid(offset: &Range<usize>) -> bool {
+ (offset.end - offset.start) % 8 == 0
+ }
+
+ pub(crate) fn write(
+ sorted_entries: &[multi_index::write::Entry],
+ mut num_large_offsets: usize,
+ mut out: impl std::io::Write,
+ ) -> std::io::Result<()> {
+ for offset in sorted_entries
+ .iter()
+ .filter_map(|e| (e.pack_offset > LARGE_OFFSET_THRESHOLD).then_some(e.pack_offset))
+ {
+ out.write_all(&offset.to_be_bytes())?;
+ num_large_offsets = num_large_offsets
+ .checked_sub(1)
+ .expect("BUG: wrote more offsets the previously found");
+ }
+ assert_eq!(num_large_offsets, 0, "BUG: wrote less offsets than initially counted");
+ Ok(())
+ }
+
+ /// Return the amount of bytes needed to store the given amount of `large_offsets`
+ pub(crate) fn storage_size(large_offsets: usize) -> u64 {
+ 8 * large_offsets as u64
+ }
+}
diff --git a/vendor/gix-pack/src/multi_index/init.rs b/vendor/gix-pack/src/multi_index/init.rs
new file mode 100644
index 000000000..190b40a7b
--- /dev/null
+++ b/vendor/gix-pack/src/multi_index/init.rs
@@ -0,0 +1,157 @@
+use std::{convert::TryFrom, path::Path};
+
+use crate::multi_index::{chunk, File, Version};
+
+mod error {
+ use crate::multi_index::chunk;
+
+ /// The error returned by [File::at()][super::File::at()].
+ #[derive(Debug, thiserror::Error)]
+ #[allow(missing_docs)]
+ pub enum Error {
+ #[error("Could not open multi-index file at '{path}'")]
+ Io {
+ source: std::io::Error,
+ path: std::path::PathBuf,
+ },
+ #[error("{message}")]
+ Corrupt { message: &'static str },
+ #[error("Unsupported multi-index version: {version})")]
+ UnsupportedVersion { version: u8 },
+ #[error("Unsupported hash kind: {kind})")]
+ UnsupportedObjectHash { kind: u8 },
+ #[error(transparent)]
+ ChunkFileDecode(#[from] gix_chunk::file::decode::Error),
+ #[error(transparent)]
+ MissingChunk(#[from] gix_chunk::file::index::offset_by_kind::Error),
+ #[error(transparent)]
+ FileTooLarge(#[from] gix_chunk::file::index::data_by_kind::Error),
+ #[error("The multi-pack fan doesn't have the correct size of 256 * 4 bytes")]
+ MultiPackFanSize,
+ #[error(transparent)]
+ PackNames(#[from] chunk::index_names::decode::Error),
+ #[error("multi-index chunk {:?} has invalid size: {message}", String::from_utf8_lossy(.id))]
+ InvalidChunkSize { id: gix_chunk::Id, message: &'static str },
+ }
+}
+
+pub use error::Error;
+
+/// Initialization
+impl File {
+ /// Open the multi-index file at the given `path`.
+ pub fn at(path: impl AsRef<Path>) -> Result<Self, Error> {
+ Self::try_from(path.as_ref())
+ }
+}
+
+impl TryFrom<&Path> for File {
+ type Error = Error;
+
+ fn try_from(path: &Path) -> Result<Self, Self::Error> {
+ let data = crate::mmap::read_only(path).map_err(|source| Error::Io {
+ source,
+ path: path.to_owned(),
+ })?;
+
+ const TRAILER_LEN: usize = gix_hash::Kind::shortest().len_in_bytes(); /* trailing hash */
+ if data.len()
+ < Self::HEADER_LEN
+ + gix_chunk::file::Index::size_for_entries(4 /*index names, fan, offsets, oids*/)
+ + chunk::fanout::SIZE
+ + TRAILER_LEN
+ {
+ return Err(Error::Corrupt {
+ message: "multi-index file is truncated and too short",
+ });
+ }
+
+ let (version, object_hash, num_chunks, num_indices) = {
+ let (signature, data) = data.split_at(4);
+ if signature != Self::SIGNATURE {
+ return Err(Error::Corrupt {
+ message: "Invalid signature",
+ });
+ }
+ let (version, data) = data.split_at(1);
+ let version = match version[0] {
+ 1 => Version::V1,
+ version => return Err(Error::UnsupportedVersion { version }),
+ };
+
+ let (object_hash, data) = data.split_at(1);
+ let object_hash = gix_hash::Kind::try_from(object_hash[0])
+ .map_err(|unknown| Error::UnsupportedObjectHash { kind: unknown })?;
+ let (num_chunks, data) = data.split_at(1);
+ let num_chunks = num_chunks[0];
+
+ let (_num_base_files, data) = data.split_at(1); // TODO: handle base files once it's clear what this does
+
+ let (num_indices, _) = data.split_at(4);
+ let num_indices = crate::read_u32(num_indices);
+
+ (version, object_hash, num_chunks, num_indices)
+ };
+
+ let chunks = gix_chunk::file::Index::from_bytes(&data, Self::HEADER_LEN, num_chunks as u32)?;
+
+ let index_names = chunks.data_by_id(&data, chunk::index_names::ID)?;
+ let index_names = chunk::index_names::from_bytes(index_names, num_indices)?;
+
+ let fan = chunks.data_by_id(&data, chunk::fanout::ID)?;
+ let fan = chunk::fanout::from_bytes(fan).ok_or(Error::MultiPackFanSize)?;
+ let num_objects = fan[255];
+
+ let lookup = chunks.validated_usize_offset_by_id(chunk::lookup::ID, |offset| {
+ chunk::lookup::is_valid(&offset, object_hash, num_objects)
+ .then_some(offset)
+ .ok_or(Error::InvalidChunkSize {
+ id: chunk::lookup::ID,
+ message: "The chunk with alphabetically ordered object ids doesn't have the correct size",
+ })
+ })??;
+ let offsets = chunks.validated_usize_offset_by_id(chunk::offsets::ID, |offset| {
+ chunk::offsets::is_valid(&offset, num_objects)
+ .then_some(offset)
+ .ok_or(Error::InvalidChunkSize {
+ id: chunk::offsets::ID,
+ message: "The chunk with offsets into the pack doesn't have the correct size",
+ })
+ })??;
+ let large_offsets = chunks
+ .validated_usize_offset_by_id(chunk::large_offsets::ID, |offset| {
+ chunk::large_offsets::is_valid(&offset)
+ .then_some(offset)
+ .ok_or(Error::InvalidChunkSize {
+ id: chunk::large_offsets::ID,
+ message: "The chunk with large offsets into the pack doesn't have the correct size",
+ })
+ })
+ .ok()
+ .transpose()?;
+
+ let checksum_offset = chunks.highest_offset() as usize;
+ let trailer = &data[checksum_offset..];
+ if trailer.len() != object_hash.len_in_bytes() {
+ return Err(Error::Corrupt {
+ message:
+ "Trailing checksum didn't have the expected size or there were unknown bytes after the checksum.",
+ });
+ }
+
+ Ok(File {
+ data,
+ path: path.to_owned(),
+ version,
+ hash_len: object_hash.len_in_bytes(),
+ object_hash,
+ fan,
+ index_names,
+ lookup_ofs: lookup.start,
+ offsets_ofs: offsets.start,
+ large_offsets_ofs: large_offsets.map(|r| r.start),
+ num_objects,
+ num_indices,
+ })
+ }
+}
diff --git a/vendor/gix-pack/src/multi_index/mod.rs b/vendor/gix-pack/src/multi_index/mod.rs
new file mode 100644
index 000000000..3f7ed0ff5
--- /dev/null
+++ b/vendor/gix-pack/src/multi_index/mod.rs
@@ -0,0 +1,57 @@
+use std::path::PathBuf;
+
+use memmap2::Mmap;
+
+/// Known multi-index file versions
+#[derive(PartialEq, Eq, Ord, PartialOrd, Debug, Hash, Clone, Copy)]
+#[cfg_attr(feature = "serde1", derive(serde::Serialize, serde::Deserialize))]
+#[allow(missing_docs)]
+pub enum Version {
+ V1 = 1,
+}
+
+impl Default for Version {
+ fn default() -> Self {
+ Version::V1
+ }
+}
+
+/// An index into our [`File::index_names()`] array yielding the name of the index and by implication, its pack file.
+pub type PackIndex = u32;
+
+/// The type for referring to indices of an entry within the index file.
+pub type EntryIndex = u32;
+
+/// A representation of an index file for multiple packs at the same time, typically stored in a file
+/// named 'multi-pack-index'.
+pub struct File {
+ data: Mmap,
+ path: std::path::PathBuf,
+ version: Version,
+ hash_len: usize,
+ object_hash: gix_hash::Kind,
+ /// The amount of pack files contained within
+ num_indices: u32,
+ num_objects: u32,
+
+ fan: [u32; 256],
+ index_names: Vec<PathBuf>,
+ lookup_ofs: usize,
+ offsets_ofs: usize,
+ large_offsets_ofs: Option<usize>,
+}
+
+///
+pub mod write;
+
+///
+mod access;
+
+///
+pub mod verify;
+
+///
+pub mod chunk;
+
+///
+pub mod init;
diff --git a/vendor/gix-pack/src/multi_index/verify.rs b/vendor/gix-pack/src/multi_index/verify.rs
new file mode 100644
index 000000000..856a48501
--- /dev/null
+++ b/vendor/gix-pack/src/multi_index/verify.rs
@@ -0,0 +1,337 @@
+use std::{cmp::Ordering, sync::atomic::AtomicBool, time::Instant};
+
+use gix_features::progress::Progress;
+
+use crate::{index, multi_index::File};
+
+///
+pub mod integrity {
+ use crate::multi_index::EntryIndex;
+
+ /// Returned by [`multi_index::File::verify_integrity()`][crate::multi_index::File::verify_integrity()].
+ #[derive(thiserror::Error, Debug)]
+ #[allow(missing_docs)]
+ pub enum Error {
+ #[error("Object {id} should be at pack-offset {expected_pack_offset} but was found at {actual_pack_offset}")]
+ PackOffsetMismatch {
+ id: gix_hash::ObjectId,
+ expected_pack_offset: u64,
+ actual_pack_offset: u64,
+ },
+ #[error(transparent)]
+ MultiIndexChecksum(#[from] crate::multi_index::verify::checksum::Error),
+ #[error(transparent)]
+ IndexIntegrity(#[from] crate::index::verify::integrity::Error),
+ #[error(transparent)]
+ BundleInit(#[from] crate::bundle::init::Error),
+ #[error("Counted {actual} objects, but expected {expected} as per multi-index")]
+ UnexpectedObjectCount { actual: usize, expected: usize },
+ #[error("{id} wasn't found in the index referenced in the multi-pack index")]
+ OidNotFound { id: gix_hash::ObjectId },
+ #[error("The object id at multi-index entry {index} wasn't in order")]
+ OutOfOrder { index: EntryIndex },
+ #[error("The fan at index {index} is out of order as it's larger then the following value.")]
+ Fan { index: usize },
+ #[error("The multi-index claims to have no objects")]
+ Empty,
+ #[error("Interrupted")]
+ Interrupted,
+ }
+
+ /// Returned by [`multi_index::File::verify_integrity()`][crate::multi_index::File::verify_integrity()].
+ pub struct Outcome<P> {
+ /// The computed checksum of the multi-index which matched the stored one.
+ pub actual_index_checksum: gix_hash::ObjectId,
+ /// The for each entry in [`index_names()`][super::File::index_names()] provide the corresponding pack traversal outcome.
+ pub pack_traverse_statistics: Vec<crate::index::traverse::Statistics>,
+ /// The provided progress instance.
+ pub progress: P,
+ }
+
+ /// The progress ids used in [`multi_index::File::verify_integrity()`][crate::multi_index::File::verify_integrity()].
+ ///
+ /// Use this information to selectively extract the progress of interest in case the parent application has custom visualization.
+ #[derive(Debug, Copy, Clone)]
+ pub enum ProgressId {
+ /// The amount of bytes read to verify the multi-index checksum.
+ ChecksumBytes,
+ /// The amount of objects whose offset has been checked.
+ ObjectOffsets,
+ }
+
+ impl From<ProgressId> for gix_features::progress::Id {
+ fn from(v: ProgressId) -> Self {
+ match v {
+ ProgressId::ChecksumBytes => *b"MVCK",
+ ProgressId::ObjectOffsets => *b"MVOF",
+ }
+ }
+ }
+}
+
+///
+pub mod checksum {
+ /// Returned by [`multi_index::File::verify_checksum()`][crate::multi_index::File::verify_checksum()].
+ pub type Error = crate::verify::checksum::Error;
+}
+
+impl File {
+ /// Validate that our [`checksum()`][File::checksum()] matches the actual contents
+ /// of this index file, and return it if it does.
+ pub fn verify_checksum(
+ &self,
+ progress: impl Progress,
+ should_interrupt: &AtomicBool,
+ ) -> Result<gix_hash::ObjectId, checksum::Error> {
+ crate::verify::checksum_on_disk_or_mmap(
+ self.path(),
+ &self.data,
+ self.checksum(),
+ self.object_hash,
+ progress,
+ should_interrupt,
+ )
+ }
+
+ /// Similar to [`verify_integrity()`][File::verify_integrity()] but without any deep inspection of objects.
+ ///
+ /// Instead we only validate the contents of the multi-index itself.
+ pub fn verify_integrity_fast<P>(
+ &self,
+ progress: P,
+ should_interrupt: &AtomicBool,
+ ) -> Result<(gix_hash::ObjectId, P), integrity::Error>
+ where
+ P: Progress,
+ {
+ self.verify_integrity_inner(
+ progress,
+ should_interrupt,
+ false,
+ index::verify::integrity::Options::default(),
+ )
+ .map_err(|err| match err {
+ index::traverse::Error::Processor(err) => err,
+ _ => unreachable!("BUG: no other error type is possible"),
+ })
+ .map(|o| (o.actual_index_checksum, o.progress))
+ }
+
+ /// Similar to [`crate::Bundle::verify_integrity()`] but checks all contained indices and their packs.
+ ///
+ /// Note that it's considered a failure if an index doesn't have a corresponding pack.
+ pub fn verify_integrity<C, P, F>(
+ &self,
+ progress: P,
+ should_interrupt: &AtomicBool,
+ options: index::verify::integrity::Options<F>,
+ ) -> Result<integrity::Outcome<P>, index::traverse::Error<integrity::Error>>
+ where
+ P: Progress,
+ C: crate::cache::DecodeEntry,
+ F: Fn() -> C + Send + Clone,
+ {
+ self.verify_integrity_inner(progress, should_interrupt, true, options)
+ }
+
+ fn verify_integrity_inner<C, P, F>(
+ &self,
+ mut progress: P,
+ should_interrupt: &AtomicBool,
+ deep_check: bool,
+ options: index::verify::integrity::Options<F>,
+ ) -> Result<integrity::Outcome<P>, index::traverse::Error<integrity::Error>>
+ where
+ P: Progress,
+ C: crate::cache::DecodeEntry,
+ F: Fn() -> C + Send + Clone,
+ {
+ let parent = self.path.parent().expect("must be in a directory");
+
+ let actual_index_checksum = self
+ .verify_checksum(
+ progress.add_child_with_id(
+ format!("{}: checksum", self.path.display()),
+ integrity::ProgressId::ChecksumBytes.into(),
+ ),
+ should_interrupt,
+ )
+ .map_err(integrity::Error::from)
+ .map_err(index::traverse::Error::Processor)?;
+
+ if let Some(first_invalid) = crate::verify::fan(&self.fan) {
+ return Err(index::traverse::Error::Processor(integrity::Error::Fan {
+ index: first_invalid,
+ }));
+ }
+
+ if self.num_objects == 0 {
+ return Err(index::traverse::Error::Processor(integrity::Error::Empty));
+ }
+
+ let mut pack_traverse_statistics = Vec::new();
+
+ let operation_start = Instant::now();
+ let mut total_objects_checked = 0;
+ let mut pack_ids_and_offsets = Vec::with_capacity(self.num_objects as usize);
+ {
+ let order_start = Instant::now();
+ let mut progress = progress.add_child_with_id("checking oid order", gix_features::progress::UNKNOWN);
+ progress.init(
+ Some(self.num_objects as usize),
+ gix_features::progress::count("objects"),
+ );
+
+ for entry_index in 0..(self.num_objects - 1) {
+ let lhs = self.oid_at_index(entry_index);
+ let rhs = self.oid_at_index(entry_index + 1);
+
+ if rhs.cmp(lhs) != Ordering::Greater {
+ return Err(index::traverse::Error::Processor(integrity::Error::OutOfOrder {
+ index: entry_index,
+ }));
+ }
+ let (pack_id, _) = self.pack_id_and_pack_offset_at_index(entry_index);
+ pack_ids_and_offsets.push((pack_id, entry_index));
+ progress.inc();
+ }
+ {
+ let entry_index = self.num_objects - 1;
+ let (pack_id, _) = self.pack_id_and_pack_offset_at_index(entry_index);
+ pack_ids_and_offsets.push((pack_id, entry_index));
+ }
+ // sort by pack-id to allow handling all indices matching a pack while its open.
+ pack_ids_and_offsets.sort_by(|l, r| l.0.cmp(&r.0));
+ progress.show_throughput(order_start);
+ };
+
+ progress.init(
+ Some(self.num_indices as usize),
+ gix_features::progress::count("indices"),
+ );
+
+ let mut pack_ids_slice = pack_ids_and_offsets.as_slice();
+
+ for (pack_id, index_file_name) in self.index_names.iter().enumerate() {
+ progress.set_name(index_file_name.display().to_string());
+ progress.inc();
+
+ let mut bundle = None;
+ let index;
+ let index_path = parent.join(index_file_name);
+ let index = if deep_check {
+ bundle = crate::Bundle::at(index_path, self.object_hash)
+ .map_err(integrity::Error::from)
+ .map_err(index::traverse::Error::Processor)?
+ .into();
+ bundle.as_ref().map(|b| &b.index).expect("just set")
+ } else {
+ index = Some(
+ index::File::at(index_path, self.object_hash)
+ .map_err(|err| integrity::Error::BundleInit(crate::bundle::init::Error::Index(err)))
+ .map_err(index::traverse::Error::Processor)?,
+ );
+ index.as_ref().expect("just set")
+ };
+
+ let slice_end = pack_ids_slice.partition_point(|e| e.0 == pack_id as crate::data::Id);
+ let multi_index_entries_to_check = &pack_ids_slice[..slice_end];
+ {
+ let offset_start = Instant::now();
+ let mut offsets_progress =
+ progress.add_child_with_id("verify object offsets", integrity::ProgressId::ObjectOffsets.into());
+ offsets_progress.init(
+ Some(pack_ids_and_offsets.len()),
+ gix_features::progress::count("objects"),
+ );
+ pack_ids_slice = &pack_ids_slice[slice_end..];
+
+ for entry_id in multi_index_entries_to_check.iter().map(|e| e.1) {
+ let oid = self.oid_at_index(entry_id);
+ let (_, expected_pack_offset) = self.pack_id_and_pack_offset_at_index(entry_id);
+ let entry_in_bundle_index = index.lookup(oid).ok_or_else(|| {
+ index::traverse::Error::Processor(integrity::Error::OidNotFound { id: oid.to_owned() })
+ })?;
+ let actual_pack_offset = index.pack_offset_at_index(entry_in_bundle_index);
+ if actual_pack_offset != expected_pack_offset {
+ return Err(index::traverse::Error::Processor(
+ integrity::Error::PackOffsetMismatch {
+ id: oid.to_owned(),
+ expected_pack_offset,
+ actual_pack_offset,
+ },
+ ));
+ }
+ offsets_progress.inc();
+ }
+
+ if should_interrupt.load(std::sync::atomic::Ordering::Relaxed) {
+ return Err(index::traverse::Error::Processor(integrity::Error::Interrupted));
+ }
+ offsets_progress.show_throughput(offset_start);
+ }
+
+ total_objects_checked += multi_index_entries_to_check.len();
+
+ if let Some(bundle) = bundle {
+ progress.set_name(format!("Validating {}", index_file_name.display()));
+ let crate::bundle::verify::integrity::Outcome {
+ actual_index_checksum: _,
+ pack_traverse_outcome,
+ progress: returned_progress,
+ } = bundle
+ .verify_integrity(progress, should_interrupt, options.clone())
+ .map_err(|err| {
+ use index::traverse::Error::*;
+ match err {
+ Processor(err) => Processor(integrity::Error::IndexIntegrity(err)),
+ VerifyChecksum(err) => VerifyChecksum(err),
+ Tree(err) => Tree(err),
+ TreeTraversal(err) => TreeTraversal(err),
+ PackDecode { id, offset, source } => PackDecode { id, offset, source },
+ PackMismatch { expected, actual } => PackMismatch { expected, actual },
+ PackObjectMismatch {
+ expected,
+ actual,
+ offset,
+ kind,
+ } => PackObjectMismatch {
+ expected,
+ actual,
+ offset,
+ kind,
+ },
+ Crc32Mismatch {
+ expected,
+ actual,
+ offset,
+ kind,
+ } => Crc32Mismatch {
+ expected,
+ actual,
+ offset,
+ kind,
+ },
+ Interrupted => Interrupted,
+ }
+ })?;
+ progress = returned_progress;
+ pack_traverse_statistics.push(pack_traverse_outcome);
+ }
+ }
+
+ assert_eq!(
+ self.num_objects as usize, total_objects_checked,
+ "BUG: our slicing should allow to visit all objects"
+ );
+
+ progress.set_name("Validating multi-pack");
+ progress.show_throughput(operation_start);
+
+ Ok(integrity::Outcome {
+ actual_index_checksum,
+ pack_traverse_statistics,
+ progress,
+ })
+ }
+}
diff --git a/vendor/gix-pack/src/multi_index/write.rs b/vendor/gix-pack/src/multi_index/write.rs
new file mode 100644
index 000000000..314506401
--- /dev/null
+++ b/vendor/gix-pack/src/multi_index/write.rs
@@ -0,0 +1,244 @@
+use std::{
+ convert::TryInto,
+ path::PathBuf,
+ sync::atomic::{AtomicBool, Ordering},
+ time::{Instant, SystemTime},
+};
+
+use gix_features::progress::Progress;
+
+use crate::multi_index;
+
+mod error {
+ /// The error returned by [multi_index::File::write_from_index_paths()][super::multi_index::File::write_from_index_paths()]..
+ #[derive(Debug, thiserror::Error)]
+ #[allow(missing_docs)]
+ pub enum Error {
+ #[error(transparent)]
+ Io(#[from] std::io::Error),
+ #[error("Interrupted")]
+ Interrupted,
+ #[error(transparent)]
+ OpenIndex(#[from] crate::index::init::Error),
+ }
+}
+pub use error::Error;
+
+/// An entry suitable for sorting and writing
+pub(crate) struct Entry {
+ pub(crate) id: gix_hash::ObjectId,
+ pub(crate) pack_index: u32,
+ pub(crate) pack_offset: crate::data::Offset,
+ /// Used for sorting in case of duplicates
+ index_mtime: SystemTime,
+}
+
+/// Options for use in [`multi_index::File::write_from_index_paths()`].
+pub struct Options {
+ /// The kind of hash to use for objects and to expect in the input files.
+ pub object_hash: gix_hash::Kind,
+}
+
+/// The result of [`multi_index::File::write_from_index_paths()`].
+pub struct Outcome<P> {
+ /// The calculated multi-index checksum of the file at `multi_index_path`.
+ pub multi_index_checksum: gix_hash::ObjectId,
+ /// The input progress
+ pub progress: P,
+}
+
+/// The progress ids used in [`write_from_index_paths()`][multi_index::File::write_from_index_paths()].
+///
+/// Use this information to selectively extract the progress of interest in case the parent application has custom visualization.
+#[derive(Debug, Copy, Clone)]
+pub enum ProgressId {
+ /// Counts each path in the input set whose entries we enumerate and write into the multi-index
+ FromPathsCollectingEntries,
+ /// The amount of bytes written as part of the multi-index.
+ BytesWritten,
+}
+
+impl From<ProgressId> for gix_features::progress::Id {
+ fn from(v: ProgressId) -> Self {
+ match v {
+ ProgressId::FromPathsCollectingEntries => *b"MPCE",
+ ProgressId::BytesWritten => *b"MPBW",
+ }
+ }
+}
+
+impl multi_index::File {
+ pub(crate) const SIGNATURE: &'static [u8] = b"MIDX";
+ pub(crate) const HEADER_LEN: usize = 4 /*signature*/ +
+ 1 /*version*/ +
+ 1 /*object id version*/ +
+ 1 /*num chunks */ +
+ 1 /*num base files */ +
+ 4 /*num pack files*/;
+
+ /// Create a new multi-index file for writing to `out` from the pack index files at `index_paths`.
+ ///
+ /// Progress is sent to `progress` and interruptions checked via `should_interrupt`.
+ pub fn write_from_index_paths<P>(
+ mut index_paths: Vec<PathBuf>,
+ out: impl std::io::Write,
+ mut progress: P,
+ should_interrupt: &AtomicBool,
+ Options { object_hash }: Options,
+ ) -> Result<Outcome<P>, Error>
+ where
+ P: Progress,
+ {
+ let out = gix_features::hash::Write::new(out, object_hash);
+ let (index_paths_sorted, index_filenames_sorted) = {
+ index_paths.sort();
+ let file_names = index_paths
+ .iter()
+ .map(|p| PathBuf::from(p.file_name().expect("file name present")))
+ .collect::<Vec<_>>();
+ (index_paths, file_names)
+ };
+
+ let entries = {
+ let mut entries = Vec::new();
+ let start = Instant::now();
+ let mut progress =
+ progress.add_child_with_id("Collecting entries", ProgressId::FromPathsCollectingEntries.into());
+ progress.init(Some(index_paths_sorted.len()), gix_features::progress::count("indices"));
+
+ // This could be parallelized… but it's probably not worth it unless you have 500mio objects.
+ for (index_id, index) in index_paths_sorted.iter().enumerate() {
+ let mtime = index
+ .metadata()
+ .and_then(|m| m.modified())
+ .unwrap_or(SystemTime::UNIX_EPOCH);
+ let index = crate::index::File::at(index, object_hash)?;
+
+ entries.reserve(index.num_objects() as usize);
+ entries.extend(index.iter().map(|e| Entry {
+ id: e.oid,
+ pack_index: index_id as u32,
+ pack_offset: e.pack_offset,
+ index_mtime: mtime,
+ }));
+ progress.inc();
+ if should_interrupt.load(Ordering::Relaxed) {
+ return Err(Error::Interrupted);
+ }
+ }
+ progress.show_throughput(start);
+
+ let start = Instant::now();
+ progress.set_name("Deduplicate");
+ progress.init(Some(entries.len()), gix_features::progress::count("entries"));
+ entries.sort_by(|l, r| {
+ l.id.cmp(&r.id)
+ .then_with(|| l.index_mtime.cmp(&r.index_mtime).reverse())
+ .then_with(|| l.pack_index.cmp(&r.pack_index))
+ });
+ entries.dedup_by_key(|e| e.id);
+ progress.inc_by(entries.len());
+ progress.show_throughput(start);
+ if should_interrupt.load(Ordering::Relaxed) {
+ return Err(Error::Interrupted);
+ }
+ entries
+ };
+
+ let mut cf = gix_chunk::file::Index::for_writing();
+ cf.plan_chunk(
+ multi_index::chunk::index_names::ID,
+ multi_index::chunk::index_names::storage_size(&index_filenames_sorted),
+ );
+ cf.plan_chunk(multi_index::chunk::fanout::ID, multi_index::chunk::fanout::SIZE as u64);
+ cf.plan_chunk(
+ multi_index::chunk::lookup::ID,
+ multi_index::chunk::lookup::storage_size(entries.len(), object_hash),
+ );
+ cf.plan_chunk(
+ multi_index::chunk::offsets::ID,
+ multi_index::chunk::offsets::storage_size(entries.len()),
+ );
+
+ let num_large_offsets = multi_index::chunk::large_offsets::num_large_offsets(&entries);
+ if let Some(num_large_offsets) = num_large_offsets {
+ cf.plan_chunk(
+ multi_index::chunk::large_offsets::ID,
+ multi_index::chunk::large_offsets::storage_size(num_large_offsets),
+ );
+ }
+
+ let mut write_progress = progress.add_child_with_id("Writing multi-index", ProgressId::BytesWritten.into());
+ let write_start = Instant::now();
+ write_progress.init(
+ Some(cf.planned_storage_size() as usize + Self::HEADER_LEN),
+ gix_features::progress::bytes(),
+ );
+ let mut out = gix_features::progress::Write {
+ inner: out,
+ progress: write_progress,
+ };
+
+ let bytes_written = Self::write_header(
+ &mut out,
+ cf.num_chunks().try_into().expect("BUG: wrote more than 256 chunks"),
+ index_paths_sorted.len() as u32,
+ object_hash,
+ )?;
+
+ {
+ progress.set_name("Writing chunks");
+ progress.init(Some(cf.num_chunks()), gix_features::progress::count("chunks"));
+
+ let mut chunk_write = cf.into_write(&mut out, bytes_written)?;
+ while let Some(chunk_to_write) = chunk_write.next_chunk() {
+ match chunk_to_write {
+ multi_index::chunk::index_names::ID => {
+ multi_index::chunk::index_names::write(&index_filenames_sorted, &mut chunk_write)?
+ }
+ multi_index::chunk::fanout::ID => multi_index::chunk::fanout::write(&entries, &mut chunk_write)?,
+ multi_index::chunk::lookup::ID => multi_index::chunk::lookup::write(&entries, &mut chunk_write)?,
+ multi_index::chunk::offsets::ID => {
+ multi_index::chunk::offsets::write(&entries, num_large_offsets.is_some(), &mut chunk_write)?
+ }
+ multi_index::chunk::large_offsets::ID => multi_index::chunk::large_offsets::write(
+ &entries,
+ num_large_offsets.expect("available if planned"),
+ &mut chunk_write,
+ )?,
+ unknown => unreachable!("BUG: forgot to implement chunk {:?}", std::str::from_utf8(&unknown)),
+ }
+ progress.inc();
+ if should_interrupt.load(Ordering::Relaxed) {
+ return Err(Error::Interrupted);
+ }
+ }
+ }
+
+ // write trailing checksum
+ let multi_index_checksum: gix_hash::ObjectId = out.inner.hash.digest().into();
+ out.inner.inner.write_all(multi_index_checksum.as_slice())?;
+ out.progress.show_throughput(write_start);
+
+ Ok(Outcome {
+ multi_index_checksum,
+ progress,
+ })
+ }
+
+ fn write_header(
+ mut out: impl std::io::Write,
+ num_chunks: u8,
+ num_indices: u32,
+ object_hash: gix_hash::Kind,
+ ) -> std::io::Result<usize> {
+ out.write_all(Self::SIGNATURE)?;
+ out.write_all(&[crate::multi_index::Version::V1 as u8])?;
+ out.write_all(&[object_hash as u8])?;
+ out.write_all(&[num_chunks])?;
+ out.write_all(&[0])?; /* unused number of base files */
+ out.write_all(&num_indices.to_be_bytes())?;
+
+ Ok(Self::HEADER_LEN)
+ }
+}