Kaynağa Gözat

Fix handling of fetching of packs over dumb http transports (#1608)

Fixes #1097
Jelmer Vernooij 1 ay önce
ebeveyn
işleme
bf8eb0f412
7 değiştirilmiş dosya ile 1168 ekleme ve 3 silme
  1. 3 0
      NEWS
  2. 44 2
      dulwich/client.py
  3. 467 0
      dulwich/dumb.py
  4. 239 0
      tests/compat/test_dumb.py
  5. 4 1
      tests/compat/test_pack.py
  6. 121 0
      tests/test_client.py
  7. 290 0
      tests/test_dumb.py

+ 3 - 0
NEWS

@@ -1,5 +1,8 @@
 0.23.1	UNRELEASED
 
+ * Support dumb repository access.
+   (Jelmer Vernooij, #1097)
+
  * Support ``depth`` for local clones.
    (Jelmer Vernooij)
 

+ 44 - 2
dulwich/client.py

@@ -2547,7 +2547,14 @@ class AbstractHttpGitClient(GitClient):
                     return refs, server_capabilities, base_url, symrefs, peeled
             else:
                 self.protocol_version = 0  # dumb servers only support protocol v0
-                (refs, peeled) = split_peeled_refs(read_info_refs(resp))
+                # Read all the response data
+                data = b""
+                while True:
+                    chunk = read(4096)
+                    if not chunk:
+                        break
+                    data += chunk
+                (refs, peeled) = split_peeled_refs(read_info_refs(BytesIO(data)))
                 if ref_prefix is not None:
                     refs = filter_ref_prefix(refs, ref_prefix)
                 return refs, set(), base_url, {}, peeled
@@ -2700,7 +2707,42 @@ class AbstractHttpGitClient(GitClient):
         if not wants:
             return FetchPackResult(refs, symrefs, agent)
         if self.dumb:
-            raise NotImplementedError(self.fetch_pack)
+            # Use dumb HTTP protocol
+            from .dumb import DumbRemoteHTTPRepo
+
+            # Pass http_request function
+            dumb_repo = DumbRemoteHTTPRepo(url, self._http_request)
+
+            # Fetch pack data from dumb remote
+            pack_data_list = list(
+                dumb_repo.fetch_pack_data(
+                    graph_walker, lambda refs: wants, progress=progress, depth=depth
+                )
+            )
+
+            # Write pack data
+            if pack_data:
+                from .pack import pack_objects_to_data, write_pack_data
+
+                # Convert unpacked objects to ShaFile objects for packing
+                objects = []
+                for unpacked in pack_data_list:
+                    objects.append(unpacked.sha_file())
+
+                # Generate pack data and write it to a buffer
+                pack_buffer = BytesIO()
+                count, unpacked_iter = pack_objects_to_data(objects)
+                write_pack_data(
+                    pack_buffer.write,
+                    unpacked_iter,
+                    num_records=count,
+                    progress=progress,
+                )
+
+                # Pass the raw pack data to pack_data callback
+                pack_data(pack_buffer.getvalue())
+
+            return FetchPackResult(refs, symrefs, agent)
         req_data = BytesIO()
         req_proto = Protocol(None, req_data.write)
         (new_shallow, new_unshallow) = _handle_upload_pack_head(

+ 467 - 0
dulwich/dumb.py

@@ -0,0 +1,467 @@
+# dumb.py -- Support for dumb HTTP(S) git repositories
+# Copyright (C) 2025 Dulwich contributors
+#
+# SPDX-License-Identifier: Apache-2.0 OR GPL-2.0-or-later
+# Dulwich is dual-licensed under the Apache License, Version 2.0 and the GNU
+# General Public License as public by the Free Software Foundation; version 2.0
+# or (at your option) any later version. You can redistribute it and/or
+# modify it under the terms of either of these two licenses.
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+# You should have received a copy of the licenses; if not, see
+# <http://www.gnu.org/licenses/> for a copy of the GNU General Public License
+# and <http://www.apache.org/licenses/LICENSE-2.0> for a copy of the Apache
+# License, Version 2.0.
+#
+
+"""Support for dumb HTTP(S) git repositories."""
+
+import os
+import tempfile
+import zlib
+from collections.abc import Iterator
+from io import BytesIO
+from typing import Optional
+from urllib.parse import urljoin
+
+from .errors import NotGitRepository, ObjectFormatException
+from .object_store import BaseObjectStore
+from .objects import (
+    ZERO_SHA,
+    Blob,
+    Commit,
+    ObjectID,
+    ShaFile,
+    Tag,
+    Tree,
+    hex_to_sha,
+    sha_to_hex,
+)
+from .pack import Pack, PackIndex, UnpackedObject, load_pack_index_file
+from .refs import Ref, read_info_refs, split_peeled_refs
+from .repo import BaseRepo
+
+
+class DumbHTTPObjectStore(BaseObjectStore):
+    """Object store implementation that fetches objects over dumb HTTP."""
+
+    def __init__(self, base_url: str, http_request_func):
+        """Initialize a DumbHTTPObjectStore.
+
+        Args:
+          base_url: Base URL of the remote repository (e.g. "https://example.com/repo.git/")
+          http_request_func: Function to make HTTP requests, should accept (url, headers)
+                           and return (response, read_func).
+        """
+        self.base_url = base_url.rstrip("/") + "/"
+        self._http_request = http_request_func
+        self._packs: Optional[list[tuple[str, Optional[PackIndex]]]] = None
+        self._cached_objects: dict[bytes, tuple[int, bytes]] = {}
+        self._temp_pack_dir = None
+
+    def _ensure_temp_pack_dir(self):
+        """Ensure we have a temporary directory for storing pack files."""
+        if self._temp_pack_dir is None:
+            self._temp_pack_dir = tempfile.mkdtemp(prefix="dulwich-dumb-")
+
+    def _fetch_url(self, path: str) -> bytes:
+        """Fetch content from a URL path relative to base_url.
+
+        Args:
+          path: Path relative to base URL
+        Returns:
+          Content as bytes
+        Raises:
+          IOError: If the URL cannot be fetched
+        """
+        url = urljoin(self.base_url, path)
+        resp, read = self._http_request(url, {})
+        try:
+            if resp.status == 404:
+                raise OSError(f"Not found: {url}")
+            elif resp.status != 200:
+                raise OSError(f"HTTP error {resp.status}: {url}")
+
+            # Read all content
+            chunks = []
+            while True:
+                chunk = read(4096)
+                if not chunk:
+                    break
+                chunks.append(chunk)
+            return b"".join(chunks)
+        finally:
+            resp.close()
+
+    def _fetch_loose_object(self, sha: bytes) -> tuple[int, bytes]:
+        """Fetch a loose object by SHA.
+
+        Args:
+          sha: SHA1 of the object (hex string as bytes)
+
+        Returns:
+          Tuple of (type_num, content)
+
+        Raises:
+          KeyError: If object not found
+        """
+        hex_sha = sha.decode("ascii")
+        path = f"objects/{hex_sha[:2]}/{hex_sha[2:]}"
+
+        try:
+            compressed = self._fetch_url(path)
+        except OSError:
+            raise KeyError(sha)
+
+        # Decompress and parse the object
+        decompressed = zlib.decompress(compressed)
+
+        # Parse header
+        header_end = decompressed.find(b"\x00")
+        if header_end == -1:
+            raise ObjectFormatException("Invalid object header")
+
+        header = decompressed[:header_end]
+        content = decompressed[header_end + 1 :]
+
+        parts = header.split(b" ", 1)
+        if len(parts) != 2:
+            raise ObjectFormatException("Invalid object header")
+
+        obj_type = parts[0]
+        obj_size = int(parts[1])
+
+        if len(content) != obj_size:
+            raise ObjectFormatException("Object size mismatch")
+
+        # Convert type name to type number
+        type_map = {
+            b"blob": Blob.type_num,
+            b"tree": Tree.type_num,
+            b"commit": Commit.type_num,
+            b"tag": Tag.type_num,
+        }
+
+        if obj_type not in type_map:
+            raise ObjectFormatException(f"Unknown object type: {obj_type!r}")
+
+        return type_map[obj_type], content
+
+    def _load_packs(self):
+        """Load the list of available packs from the remote."""
+        if self._packs is not None:
+            return
+
+        self._packs = []
+        try:
+            packs_data = self._fetch_url("objects/info/packs")
+        except OSError:
+            # No packs file, repository might only have loose objects
+            return
+
+        for line in packs_data.strip().split(b"\n"):
+            if line.startswith(b"P "):
+                pack_name = line[2:].decode("utf-8")
+                # Extract just the pack name without path
+                if "/" in pack_name:
+                    pack_name = pack_name.split("/")[-1]
+                if pack_name.endswith(".pack"):
+                    pack_name = pack_name[:-5]  # Remove .pack extension
+                self._packs.append((pack_name, None))
+
+    def _get_pack_index(self, pack_name: str) -> PackIndex:
+        """Get or fetch a pack index.
+
+        Args:
+          pack_name: Name of the pack (without .idx extension)
+
+        Returns:
+          PackIndex object
+        """
+        # Find the pack in our list
+        for i, (name, idx) in enumerate(self._packs or []):
+            if name == pack_name:
+                if idx is None:
+                    # Fetch and cache the index
+                    idx_data = self._fetch_url(f"objects/pack/{pack_name}.idx")
+
+                    idx = load_pack_index_file("<http>", BytesIO(idx_data))
+                    if self._packs is not None:
+                        self._packs[i] = (name, idx)
+                return idx
+        raise KeyError(f"Pack not found: {pack_name}")
+
+    def _fetch_from_pack(self, sha: bytes) -> tuple[int, bytes]:
+        """Try to fetch an object from pack files.
+
+        Args:
+          sha: SHA1 of the object (hex string as bytes)
+
+        Returns:
+          Tuple of (type_num, content)
+
+        Raises:
+          KeyError: If object not found in any pack
+        """
+        self._load_packs()
+        # Convert hex to binary for pack operations
+        binsha = hex_to_sha(sha)
+
+        for pack_name, idx in self._packs or []:
+            if idx is None:
+                idx = self._get_pack_index(pack_name)
+
+            try:
+                # Check if object is in this pack
+                idx.object_offset(binsha)
+            except KeyError:
+                continue
+
+            # We found the object, now we need to fetch the pack data
+            # For efficiency, we could fetch just the needed portion, but for
+            # simplicity we'll fetch the whole pack and cache it
+            self._ensure_temp_pack_dir()
+            if self._temp_pack_dir is None:
+                raise RuntimeError("Temp pack directory not initialized")
+            pack_path = os.path.join(self._temp_pack_dir, f"{pack_name}.pack")
+
+            if not os.path.exists(pack_path):
+                # Download the pack file
+                pack_data = self._fetch_url(f"objects/pack/{pack_name}.pack")
+                with open(pack_path, "wb") as f:
+                    f.write(pack_data)
+
+            # Open the pack and get the object
+            pack = Pack(pack_path[:-5])  # Remove .pack extension
+            try:
+                return pack.get_raw(binsha)
+            finally:
+                pack.close()
+
+        raise KeyError(sha)
+
+    def get_raw(self, sha: bytes) -> tuple[int, bytes]:
+        """Obtain the raw text for an object.
+
+        Args:
+          sha: SHA1 of the object
+        Returns:
+          Tuple with numeric type and object contents
+        """
+        # Check cache first
+        if sha in self._cached_objects:
+            return self._cached_objects[sha]
+
+        # Try loose object first
+        try:
+            result = self._fetch_loose_object(sha)
+            self._cached_objects[sha] = result
+            return result
+        except KeyError:
+            pass
+
+        # Try packs
+        result = self._fetch_from_pack(sha)
+        self._cached_objects[sha] = result
+        return result
+
+    def contains_loose(self, sha: bytes) -> bool:
+        """Check if a particular object is present by SHA1 and is loose."""
+        try:
+            self._fetch_loose_object(sha)
+            return True
+        except KeyError:
+            return False
+
+    def __contains__(self, sha: bytes) -> bool:
+        """Check if a particular object is present by SHA1."""
+        if sha in self._cached_objects:
+            return True
+
+        # Try loose object
+        try:
+            self._fetch_loose_object(sha)
+            return True
+        except KeyError:
+            pass
+
+        # Try packs
+        try:
+            self._fetch_from_pack(sha)
+            return True
+        except KeyError:
+            return False
+
+    def __iter__(self) -> Iterator[bytes]:
+        """Iterate over all SHAs in the store.
+
+        Note: This is inefficient for dumb HTTP as it requires
+        downloading all pack indices.
+        """
+        seen = set()
+
+        # We can't efficiently list loose objects over dumb HTTP
+        # So we only iterate pack objects
+        self._load_packs()
+
+        for pack_name, idx in self._packs or []:
+            if idx is None:
+                idx = self._get_pack_index(pack_name)
+
+            for sha in idx:
+                if sha not in seen:
+                    seen.add(sha)
+                    yield sha_to_hex(sha)
+
+    @property
+    def packs(self):
+        """Iterable of pack objects.
+
+        Note: Returns empty list as we don't have actual Pack objects.
+        """
+        return []
+
+    def add_object(self, obj) -> None:
+        """Add a single object to this object store."""
+        raise NotImplementedError("Cannot add objects to dumb HTTP repository")
+
+    def add_objects(self, objects, progress=None) -> None:
+        """Add a set of objects to this object store."""
+        raise NotImplementedError("Cannot add objects to dumb HTTP repository")
+
+    def __del__(self):
+        """Clean up temporary directory on deletion."""
+        if self._temp_pack_dir and os.path.exists(self._temp_pack_dir):
+            import shutil
+
+            shutil.rmtree(self._temp_pack_dir, ignore_errors=True)
+
+
+class DumbRemoteHTTPRepo(BaseRepo):
+    """Repository implementation for dumb HTTP remotes."""
+
+    def __init__(self, base_url: str, http_request_func):
+        """Initialize a DumbRemoteHTTPRepo.
+
+        Args:
+          base_url: Base URL of the remote repository
+          http_request_func: Function to make HTTP requests.
+        """
+        self.base_url = base_url.rstrip("/") + "/"
+        self._http_request = http_request_func
+        self._refs: Optional[dict[Ref, ObjectID]] = None
+        self._peeled: Optional[dict[Ref, ObjectID]] = None
+        self._object_store = DumbHTTPObjectStore(base_url, http_request_func)
+
+    @property
+    def object_store(self):
+        """ObjectStore for this repository."""
+        return self._object_store
+
+    def _fetch_url(self, path: str) -> bytes:
+        """Fetch content from a URL path relative to base_url."""
+        url = urljoin(self.base_url, path)
+        resp, read = self._http_request(url, {})
+        try:
+            if resp.status == 404:
+                raise OSError(f"Not found: {url}")
+            elif resp.status != 200:
+                raise OSError(f"HTTP error {resp.status}: {url}")
+
+            chunks = []
+            while True:
+                chunk = read(4096)
+                if not chunk:
+                    break
+                chunks.append(chunk)
+            return b"".join(chunks)
+        finally:
+            resp.close()
+
+    def get_refs(self) -> dict[Ref, ObjectID]:
+        """Get dictionary with all refs."""
+        if self._refs is None:
+            # Fetch info/refs
+            try:
+                refs_data = self._fetch_url("info/refs")
+            except OSError:
+                raise NotGitRepository(f"Cannot read refs from {self.base_url}")
+
+            refs_hex = read_info_refs(BytesIO(refs_data))
+            # Keep SHAs as hex
+            self._refs, self._peeled = split_peeled_refs(refs_hex)
+
+        return dict(self._refs)
+
+    def get_peeled(self, ref: Ref) -> ObjectID:
+        """Get the peeled value of a ref."""
+        # For dumb HTTP, we don't have peeled refs readily available
+        # We would need to fetch and parse tag objects
+        sha = self.get_refs().get(ref, None)
+        return sha if sha is not None else ZERO_SHA
+
+    def fetch_pack_data(self, graph_walker, determine_wants, progress=None, depth=None):
+        """Fetch pack data from the remote.
+
+        This is the main method for fetching objects from a dumb HTTP remote.
+        Since dumb HTTP doesn't support negotiation, we need to download
+        all objects reachable from the wanted refs that we don't have locally.
+
+        Args:
+          graph_walker: GraphWalker instance that can tell us which commits we have
+          determine_wants: Function that returns list of wanted SHAs
+          progress: Optional progress callback
+          depth: Depth for shallow clones (not fully supported)
+
+        Returns:
+          Iterator of UnpackedObject instances
+        """
+        refs = self.get_refs()
+        wants = determine_wants(refs)
+
+        if not wants:
+            return
+
+        # For dumb HTTP, we can't negotiate, so we need to fetch all objects
+        # reachable from wants that we don't already have
+        to_fetch = set(wants)
+        seen = set()
+
+        while to_fetch:
+            sha = to_fetch.pop()
+            if sha in seen:
+                continue
+            seen.add(sha)
+
+            # Check if we already have this object
+            haves = list(graph_walker.ack(sha))
+            if haves:
+                continue
+
+            # Fetch the object
+            type_num, content = self._object_store.get_raw(sha)
+            unpacked = UnpackedObject(type_num, sha=sha)
+            unpacked.obj_type_num = type_num
+            unpacked.obj_chunks = [content]
+            yield unpacked
+
+            # If it's a commit or tag, we need to fetch its references
+            obj = ShaFile.from_raw_string(type_num, content)
+
+            if isinstance(obj, Commit):  # Commit
+                to_fetch.add(obj.tree)
+                for parent in obj.parents:
+                    to_fetch.add(parent)
+            elif isinstance(obj, Tag):  # Tag
+                to_fetch.add(obj.object[1])
+            elif isinstance(obj, Tree):  # Tree
+                for _, _, item_sha in obj.items():
+                    to_fetch.add(item_sha)
+
+            if progress:
+                progress(f"Fetching objects: {len(seen)} done")

+ 239 - 0
tests/compat/test_dumb.py

@@ -0,0 +1,239 @@
+# test_dumb.py -- Compatibility tests for dumb HTTP git repositories
+# Copyright (C) 2025 Dulwich contributors
+#
+# SPDX-License-Identifier: Apache-2.0 OR GPL-2.0-or-later
+# Dulwich is dual-licensed under the Apache License, Version 2.0 and the GNU
+# General Public License as public by the Free Software Foundation; version 2.0
+# or (at your option) any later version. You can redistribute it and/or
+# modify it under the terms of either of these two licenses.
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+# You should have received a copy of the licenses; if not, see
+# <http://www.gnu.org/licenses/> for a copy of the GNU General Public License
+# and <http://www.apache.org/licenses/LICENSE-2.0> for a copy of the Apache
+# License, Version 2.0.
+#
+
+"""Compatibility tests for dumb HTTP git repositories."""
+
+import os
+import shutil
+import tempfile
+import threading
+from http.server import HTTPServer, SimpleHTTPRequestHandler
+from unittest import skipUnless
+
+from dulwich.client import HttpGitClient
+from dulwich.repo import Repo
+from dulwich.tests.compat.utils import (
+    CompatTestCase,
+    run_git_or_fail,
+)
+
+
+class DumbHTTPRequestHandler(SimpleHTTPRequestHandler):
+    """HTTP request handler for dumb git protocol."""
+
+    def __init__(self, *args, directory=None, **kwargs):
+        self.directory = directory
+        super().__init__(*args, directory=directory, **kwargs)
+
+    def log_message(self, format, *args):
+        # Suppress logging during tests
+        pass
+
+
+class DumbHTTPGitServer:
+    """Simple HTTP server for serving git repositories."""
+
+    def __init__(self, root_path, port=0):
+        self.root_path = root_path
+
+        def handler(*args, **kwargs):
+            return DumbHTTPRequestHandler(*args, directory=root_path, **kwargs)
+
+        self.server = HTTPServer(("localhost", port), handler)
+        self.port = self.server.server_port
+        self.thread = None
+
+    def start(self):
+        """Start the HTTP server in a background thread."""
+        self.thread = threading.Thread(target=self.server.serve_forever)
+        self.thread.daemon = True
+        self.thread.start()
+
+    def stop(self):
+        """Stop the HTTP server."""
+        self.server.shutdown()
+        if self.thread:
+            self.thread.join()
+
+    @property
+    def url(self):
+        """Get the base URL for this server."""
+        return f"http://localhost:{self.port}"
+
+
+class DumbHTTPClientTests(CompatTestCase):
+    """Tests for dumb HTTP client against real git repositories."""
+
+    def setUp(self):
+        super().setUp()
+        # Create a temporary directory for test repos
+        self.temp_dir = tempfile.mkdtemp()
+        self.addCleanup(shutil.rmtree, self.temp_dir)
+
+        # Create origin repository
+        self.origin_path = os.path.join(self.temp_dir, "origin.git")
+        os.mkdir(self.origin_path)
+        run_git_or_fail(["init", "--bare"], cwd=self.origin_path)
+
+        # Create a working repository to push from
+        self.work_path = os.path.join(self.temp_dir, "work")
+        os.mkdir(self.work_path)
+        run_git_or_fail(["init"], cwd=self.work_path)
+        run_git_or_fail(
+            ["config", "user.email", "test@example.com"], cwd=self.work_path
+        )
+        run_git_or_fail(["config", "user.name", "Test User"], cwd=self.work_path)
+
+        # Create initial commit
+        test_file = os.path.join(self.work_path, "test.txt")
+        with open(test_file, "w") as f:
+            f.write("Hello, world!\n")
+        run_git_or_fail(["add", "test.txt"], cwd=self.work_path)
+        run_git_or_fail(["commit", "-m", "Initial commit"], cwd=self.work_path)
+
+        # Push to origin
+        run_git_or_fail(
+            ["remote", "add", "origin", self.origin_path], cwd=self.work_path
+        )
+        run_git_or_fail(["push", "origin", "master"], cwd=self.work_path)
+
+        # Update server info for dumb HTTP
+        run_git_or_fail(["update-server-info"], cwd=self.origin_path)
+
+        # Start HTTP server
+        self.server = DumbHTTPGitServer(self.origin_path)
+        self.server.start()
+        self.addCleanup(self.server.stop)
+
+    def test_clone_from_dumb_http(self):
+        """Test cloning from a dumb HTTP server."""
+        dest_path = os.path.join(self.temp_dir, "cloned")
+
+        # Use dulwich to clone via dumb HTTP
+        client = HttpGitClient(self.server.url)
+
+        # Create destination repo
+        dest_repo = Repo.init(dest_path)
+
+        # Fetch from dumb HTTP
+        def determine_wants(refs):
+            return [sha for ref, sha in refs.items() if ref.startswith(b"refs/heads/")]
+
+        result = client.fetch("/", dest_repo, determine_wants=determine_wants)
+
+        # Update refs
+        for ref, sha in result.refs.items():
+            if ref.startswith(b"refs/heads/"):
+                dest_repo.refs[ref] = sha
+
+        # Checkout files
+        dest_repo.reset_index()
+
+        # Verify the clone
+        test_file = os.path.join(dest_path, "test.txt")
+        self.assertTrue(os.path.exists(test_file))
+        with open(test_file) as f:
+            self.assertEqual("Hello, world!\n", f.read())
+
+    def test_fetch_new_commit_from_dumb_http(self):
+        """Test fetching new commits from a dumb HTTP server."""
+        # First clone the repository
+        dest_path = os.path.join(self.temp_dir, "cloned")
+        run_git_or_fail(["clone", self.server.url, dest_path])
+
+        # Make a new commit in the origin
+        test_file2 = os.path.join(self.work_path, "test2.txt")
+        with open(test_file2, "w") as f:
+            f.write("Second file\n")
+        run_git_or_fail(["add", "test2.txt"], cwd=self.work_path)
+        run_git_or_fail(["commit", "-m", "Second commit"], cwd=self.work_path)
+        run_git_or_fail(["push", "origin", "master"], cwd=self.work_path)
+
+        # Update server info again
+        run_git_or_fail(["update-server-info"], cwd=self.origin_path)
+
+        # Fetch with dulwich client
+        client = HttpGitClient(self.server.url)
+        dest_repo = Repo(dest_path)
+
+        old_refs = dest_repo.get_refs()
+
+        def determine_wants(refs):
+            wants = []
+            for ref, sha in refs.items():
+                if ref.startswith(b"refs/heads/") and sha != old_refs.get(ref):
+                    wants.append(sha)
+            return wants
+
+        result = client.fetch("/", dest_repo, determine_wants=determine_wants)
+
+        # Update refs
+        for ref, sha in result.refs.items():
+            if ref.startswith(b"refs/heads/"):
+                dest_repo.refs[ref] = sha
+
+        # Reset to new commit
+        dest_repo.reset_index(dest_repo.refs[b"refs/heads/master"])
+
+        # Verify the new file exists
+        test_file2_dest = os.path.join(dest_path, "test2.txt")
+        self.assertTrue(os.path.exists(test_file2_dest))
+        with open(test_file2_dest) as f:
+            self.assertEqual("Second file\n", f.read())
+
+    @skipUnless(
+        os.name == "posix", "Skipping on non-POSIX systems due to permission handling"
+    )
+    def test_fetch_from_dumb_http_with_tags(self):
+        """Test fetching tags from a dumb HTTP server."""
+        # Create a tag in origin
+        run_git_or_fail(["tag", "-a", "v1.0", "-m", "Version 1.0"], cwd=self.work_path)
+        run_git_or_fail(["push", "origin", "v1.0"], cwd=self.work_path)
+
+        # Update server info
+        run_git_or_fail(["update-server-info"], cwd=self.origin_path)
+
+        # Clone with dulwich
+        dest_path = os.path.join(self.temp_dir, "cloned_with_tags")
+        dest_repo = Repo.init(dest_path)
+
+        client = HttpGitClient(self.server.url)
+
+        def determine_wants(refs):
+            return [
+                sha
+                for ref, sha in refs.items()
+                if ref.startswith((b"refs/heads/", b"refs/tags/"))
+            ]
+
+        result = client.fetch("/", dest_repo, determine_wants=determine_wants)
+
+        # Update refs
+        for ref, sha in result.refs.items():
+            dest_repo.refs[ref] = sha
+
+        # Check that the tag exists
+        self.assertIn(b"refs/tags/v1.0", dest_repo.refs)
+
+        # Verify tag points to the right commit
+        tag_sha = dest_repo.refs[b"refs/tags/v1.0"]
+        tag_obj = dest_repo[tag_sha]
+        self.assertEqual(b"tag", tag_obj.type_name)

+ 4 - 1
tests/compat/test_pack.py

@@ -193,7 +193,10 @@ class TestPackIndexCompat(PackTests):
 
         # Load the pack and create v2 index (most compatible)
         pack_data = PackData(pack_path + ".pack")
-        pack_data.create_index(pack_path + ".idx", version=2)
+        try:
+            pack_data.create_index(pack_path + ".idx", version=2)
+        finally:
+            pack_data.close()
 
         # Verify git can read it
         output = run_git_or_fail(["verify-pack", "-v", pack_path + ".pack"])

+ 121 - 0
tests/test_client.py

@@ -1288,6 +1288,127 @@ class HttpGitClientTests(TestCase):
         with self.assertRaises(GitProtocolError, msg=error_msg):
             client.fetch_pack(b"/", check_heads, None, None)
 
+    def test_fetch_pack_dumb_http(self) -> None:
+        import zlib
+
+        from urllib3.response import HTTPResponse
+
+        # Mock responses for dumb HTTP
+        info_refs_content = (
+            b"0123456789abcdef0123456789abcdef01234567\trefs/heads/master\n"
+        )
+
+        # Create a blob object for testing
+        blob_content = b"Hello, dumb HTTP!"
+        blob_sha = b"0123456789abcdef0123456789abcdef01234567"
+        blob_hex = blob_sha.decode("ascii")
+        blob_obj_data = (
+            b"blob " + str(len(blob_content)).encode() + b"\x00" + blob_content
+        )
+        blob_compressed = zlib.compress(blob_obj_data)
+
+        responses = {
+            "/git-upload-pack": {
+                "status": 404,
+                "content": b"Not Found",
+                "content_type": "text/plain",
+            },
+            "/info/refs": {
+                "status": 200,
+                "content": info_refs_content,
+                "content_type": "text/plain",
+            },
+            f"/objects/{blob_hex[:2]}/{blob_hex[2:]}": {
+                "status": 200,
+                "content": blob_compressed,
+                "content_type": "application/octet-stream",
+            },
+        }
+
+        class PoolManagerMock:
+            def __init__(self) -> None:
+                self.headers: dict[str, str] = {}
+
+            def request(
+                self,
+                method,
+                url,
+                fields=None,
+                headers=None,
+                redirect=True,
+                preload_content=True,
+            ):
+                # Extract path from URL
+                from urllib.parse import urlparse
+
+                parsed = urlparse(url)
+                path = parsed.path.rstrip("/")
+
+                # Find matching response
+                for pattern, resp_data in responses.items():
+                    if path.endswith(pattern):
+                        return HTTPResponse(
+                            body=BytesIO(resp_data["content"]),
+                            headers={
+                                "Content-Type": resp_data.get(
+                                    "content_type", "text/plain"
+                                )
+                            },
+                            request_method=method,
+                            request_url=url,
+                            preload_content=preload_content,
+                            status=resp_data["status"],
+                        )
+
+                # Default 404
+                return HTTPResponse(
+                    body=BytesIO(b"Not Found"),
+                    headers={"Content-Type": "text/plain"},
+                    request_method=method,
+                    request_url=url,
+                    preload_content=preload_content,
+                    status=404,
+                )
+
+        def determine_wants(heads, **kwargs):
+            # heads contains the refs with SHA values, just return the SHA we want
+            return [heads[b"refs/heads/master"]]
+
+        received_data = []
+
+        def pack_data_handler(data):
+            # Collect pack data
+            received_data.append(data)
+
+        clone_url = "https://git.example.org/repo.git/"
+        client = HttpGitClient(clone_url, pool_manager=PoolManagerMock(), config=None)
+
+        # Mock graph walker that says we don't have anything
+        class MockGraphWalker:
+            def ack(self, sha):
+                return []
+
+        graph_walker = MockGraphWalker()
+
+        result = client.fetch_pack(
+            b"/", determine_wants, graph_walker, pack_data_handler
+        )
+
+        # Verify we got the refs
+        expected_sha = blob_hex.encode("ascii")
+        self.assertEqual({b"refs/heads/master": expected_sha}, result.refs)
+
+        # Verify we received pack data
+        self.assertTrue(len(received_data) > 0)
+        pack_data = b"".join(received_data)
+        self.assertTrue(len(pack_data) > 0)
+
+        # The pack should be valid pack format
+        self.assertTrue(pack_data.startswith(b"PACK"))
+        # Pack header: PACK + version (4 bytes) + num objects (4 bytes)
+        self.assertEqual(pack_data[4:8], b"\x00\x00\x00\x02")  # version 2
+        self.assertEqual(pack_data[8:12], b"\x00\x00\x00\x01")  # 1 object
+
 
 class TCPGitClientTests(TestCase):
     def test_get_url(self) -> None:

+ 290 - 0
tests/test_dumb.py

@@ -0,0 +1,290 @@
+# test_dumb.py -- Tests for dumb HTTP git repositories
+# Copyright (C) 2025 Dulwich contributors
+#
+# SPDX-License-Identifier: Apache-2.0 OR GPL-2.0-or-later
+# Dulwich is dual-licensed under the Apache License, Version 2.0 and the GNU
+# General Public License as public by the Free Software Foundation; version 2.0
+# or (at your option) any later version. You can redistribute it and/or
+# modify it under the terms of either of these two licenses.
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+# You should have received a copy of the licenses; if not, see
+# <http://www.gnu.org/licenses/> for a copy of the GNU General Public License
+# and <http://www.apache.org/licenses/LICENSE-2.0> for a copy of the Apache
+# License, Version 2.0.
+#
+
+"""Tests for dumb HTTP git repositories."""
+
+import zlib
+from unittest import TestCase
+from unittest.mock import Mock
+
+from dulwich.dumb import DumbHTTPObjectStore, DumbRemoteHTTPRepo
+from dulwich.errors import NotGitRepository
+from dulwich.objects import Blob, Commit, Tag, Tree, sha_to_hex
+
+
+class MockResponse:
+    def __init__(self, status=200, content=b"", headers=None):
+        self.status = status
+        self.content = content
+        self.headers = headers or {}
+        self.closed = False
+
+    def close(self):
+        self.closed = True
+
+
+class DumbHTTPObjectStoreTests(TestCase):
+    """Tests for DumbHTTPObjectStore."""
+
+    def setUp(self):
+        self.base_url = "https://example.com/repo.git/"
+        self.responses = {}
+        self.store = DumbHTTPObjectStore(self.base_url, self._mock_http_request)
+
+    def _mock_http_request(self, url, headers):
+        """Mock HTTP request function."""
+        if url in self.responses:
+            resp_data = self.responses[url]
+            resp = MockResponse(
+                resp_data.get("status", 200), resp_data.get("content", b"")
+            )
+            # Create a mock read function that behaves like urllib3's read
+            content = resp.content
+            offset = [0]  # Use list to make it mutable in closure
+
+            def read_func(size=None):
+                if offset[0] >= len(content):
+                    return b""
+                if size is None:
+                    result = content[offset[0] :]
+                    offset[0] = len(content)
+                else:
+                    result = content[offset[0] : offset[0] + size]
+                    offset[0] += size
+                return result
+
+            return resp, read_func
+        else:
+            resp = MockResponse(404)
+            return resp, lambda size: b""
+
+    def _add_response(self, path, content, status=200):
+        """Add a mock response for a given path."""
+        url = self.base_url + path
+        self.responses[url] = {"status": status, "content": content}
+
+    def _make_object(self, obj):
+        """Create compressed git object data."""
+        type_name = {
+            Blob.type_num: b"blob",
+            Tree.type_num: b"tree",
+            Commit.type_num: b"commit",
+            Tag.type_num: b"tag",
+        }[obj.type_num]
+
+        content = obj.as_raw_string()
+        header = type_name + b" " + str(len(content)).encode() + b"\x00"
+        return zlib.compress(header + content)
+
+    def test_fetch_loose_object_blob(self):
+        # Create a blob object
+        blob = Blob()
+        blob.data = b"Hello, world!"
+        hex_sha = blob.id
+
+        # Add mock response
+        path = f"objects/{hex_sha[:2].decode('ascii')}/{hex_sha[2:].decode('ascii')}"
+        self._add_response(path, self._make_object(blob))
+
+        # Fetch the object
+        type_num, content = self.store._fetch_loose_object(blob.id)
+        self.assertEqual(Blob.type_num, type_num)
+        self.assertEqual(b"Hello, world!", content)
+
+    def test_fetch_loose_object_not_found(self):
+        hex_sha = b"1" * 40
+        self.assertRaises(KeyError, self.store._fetch_loose_object, hex_sha)
+
+    def test_fetch_loose_object_invalid_format(self):
+        sha = b"1" * 20
+        hex_sha = sha_to_hex(sha)
+        path = f"objects/{hex_sha[:2]}/{hex_sha[2:]}"
+
+        # Add invalid compressed data
+        self._add_response(path, b"invalid data")
+
+        self.assertRaises(Exception, self.store._fetch_loose_object, sha)
+
+    def test_load_packs_empty(self):
+        # No packs file
+        self.store._load_packs()
+        self.assertEqual([], self.store._packs)
+
+    def test_load_packs_with_entries(self):
+        packs_content = b"""P pack-1234567890abcdef1234567890abcdef12345678.pack
+P pack-abcdef1234567890abcdef1234567890abcdef12.pack
+"""
+        self._add_response("objects/info/packs", packs_content)
+
+        self.store._load_packs()
+        self.assertEqual(2, len(self.store._packs))
+        self.assertEqual(
+            "pack-1234567890abcdef1234567890abcdef12345678", self.store._packs[0][0]
+        )
+        self.assertEqual(
+            "pack-abcdef1234567890abcdef1234567890abcdef12", self.store._packs[1][0]
+        )
+
+    def test_get_raw_from_cache(self):
+        sha = b"1" * 40
+        self.store._cached_objects[sha] = (Blob.type_num, b"cached content")
+
+        type_num, content = self.store.get_raw(sha)
+        self.assertEqual(Blob.type_num, type_num)
+        self.assertEqual(b"cached content", content)
+
+    def test_contains_loose(self):
+        # Create a blob object
+        blob = Blob()
+        blob.data = b"Test blob"
+        hex_sha = blob.id
+
+        # Add mock response
+        path = f"objects/{hex_sha[:2].decode('ascii')}/{hex_sha[2:].decode('ascii')}"
+        self._add_response(path, self._make_object(blob))
+
+        self.assertTrue(self.store.contains_loose(hex_sha))
+        self.assertFalse(self.store.contains_loose(b"0" * 40))
+
+    def test_add_object_not_implemented(self):
+        blob = Blob()
+        blob.data = b"test"
+        self.assertRaises(NotImplementedError, self.store.add_object, blob)
+
+    def test_add_objects_not_implemented(self):
+        self.assertRaises(NotImplementedError, self.store.add_objects, [])
+
+
+class DumbRemoteHTTPRepoTests(TestCase):
+    """Tests for DumbRemoteHTTPRepo."""
+
+    def setUp(self):
+        self.base_url = "https://example.com/repo.git/"
+        self.responses = {}
+        self.repo = DumbRemoteHTTPRepo(self.base_url, self._mock_http_request)
+
+    def _mock_http_request(self, url, headers):
+        """Mock HTTP request function."""
+        if url in self.responses:
+            resp_data = self.responses[url]
+            resp = MockResponse(
+                resp_data.get("status", 200), resp_data.get("content", b"")
+            )
+            # Create a mock read function that behaves like urllib3's read
+            content = resp.content
+            offset = [0]  # Use list to make it mutable in closure
+
+            def read_func(size=None):
+                if offset[0] >= len(content):
+                    return b""
+                if size is None:
+                    result = content[offset[0] :]
+                    offset[0] = len(content)
+                else:
+                    result = content[offset[0] : offset[0] + size]
+                    offset[0] += size
+                return result
+
+            return resp, read_func
+        else:
+            resp = MockResponse(404)
+            return resp, lambda size: b""
+
+    def _add_response(self, path, content, status=200):
+        """Add a mock response for a given path."""
+        url = self.base_url + path
+        self.responses[url] = {"status": status, "content": content}
+
+    def test_get_refs(self):
+        refs_content = b"""0123456789abcdef0123456789abcdef01234567\trefs/heads/master
+abcdef0123456789abcdef0123456789abcdef01\trefs/heads/develop
+fedcba9876543210fedcba9876543210fedcba98\trefs/tags/v1.0
+"""
+        self._add_response("info/refs", refs_content)
+
+        refs = self.repo.get_refs()
+        self.assertEqual(3, len(refs))
+        self.assertEqual(
+            b"0123456789abcdef0123456789abcdef01234567",
+            refs[b"refs/heads/master"],
+        )
+        self.assertEqual(
+            b"abcdef0123456789abcdef0123456789abcdef01",
+            refs[b"refs/heads/develop"],
+        )
+        self.assertEqual(
+            b"fedcba9876543210fedcba9876543210fedcba98",
+            refs[b"refs/tags/v1.0"],
+        )
+
+    def test_get_refs_not_found(self):
+        self.assertRaises(NotGitRepository, self.repo.get_refs)
+
+    def test_get_peeled(self):
+        refs_content = b"0123456789abcdef0123456789abcdef01234567\trefs/heads/master\n"
+        self._add_response("info/refs", refs_content)
+
+        # For dumb HTTP, peeled just returns the ref value
+        peeled = self.repo.get_peeled(b"refs/heads/master")
+        self.assertEqual(b"0123456789abcdef0123456789abcdef01234567", peeled)
+
+    def test_fetch_pack_data_no_wants(self):
+        refs_content = b"0123456789abcdef0123456789abcdef01234567\trefs/heads/master\n"
+        self._add_response("info/refs", refs_content)
+
+        graph_walker = Mock()
+
+        def determine_wants(refs):
+            return []
+
+        result = list(self.repo.fetch_pack_data(graph_walker, determine_wants))
+        self.assertEqual([], result)
+
+    def test_fetch_pack_data_with_blob(self):
+        # Set up refs
+        refs_content = b"0123456789abcdef0123456789abcdef01234567\trefs/heads/master\n"
+        self._add_response("info/refs", refs_content)
+
+        # Create a simple blob object
+        blob = Blob()
+        blob.data = b"Test content"
+        blob_sha = blob.id
+        # Add blob response
+        self.repo._object_store._cached_objects[blob_sha] = (
+            Blob.type_num,
+            blob.as_raw_string(),
+        )
+
+        # Mock graph walker
+        graph_walker = Mock()
+        graph_walker.ack.return_value = []  # No existing objects
+
+        def determine_wants(refs):
+            return [blob_sha]
+
+        result = list(self.repo.fetch_pack_data(graph_walker, determine_wants))
+        self.assertEqual(1, len(result))
+        self.assertEqual(Blob.type_num, result[0].pack_type_num)
+        self.assertEqual([blob.as_raw_string()], result[0].obj_chunks)
+
+    def test_object_store_property(self):
+        self.assertIsInstance(self.repo.object_store, DumbHTTPObjectStore)
+        self.assertEqual(self.base_url, self.repo.object_store.base_url)