Explorar el Código

Add dulwich.aiohttp.

Jelmer Vernooij hace 3 años
padre
commit
3d1dbb2566

+ 3 - 0
NEWS

@@ -6,6 +6,9 @@
  * Add ``parse_commit_broken`` function to parse broken commits.
    (Valentin Lorentz, Jelmer Vernooij)
 
+ * Add basic ``dulwich.aiohttp`` module that provides
+   server support. (Jelmer Vernooij)
+
 0.25.0	2025-12-17
 
 **PLEASE NOTE**: This release makes quite a lot of changes to public APIs. This

+ 22 - 0
dulwich/aiohttp/__init__.py

@@ -0,0 +1,22 @@
+# __init__.py -- aiohttp support
+# Copyright (C) 2022 Jelmer Vernooij <jelmer@jelmer.uk>
+#
+# 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 published 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.
+#
+
+"""aiohttp support for Dulwich."""

+ 361 - 0
dulwich/aiohttp/server.py

@@ -0,0 +1,361 @@
+# aiohttp.py -- aiohttp smart client/server
+# Copyright (C) 2022 Jelmer Vernooij <jelmer@jelmer.uk>
+#
+# 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 published 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.
+#
+
+"""aiohttp client/server support."""
+
+import asyncio
+import sys
+from io import BytesIO
+from typing import BinaryIO, cast
+
+from aiohttp import web
+
+from .. import log_utils
+from ..errors import HangupException
+from ..protocol import ReceivableProtocol
+from ..repo import Repo
+from ..server import (
+    DEFAULT_HANDLERS,
+    BackendRepo,
+    DictBackend,
+    generate_info_refs,
+    generate_objects_info_packs,
+)
+from ..web import NO_CACHE_HEADERS, cache_forever_headers
+
+logger = log_utils.getLogger(__name__)
+
+
+async def send_file(
+    req: web.Request, f: BinaryIO | None, headers: dict[str, str]
+) -> web.StreamResponse:
+    """Send a file-like object to the request output.
+
+    Args:
+      req: The HTTPGitRequest object to send output to.
+      f: An open file-like object to send; will be closed.
+      headers: Headers to send
+    Returns: Iterator over the contents of the file, as chunks.
+    """
+    if f is None:
+        raise web.HTTPNotFound(text="File not found")
+    response = web.StreamResponse(status=200, reason="OK", headers=headers)
+    await response.prepare(req)
+    try:
+        while True:
+            data = f.read(10240)
+            if not data:
+                break
+            await response.write(data)
+    except OSError:
+        raise web.HTTPInternalServerError(text="Error reading file")
+    finally:
+        f.close()
+    await response.write_eof()
+    return response
+
+
+async def get_loose_object(request: web.Request) -> web.Response:
+    """Handle request for a loose object.
+
+    Args:
+      request: aiohttp request object
+    Returns: Response with the loose object data
+    """
+    sha = (request.match_info["dir"] + request.match_info["file"]).encode("ascii")
+    logger.info("Sending loose object %s", sha)
+    object_store = request.app["repo"].object_store
+    if not object_store.contains_loose(sha):
+        raise web.HTTPNotFound(text="Object not found")
+    try:
+        data = object_store[sha].as_legacy_object()
+    except OSError:
+        raise web.HTTPInternalServerError(text="Error reading object")
+    headers = {"Content-Type": "application/x-git-loose-object"}
+    headers.update(cache_forever_headers())
+    return web.Response(status=200, headers=headers, body=data)
+
+
+async def get_text_file(request: web.Request) -> web.StreamResponse:
+    """Handle request for a text file.
+
+    Args:
+      request: aiohttp request object
+    Returns: Response with the text file contents
+    """
+    headers = {"Content-Type": "text/plain"}
+    headers.update(NO_CACHE_HEADERS)
+    path = request.match_info["file"]
+    logger.info("Sending plain text file %s", path)
+    repo = request.app["repo"]
+    return await send_file(request, repo.get_named_file(path), headers)
+
+
+async def refs_request(
+    repo: Repo, request: web.Request, handlers: dict[bytes, type] | None = None
+) -> web.StreamResponse | web.Response:
+    """Handle a refs request.
+
+    Args:
+      repo: Repository object
+      request: aiohttp request object
+      handlers: Optional dict of service handlers
+    Returns: Response with refs information
+    """
+    service = request.query.get("service")
+    if service:
+        if handlers is None:
+            handlers = dict(DEFAULT_HANDLERS)
+        handler_cls = handlers.get(service.encode("ascii"), None)
+        if handler_cls is None:
+            raise web.HTTPForbidden(text="Unsupported service")
+        headers = {"Content-Type": f"application/x-{service}-advertisement"}
+        headers.update(NO_CACHE_HEADERS)
+
+        response = web.StreamResponse(status=200, headers=headers)
+
+        await response.prepare(request)
+
+        out = BytesIO()
+        proto = ReceivableProtocol(BytesIO().read, out.write)
+        handler = handler_cls(
+            DictBackend({b".": cast(BackendRepo, repo)}),
+            [b"."],
+            proto,
+            stateless_rpc=True,
+            advertise_refs=True,
+        )
+        handler.proto.write_pkt_line(b"# service=" + service.encode("ascii") + b"\n")
+        handler.proto.write_pkt_line(None)
+
+        # TODO(jelmer): Implement this with proper async code
+        await asyncio.to_thread(handler.handle)
+
+        await response.write(out.getvalue())
+
+        await response.write_eof()
+
+        return response
+    else:
+        # non-smart fallback
+        headers = {"Content-Type": "text/plain"}
+        headers.update(NO_CACHE_HEADERS)
+        logger.info("Emulating dumb info/refs")
+        return web.Response(body=b"".join(generate_info_refs(repo)), headers=headers)
+
+
+async def get_info_refs(request: web.Request) -> web.StreamResponse | web.Response:
+    """Handle request for /info/refs.
+
+    Args:
+      request: aiohttp request object
+    Returns: Response with refs information
+    """
+    repo = request.app["repo"]
+    return await refs_request(repo, request, request.app["handlers"])
+
+
+async def get_info_packs(request: web.Request) -> web.Response:
+    """Handle request for /info/packs.
+
+    Args:
+      request: aiohttp request object
+    Returns: Response with pack information
+    """
+    headers = {"Content-Type": "text/plain"}
+    headers.update(NO_CACHE_HEADERS)
+    logger.info("Emulating dumb info/packs")
+    return web.Response(
+        body=b"".join(generate_objects_info_packs(request.app["repo"])), headers=headers
+    )
+
+
+async def get_pack_file(request: web.Request) -> web.StreamResponse:
+    """Handle request for a pack file.
+
+    Args:
+      request: aiohttp request object
+    Returns: Response with the pack file data
+    """
+    headers = {"Content-Type": "application/x-git-packed-objects"}
+    headers.update(cache_forever_headers())
+    sha = request.match_info["sha"]
+    path = f"objects/pack/pack-{sha}.pack"
+    logger.info("Sending pack file %s", path)
+    return await send_file(
+        request,
+        request.app["repo"].get_named_file(path),
+        headers=headers,
+    )
+
+
+async def get_index_file(request: web.Request) -> web.StreamResponse:
+    """Handle request for a pack index file.
+
+    Args:
+      request: aiohttp request object
+    Returns: Response with the pack index file data
+    """
+    headers = {"Content-Type": "application/x-git-packed-objects-toc"}
+    headers.update(cache_forever_headers())
+    sha = request.match_info["sha"]
+    path = f"objects/pack/pack-{sha}.idx"
+    logger.info("Sending pack file %s", path)
+    return await send_file(
+        request, request.app["repo"].get_named_file(path), headers=headers
+    )
+
+
+async def service_request(
+    repo: Repo, request: web.Request, handlers: dict[bytes, type] | None = None
+) -> web.StreamResponse:
+    """Handle a git service request (upload-pack or receive-pack).
+
+    Args:
+      repo: Repository object
+      request: aiohttp request object
+      handlers: Optional dict of service handlers
+    Returns: Response with service result
+    """
+    service = request.match_info["service"]
+    if handlers is None:
+        handlers = dict(DEFAULT_HANDLERS)
+    logger.info("Handling service request for %s", service)
+    handler_cls = handlers.get(service.encode("ascii"), None)
+    if handler_cls is None:
+        raise web.HTTPForbidden(text="Unsupported service")
+    headers = {"Content-Type": f"application/x-{service}-result"}
+    headers.update(NO_CACHE_HEADERS)
+
+    response = web.StreamResponse(status=200, headers=headers)
+
+    await response.prepare(request)
+
+    inf = BytesIO(await request.read())
+    outf = BytesIO()
+
+    def handle() -> None:
+        proto = ReceivableProtocol(inf.read, outf.write)
+        handler = handler_cls(
+            DictBackend({b".": cast(BackendRepo, repo)}),
+            [b"."],
+            proto,
+            stateless_rpc=True,
+        )
+        try:
+            handler.handle()
+        except HangupException:
+            response.force_close()
+
+    # TODO(jelmer): Implement this with proper async code
+    await asyncio.to_thread(handle)
+
+    await response.write(outf.getvalue())
+
+    await response.write_eof()
+    return response
+
+
+async def handle_service_request(request: web.Request) -> web.StreamResponse:
+    """Handle a service request endpoint.
+
+    Args:
+      request: aiohttp request object
+    Returns: Response with service result
+    """
+    repo = request.app["repo"]
+
+    return await service_request(repo, request, request.app["handlers"])
+
+
+def create_repo_app(
+    repo: Repo, handlers: dict[bytes, type] | None = None, dumb: bool = False
+) -> web.Application:
+    """Create an aiohttp application for serving a git repository.
+
+    Args:
+      repo: Repository object to serve
+      handlers: Optional dict of service handlers
+      dumb: Whether to enable dumb HTTP protocol support
+    Returns: Configured aiohttp Application
+    """
+    app = web.Application()
+    app["repo"] = repo
+    if handlers is None:
+        handlers = dict(DEFAULT_HANDLERS)
+    app["handlers"] = handlers
+    app["dumb"] = dumb
+    app.router.add_get("/info/refs", get_info_refs)
+    app.router.add_post(
+        "/{service:git-upload-pack|git-receive-pack}", handle_service_request
+    )
+    if dumb:
+        app.router.add_get("/{file:HEAD}", get_text_file)
+        app.router.add_get("/{file:objects/info/alternates}", get_text_file)
+        app.router.add_get("/{file:objects/info/http-alternates}", get_text_file)
+        app.router.add_get("/objects/info/packs", get_info_packs)
+        app.router.add_get(
+            "/objects/{dir:[0-9a-f]{2}}/{file:[0-9a-f]{38}}", get_loose_object
+        )
+        app.router.add_get(
+            "/objects/pack/pack-{sha:[0-9a-f]{40}}\\.pack", get_pack_file
+        )
+        app.router.add_get(
+            "/objects/pack/pack-{sha:[0-9a-f]{40}}\\.idx", get_index_file
+        )
+    return app
+
+
+def main(argv: list[str] | None = None) -> None:
+    """Entry point for starting an HTTP git server."""
+    import argparse
+
+    parser = argparse.ArgumentParser()
+    parser.add_argument(
+        "-l",
+        "--listen_address",
+        dest="listen_address",
+        default="localhost",
+        help="Binding IP address.",
+    )
+    parser.add_argument(
+        "-p",
+        "--port",
+        dest="port",
+        type=int,
+        default=8000,
+        help="Port to listen on.",
+    )
+    parser.add_argument("gitdir", type=str, default=".", nargs="?")
+    args = parser.parse_args(argv)
+
+    log_utils.default_logging_config()
+    app = create_repo_app(Repo(args.gitdir))
+    logger.info(
+        "Listening for HTTP connections on %s:%d",
+        args.listen_address,
+        args.port,
+    )
+    web.run_app(app, port=args.port, host=args.listen_address)
+
+
+if __name__ == "__main__":
+    main(sys.argv[1:])

+ 1 - 0
pyproject.toml

@@ -51,6 +51,7 @@ dev = [
 merge = ["merge3"]
 fuzzing = ["atheris"]
 patiencediff = ["patiencediff"]
+aiohttp = ["aiohttp"]
 
 [project.scripts]
 dulwich = "dulwich.cli:main"

+ 1 - 0
tests/__init__.py

@@ -123,6 +123,7 @@ class BlackboxTestCase(TestCase):
 
 def self_test_suite() -> unittest.TestSuite:
     names = [
+        "aiohttp",
         "annotate",
         "approxidate",
         "archive",

+ 1 - 0
tests/compat/__init__.py

@@ -26,6 +26,7 @@ import unittest
 
 def test_suite() -> unittest.TestSuite:
     names = [
+        "aiohttp",
         "bitmap",
         "bundle",
         "check_ignore",

+ 216 - 0
tests/compat/test_aiohttp.py

@@ -0,0 +1,216 @@
+# test_aiohttp.py -- Compatibility tests for the aiohttp HTTP server.
+# Copyright (C) 2025 Jelmer Vernooij <jelmer@jelmer.uk>
+#
+# 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 published 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 between Dulwich and the cgit HTTP client using aiohttp server.
+
+warning: these tests should be fairly stable, but when writing/debugging new
+    tests, deadlocks may freeze the test process such that it cannot be
+    Ctrl-C'ed. On POSIX systems, you can kill the tests with Ctrl-Z, "kill %".
+"""
+
+import asyncio
+import sys
+import threading
+from typing import NoReturn
+
+try:
+    from aiohttp import web
+
+    aiohttp_missing = False
+except ImportError:
+    web = None  # type: ignore
+    aiohttp_missing = True
+
+from dulwich.server import ReceivePackHandler
+
+from .. import SkipTest, skipIf
+from .server_utils import NoSideBand64kReceivePackHandler, ServerTests
+from .utils import CompatTestCase
+
+if not aiohttp_missing:
+    from dulwich.aiohttp.server import create_repo_app
+
+
+@skipIf(aiohttp_missing, "aiohttp not available")
+@skipIf(sys.platform == "win32", "Broken on windows, with very long fail time.")
+class AiohttpServerTests(ServerTests):
+    """Base tests for aiohttp server tests.
+
+    Contains utility and setUp/tearDown methods, but does not inherit from
+    TestCase so tests are not automatically run.
+    """
+
+    protocol = "http"
+
+    def _start_server(self, repo):
+        app = self._make_app(repo)
+        runner = web.AppRunner(app)
+
+        loop = asyncio.new_event_loop()
+
+        async def start():
+            await runner.setup()
+            site = web.TCPSite(runner, "localhost", 0)
+            await site.start()
+            return site
+
+        site = loop.run_until_complete(start())
+
+        # Get the actual port
+        port = site._server.sockets[0].getsockname()[1]
+
+        # Run the event loop in a separate thread
+        def run_loop():
+            asyncio.set_event_loop(loop)
+            loop.run_forever()
+
+        thread = threading.Thread(target=run_loop, daemon=True)
+        thread.start()
+
+        # Cleanup function
+        def cleanup():
+            async def stop():
+                await runner.cleanup()
+
+            future = asyncio.run_coroutine_threadsafe(stop(), loop)
+            future.result(timeout=5)
+            loop.call_soon_threadsafe(loop.stop)
+            thread.join(timeout=1.0)
+
+        self.addCleanup(cleanup)
+        self._server = runner
+        return port
+
+
+@skipIf(aiohttp_missing, "aiohttp not available")
+@skipIf(sys.platform == "win32", "Broken on windows, with very long fail time.")
+class SmartAiohttpTestCase(AiohttpServerTests, CompatTestCase):
+    """Test cases for smart HTTP server using aiohttp.
+
+    This server test case does not use side-band-64k in git-receive-pack.
+    """
+
+    min_git_version: tuple[int, ...] = (1, 6, 6)
+
+    def _handlers(self):
+        return {b"git-receive-pack": NoSideBand64kReceivePackHandler}
+
+    def _make_app(self, repo):
+        from dulwich.server import DEFAULT_HANDLERS
+
+        handlers = dict(DEFAULT_HANDLERS)
+        handlers.update(self._handlers())
+        return create_repo_app(repo, handlers=handlers)
+
+
+def patch_capabilities(handler, caps_removed):
+    # Patch a handler's capabilities by specifying a list of them to be
+    # removed, and return the original method for restoration.
+    original_capabilities = handler.capabilities
+
+    def capabilities(self):
+        # Call original to get base capabilities (including object-format)
+        base_caps = original_capabilities(self)
+        # Filter out the capabilities we want to remove
+        return [i for i in base_caps if i not in caps_removed]
+
+    handler.capabilities = capabilities
+    return original_capabilities
+
+
+@skipIf(aiohttp_missing, "aiohttp not available")
+@skipIf(sys.platform == "win32", "Broken on windows, with very long fail time.")
+class SmartAiohttpSideBand64kTestCase(SmartAiohttpTestCase):
+    """Test cases for smart HTTP server with side-band-64k support using aiohttp."""
+
+    # side-band-64k in git-receive-pack was introduced in git 1.7.0.2
+    min_git_version = (1, 7, 0, 2)
+
+    def setUp(self) -> None:
+        from dulwich.server import UploadPackHandler
+
+        self.o_uph_cap = patch_capabilities(UploadPackHandler, (b"no-done",))
+        self.o_rph_cap = patch_capabilities(ReceivePackHandler, (b"no-done",))
+        super().setUp()
+
+    def tearDown(self) -> None:
+        from dulwich.server import UploadPackHandler
+
+        super().tearDown()
+        UploadPackHandler.capabilities = self.o_uph_cap
+        ReceivePackHandler.capabilities = self.o_rph_cap
+
+    def _make_app(self, repo):
+        return create_repo_app(repo)
+
+
+@skipIf(aiohttp_missing, "aiohttp not available")
+@skipIf(sys.platform == "win32", "Broken on windows, with very long fail time.")
+class SmartAiohttpSideBand64kNoDoneTestCase(SmartAiohttpTestCase):
+    """Test cases for smart HTTP server with side-band-64k and no-done
+    support using aiohttp.
+    """
+
+    # no-done was introduced in git 1.7.4
+    min_git_version = (1, 7, 4)
+
+    def _make_app(self, repo):
+        return create_repo_app(repo)
+
+
+@skipIf(aiohttp_missing, "aiohttp not available")
+@skipIf(sys.platform == "win32", "Broken on windows, with very long fail time.")
+class DumbAiohttpTestCase(AiohttpServerTests, CompatTestCase):
+    """Test cases for dumb HTTP server using aiohttp."""
+
+    def _make_app(self, repo):
+        return create_repo_app(repo, dumb=True)
+
+    def test_push_to_dulwich(self) -> NoReturn:
+        # Note: remove this if dulwich implements dumb web pushing.
+        raise SkipTest("Dumb web pushing not supported.")
+
+    def test_push_to_dulwich_remove_branch(self) -> NoReturn:
+        # Note: remove this if dumb pushing is supported
+        raise SkipTest("Dumb web pushing not supported.")
+
+    def test_new_shallow_clone_from_dulwich(self) -> NoReturn:
+        # Note: remove this if C git and dulwich implement dumb web shallow
+        # clones.
+        raise SkipTest("Dumb web shallow cloning not supported.")
+
+    def test_shallow_clone_from_git_is_identical(self) -> NoReturn:
+        # Note: remove this if C git and dulwich implement dumb web shallow
+        # clones.
+        raise SkipTest("Dumb web shallow cloning not supported.")
+
+    def test_fetch_same_depth_into_shallow_clone_from_dulwich(self) -> NoReturn:
+        # Note: remove this if C git and dulwich implement dumb web shallow
+        # clones.
+        raise SkipTest("Dumb web shallow cloning not supported.")
+
+    def test_fetch_full_depth_into_shallow_clone_from_dulwich(self) -> NoReturn:
+        # Note: remove this if C git and dulwich implement dumb web shallow
+        # clones.
+        raise SkipTest("Dumb web shallow cloning not supported.")
+
+    def test_push_to_dulwich_issue_88_standard(self) -> NoReturn:
+        raise SkipTest("Dumb web pushing not supported.")

+ 129 - 0
tests/test_aiohttp.py

@@ -0,0 +1,129 @@
+# test_aiohttp.py -- Tests for the aiohttp HTTP server
+# Copyright (C) 2025 Jelmer Vernooij <jelmer@jelmer.uk>
+#
+# 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 published 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 the aiohttp Git HTTP server."""
+
+try:
+    from aiohttp.test_utils import AioHTTPTestCase
+
+    aiohttp_missing = False
+except ImportError:
+    from unittest import TestCase as AioHTTPTestCase  # type: ignore
+
+    aiohttp_missing = True
+
+from dulwich.objects import Blob
+from dulwich.repo import MemoryRepo
+from dulwich.tests.utils import make_object
+
+from . import skipIf
+
+if not aiohttp_missing:
+    from dulwich.aiohttp.server import create_repo_app
+
+
+@skipIf(aiohttp_missing, "aiohttp not available")
+class AiohttpAppTestCase(AioHTTPTestCase):  # type: ignore
+    """Test the aiohttp application."""
+
+    def setUp(self):
+        super().setUp()
+        self.repo = MemoryRepo.init_bare([], {})
+        self.blob = make_object(Blob, data=b"blob contents")
+        self.repo.object_store.add_object(self.blob)
+        self.repo.refs[b"refs/heads/master"] = self.blob.id
+
+    def get_app(self):
+        return create_repo_app(self.repo)
+
+    async def test_get_info_refs_dumb(self):
+        """Test GET /info/refs without service parameter (dumb protocol)."""
+        resp = await self.client.request("GET", "/info/refs")
+        self.assertEqual(resp.status, 200)
+        self.assertEqual(resp.content_type, "text/plain")
+        text = await resp.text()
+        self.assertIn(self.blob.id.decode("ascii"), text)
+        self.assertIn("refs/heads/master", text)
+
+    async def test_get_info_refs_smart(self):
+        """Test GET /info/refs?service=git-upload-pack (smart protocol)."""
+        resp = await self.client.request("GET", "/info/refs?service=git-upload-pack")
+        self.assertEqual(resp.status, 200)
+        self.assertIn("git-upload-pack", resp.content_type)
+
+    async def test_post_upload_pack(self):
+        """Test POST /git-upload-pack."""
+        # Simple test that the endpoint exists and accepts POST
+        resp = await self.client.request(
+            "POST",
+            "/git-upload-pack",
+            data=b"0000",
+            headers={"Content-Type": "application/x-git-upload-pack-request"},
+        )
+        # Should respond with 200 even for invalid/minimal input
+        self.assertEqual(resp.status, 200)
+        self.assertIn("git-upload-pack", resp.content_type)
+
+
+@skipIf(aiohttp_missing, "aiohttp not available")
+class AiohttpDumbAppTestCase(AioHTTPTestCase):  # type: ignore
+    """Test the aiohttp application with dumb protocol."""
+
+    def setUp(self):
+        super().setUp()
+        self.repo = MemoryRepo.init_bare([], {})
+        self.repo._put_named_file("HEAD", b"ref: refs/heads/master\n")
+        self.blob = make_object(Blob, data=b"blob contents")
+        self.repo.object_store.add_object(self.blob)
+        self.repo.refs[b"refs/heads/master"] = self.blob.id
+
+    def get_app(self):
+        return create_repo_app(self.repo, dumb=True)
+
+    async def test_get_head(self):
+        """Test GET /HEAD."""
+        resp = await self.client.request("GET", "/HEAD")
+        self.assertEqual(resp.status, 200)
+        self.assertEqual(resp.content_type, "text/plain")
+        text = await resp.text()
+        self.assertEqual(text, "ref: refs/heads/master\n")
+
+    async def test_get_info_packs(self):
+        """Test GET /objects/info/packs."""
+        resp = await self.client.request("GET", "/objects/info/packs")
+        self.assertEqual(resp.status, 200)
+        self.assertEqual(resp.content_type, "text/plain")
+
+    async def test_get_loose_object(self):
+        """Test GET /objects/{dir}/{file} for loose objects."""
+        sha = self.blob.id.decode("ascii")
+        dir_part = sha[:2]
+        file_part = sha[2:]
+        resp = await self.client.request("GET", f"/objects/{dir_part}/{file_part}")
+        self.assertEqual(resp.status, 200)
+        self.assertEqual(resp.content_type, "application/x-git-loose-object")
+        body = await resp.read()
+        self.assertEqual(body, self.blob.as_legacy_object())
+
+    async def test_get_loose_object_not_found(self):
+        """Test GET /objects/{dir}/{file} for non-existent object."""
+        resp = await self.client.request("GET", "/objects/ab/cdef" + "0" * 36)
+        self.assertEqual(resp.status, 404)