Browse Source

Use a DB FOR UPDATE lock when updating the FlatFileIndex (#54442)

This is based on top of https://github.com/getsentry/sentry/pull/54440
and reverts most of https://github.com/getsentry/sentry/pull/53690

This reverts all of the redis locks and manual
`get_or_create`/`update_or_create` workarounds. Instead of using redis
locks, this will now use a non blocking `FOR UPDATE` lock on the
database when updating the flat file index.

The main `update_artifact_bundle_index` function is also changed to work
in a batched way, able to supply multiple manifests at the same time.
Arpad Borsos 1 year ago
parent
commit
45dac913da

+ 1 - 1
src/sentry/api/endpoints/artifact_lookup.py

@@ -81,7 +81,7 @@ class ProjectArtifactLookupEndpoint(ProjectEndpoint):
             if file is not None and (data := file.load_flat_file_index()):
                 return HttpResponse(data, content_type="application/json")
             else:
-                return Http404
+                raise Http404
 
         if file is None:
             raise Http404

+ 148 - 141
src/sentry/debug_files/artifact_bundle_indexing.py

@@ -7,12 +7,10 @@ from datetime import datetime
 from typing import Any, Dict, List, NamedTuple, Optional, Tuple, TypeVar
 
 import sentry_sdk
-from django.db import router
+from django.db import DatabaseError, router
 from django.utils import timezone
 
 from sentry.debug_files.artifact_bundles import get_redis_cluster_for_artifact_bundles
-from sentry.debug_files.utils import size_in_bytes
-from sentry.locks import locks
 from sentry.models.artifactbundle import (
     NULL_STRING,
     ArtifactBundle,
@@ -23,8 +21,6 @@ from sentry.models.artifactbundle import (
 )
 from sentry.utils import json, metrics
 from sentry.utils.db import atomic_transaction
-from sentry.utils.locking.lock import Lock
-from sentry.utils.retries import TimedRetryPolicy
 
 logger = logging.getLogger(__name__)
 
@@ -39,6 +35,31 @@ class BundleMeta:
     id: int
     timestamp: datetime
 
+    @staticmethod
+    def from_artifact_bundle(artifact_bundle: ArtifactBundle) -> BundleMeta:
+        return BundleMeta(
+            id=artifact_bundle.id,
+            # We give priority to the date last modified for total ordering.
+            timestamp=(artifact_bundle.date_last_modified or artifact_bundle.date_uploaded),
+        )
+
+
+@dataclass(frozen=True)
+class BundleManifest:
+    meta: BundleMeta
+    urls: List[str]
+    debug_ids: List[str]
+
+    @staticmethod
+    def from_artifact_bundle(
+        artifact_bundle: ArtifactBundle, archive: ArtifactBundleArchive
+    ) -> BundleManifest:
+        meta = BundleMeta.from_artifact_bundle(artifact_bundle)
+        urls = archive.get_all_urls()
+        debug_ids = archive.get_all_debug_ids()
+
+        return BundleManifest(meta=meta, urls=urls, debug_ids=debug_ids)
+
 
 @dataclass(frozen=True)
 class FlatFileMeta:
@@ -64,77 +85,6 @@ class FlatFileMeta:
         return self.id == -1 and self.date == datetime.utcfromtimestamp(0)
 
 
-@sentry_sdk.tracing.trace
-def mark_bundle_for_flat_file_indexing(
-    artifact_bundle: ArtifactBundle,
-    project_ids: List[int],
-    release: Optional[str],
-    dist: Optional[str],
-):
-    identifiers = []
-
-    for project_id in project_ids:
-        if release:
-            identifiers.append(
-                FlatFileIdentifier(project_id, release=release, dist=dist or NULL_STRING)
-            )
-
-        identifiers.append(FlatFileIdentifier.for_debug_id(project_id))
-
-    # Create / Update the indexing state in the database
-    for identifier in identifiers:
-        # It turns out our DB integrity, and usage of `get_or_create` is not safe
-        # in the sense that it can end up with duplicates in the database, yay!
-        # So just wrap all of this in a lock so we are definitely not creating
-        # duplicated index entries concurrently.
-        lock = identifier.get_lock()
-        with TimedRetryPolicy(60)(lock.acquire), atomic_transaction(
-            using=(
-                router.db_for_write(ArtifactBundleFlatFileIndex),
-                router.db_for_write(FlatFileIndexState),
-            )
-        ):
-            # This used to be `get_or_create`, but that is completely broken
-            # when you end up with duplicates, so now we gotta clean that mess up:
-            flat_file_indexes = list(
-                ArtifactBundleFlatFileIndex.objects.filter(
-                    project_id=identifier.project_id,
-                    release_name=identifier.release,
-                    dist_name=identifier.dist,
-                )
-            )
-            if len(flat_file_indexes) > 0:
-                flat_file_index = flat_file_indexes.pop(0)
-                # remove duplicates from the DB:
-                if len(flat_file_indexes) > 0:
-                    ids = [index.id for index in flat_file_indexes]
-                    ArtifactBundleFlatFileIndex.objects.filter(id__in=ids).delete()
-            else:
-                flat_file_index = ArtifactBundleFlatFileIndex.objects.create(
-                    project_id=identifier.project_id,
-                    release_name=identifier.release,
-                    dist_name=identifier.dist,
-                )
-
-            # Lol, turns out that `update_or_create` will also do a `get` under the hood,
-            # which is equally broken if you end up with duplicates.
-            rows_updated = FlatFileIndexState.objects.filter(
-                flat_file_index=flat_file_index,
-                artifact_bundle=artifact_bundle,
-            ).update(
-                indexing_state=ArtifactBundleIndexingState.NOT_INDEXED.value,
-                date_added=timezone.now(),
-            )
-            if rows_updated == 0:
-                FlatFileIndexState.objects.create(
-                    flat_file_index=flat_file_index,
-                    artifact_bundle=artifact_bundle,
-                    indexing_state=ArtifactBundleIndexingState.NOT_INDEXED.value,
-                )
-
-    return identifiers
-
-
 class FlatFileIdentifier(NamedTuple):
     project_id: int
     release: str
@@ -155,12 +105,6 @@ class FlatFileIdentifier(NamedTuple):
     def _flat_file_meta_cache_key(self):
         return f"flat_file_index:{self._hashed()}"
 
-    def delete_flat_file_meta_from_cache(self):
-        cache_key = self._flat_file_meta_cache_key()
-        redis_client = get_redis_cluster_for_artifact_bundles()
-
-        redis_client.delete(cache_key)
-
     def set_flat_file_meta_in_cache(self, flat_file_meta: FlatFileMeta):
         cache_key = self._flat_file_meta_cache_key()
         redis_client = get_redis_cluster_for_artifact_bundles()
@@ -227,70 +171,146 @@ class FlatFileIdentifier(NamedTuple):
 
         return meta
 
-    def _locking_key(self):
-        return f"bundle_index:write:{self._hashed()}"
 
-    def get_lock(self) -> Lock:
-        locking_key = self._locking_key()
+@sentry_sdk.tracing.trace
+def mark_bundle_for_flat_file_indexing(
+    artifact_bundle: ArtifactBundle,
+    project_ids: List[int],
+    release: Optional[str],
+    dist: Optional[str],
+) -> List[FlatFileIdentifier]:
+    identifiers = []
+
+    for project_id in project_ids:
+        if release:
+            identifiers.append(
+                FlatFileIdentifier(project_id, release=release, dist=dist or NULL_STRING)
+            )
+
+        identifiers.append(FlatFileIdentifier.for_debug_id(project_id))
+
+    # Create / Update the indexing state in the database
+    for identifier in identifiers:
+        with atomic_transaction(
+            using=(
+                router.db_for_write(ArtifactBundleFlatFileIndex),
+                router.db_for_write(FlatFileIndexState),
+            )
+        ):
+            flat_file_index, _created = ArtifactBundleFlatFileIndex.objects.get_or_create(
+                project_id=identifier.project_id,
+                release_name=identifier.release,
+                dist_name=identifier.dist,
+            )
+            FlatFileIndexState.objects.update_or_create(
+                flat_file_index=flat_file_index,
+                artifact_bundle=artifact_bundle,
+                defaults={
+                    "indexing_state": ArtifactBundleIndexingState.NOT_INDEXED.value,
+                    "date_added": timezone.now(),
+                },
+            )
+
+    return identifiers
 
-        return locks.get(locking_key, duration=60 * 10, name="bundle_index")
+
+@sentry_sdk.tracing.trace
+def remove_artifact_bundle_from_indexes(artifact_bundle: ArtifactBundle):
+    flat_file_indexes = ArtifactBundleFlatFileIndex.objects.filter(
+        flatfileindexstate__artifact_bundle=artifact_bundle
+    )
+    for idx in flat_file_indexes:
+        identifier = FlatFileIdentifier(
+            project_id=idx.project_id, release=idx.release_name, dist=idx.dist_name
+        )
+
+        was_removed = update_artifact_bundle_index(
+            identifier, bundles_to_remove=[artifact_bundle.id]
+        )
+        if not was_removed:
+            metrics.incr("artifact_bundle_flat_file_indexing.removal.would_block")
+            # TODO: mark this for async removal
+            pass
 
 
 @sentry_sdk.tracing.trace
 def update_artifact_bundle_index(
-    bundle_meta: BundleMeta, bundle_archive: ArtifactBundleArchive, identifier: FlatFileIdentifier
-):
+    identifier: FlatFileIdentifier,
+    blocking: bool = False,
+    bundles_to_add: List[BundleManifest] | None = None,
+    bundles_to_remove: List[int] | None = None,
+) -> bool:
     """
-    This will merge the `ArtifactBundle` given via `bundle_meta` and `bundle_archive`
-    into the index identified via `identifier`.
+    This will update the index identified via `identifier`.
+    Multiple manifests given in `bundles_to_add` and `bundles_to_remove` will be merged
+    into the index as one batched operation.
 
-    If this function fails for any reason, it can be, and *has to be* retried at a later point.
+    If this function fails for any reason, it can be, and *has to be* retried at a later point,
+    as not doing so will leave inconsistent indexes around.
     """
-    # TODO: maybe query `FlatFileIndexState` to avoid double-indexing?
-
-    lock = identifier.get_lock()
-    with TimedRetryPolicy(60)(lock.acquire):
-        flat_file_index = ArtifactBundleFlatFileIndex.objects.filter(
-            project_id=identifier.project_id,
-            release_name=identifier.release,
-            dist_name=identifier.dist,
-        ).first()
+    with atomic_transaction(
+        using=(
+            router.db_for_write(ArtifactBundleFlatFileIndex),
+            router.db_for_write(FlatFileIndexState),
+        )
+    ):
+        # The `nowait=True` will opportunistically lock the row/index without blocking,
+        # and throw an error otherwise which we will pass down and handle in the caller.
+        try:
+            flat_file_index = (
+                ArtifactBundleFlatFileIndex.objects.filter(
+                    project_id=identifier.project_id,
+                    release_name=identifier.release,
+                    dist_name=identifier.dist,
+                )
+                .select_for_update(nowait=not blocking)
+                .first()
+            )
+        except DatabaseError:
+            return False
 
         index = FlatFileIndex()
         # Load the index from the file if it exists
         if existing_index := flat_file_index.load_flat_file_index():
             index.from_json(existing_index)
 
-        # Before merging new data into the index, we will clear any existing
-        # data from the index related to this bundle.
-        # This is related to an edge-case in which the same `bundle_id` could be
-        # re-used but with different file contents.
-        index.remove(bundle_meta.id)
+        for bundle in bundles_to_add or []:
+            # Before merging new data into the index, we will clear any existing
+            # data from the index related to this bundle.
+            # This is related to an edge-case in which the same `bundle_id` could be
+            # re-used but with different file contents.
+            index.remove(bundle.meta.id)
 
-        # We merge the index based on the identifier type.
-        if identifier.is_indexing_by_release():
-            index.merge_urls(bundle_meta, bundle_archive)
-        else:
-            index.merge_debug_ids(bundle_meta, bundle_archive)
+            # We merge the index based on the identifier type.
+            if identifier.is_indexing_by_release():
+                index.merge_urls(bundle.meta, bundle.urls)
+            else:
+                index.merge_debug_ids(bundle.meta, bundle.debug_ids)
+
+        for bundle_id in bundles_to_remove or []:
+            index.remove(bundle_id)
 
         # Store the updated index file
         new_json_index = index.to_json()
         flat_file_index.update_flat_file_index(new_json_index)
 
-        # And then mark the bundle as indexed
-        was_updated = FlatFileIndexState.compare_state_and_set(
-            flat_file_index.id,
-            bundle_meta.id,
-            ArtifactBundleIndexingState.NOT_INDEXED,
-            ArtifactBundleIndexingState.WAS_INDEXED,
-        )
-        if not was_updated:
-            metrics.incr("artifact_bundle_flat_file_indexing.duplicated_indexing")
-            logger.error("`ArtifactBundle` %r was already indexed into %r", bundle_meta, identifier)
+        # And then mark the bundles as indexed
+        for bundle in bundles_to_add or []:
+            was_updated = FlatFileIndexState.mark_as_indexed(
+                flat_file_index_id=flat_file_index.id, artifact_bundle_id=bundle.meta.id
+            )
+            if not was_updated:
+                metrics.incr("artifact_bundle_flat_file_indexing.duplicated_indexing")
+                logger.error(
+                    "`ArtifactBundle` %r was already indexed into %r", bundle.meta, identifier
+                )
 
-        # We invalidate the cache which is holding the FlatFileMeta for this specific identifier. This is done
+        # We update the cache which is holding the FlatFileMeta for this specific identifier. This is done
         # so that any upcoming event will load the new meta from the db and store it in cache.
-        identifier.delete_flat_file_meta_from_cache()
+        identifier.set_flat_file_meta_in_cache(
+            FlatFileMeta(id=flat_file_index.id, date=flat_file_index.date_added)
+        )
+        return True
 
 
 Bundles = List[BundleMeta]
@@ -340,35 +360,22 @@ class FlatFileIndex:
             "files_by_debug_id": self._files_by_debug_id,
         }
 
-        json_index = json.dumps(json_idx)
-
-        if len(self._files_by_url) == 0:
-            metrics.timing(
-                "artifact_bundle_flat_file_indexing.debug_id_index.size_in_bytes",
-                value=size_in_bytes(json_index),
-            )
-        else:
-            metrics.timing(
-                "artifact_bundle_flat_file_indexing.url_index.size_in_bytes",
-                value=size_in_bytes(json_index),
-            )
-
-        return json_index
+        return json.dumps(json_idx)
 
-    def merge_urls(self, bundle_meta: BundleMeta, bundle_archive: ArtifactBundleArchive):
+    def merge_urls(self, bundle_meta: BundleMeta, urls: List[str]):
         bundle_index = self._add_or_update_bundle(bundle_meta)
         if bundle_index is None:
             return
 
-        for url in bundle_archive.get_all_urls():
+        for url in urls:
             self._add_sorted_entry(self._files_by_url, url, bundle_index)
 
-    def merge_debug_ids(self, bundle_meta: BundleMeta, bundle_archive: ArtifactBundleArchive):
+    def merge_debug_ids(self, bundle_meta: BundleMeta, debug_ids: List[str]):
         bundle_index = self._add_or_update_bundle(bundle_meta)
         if bundle_index is None:
             return
 
-        for debug_id, _ in bundle_archive.get_all_debug_ids():
+        for debug_id in debug_ids:
             self._add_sorted_entry(self._files_by_debug_id, debug_id, bundle_index)
 
     def _add_or_update_bundle(self, bundle_meta: BundleMeta) -> Optional[int]:

+ 0 - 12
src/sentry/debug_files/utils.py

@@ -1,12 +0,0 @@
-import sentry_sdk
-
-
-def size_in_bytes(string: str) -> int:
-    """
-    Computes the size of a string in MB.
-    """
-    try:
-        return len(string.encode())
-    except Exception as e:
-        sentry_sdk.capture_exception(e)
-        return 0

+ 36 - 14
src/sentry/models/artifactbundle.py

@@ -4,9 +4,10 @@ import zipfile
 from enum import Enum
 from typing import IO, Callable, Dict, List, Mapping, Optional, Set, Tuple
 
+import sentry_sdk
 from django.conf import settings
 from django.db import models
-from django.db.models.signals import post_delete
+from django.db.models.signals import post_delete, pre_delete
 from django.utils import timezone
 from symbolic.debuginfo import normalize_debug_id
 from symbolic.exceptions import SymbolicError
@@ -19,7 +20,7 @@ from sentry.db.models import (
     region_silo_only_model,
 )
 from sentry.nodestore.base import NodeStorage
-from sentry.utils import json
+from sentry.utils import json, metrics
 from sentry.utils.hashlib import sha1_text
 from sentry.utils.services import LazyServiceWrapper
 
@@ -115,6 +116,16 @@ def delete_file_for_artifact_bundle(instance, **kwargs):
     instance.file.delete()
 
 
+def delete_bundle_from_index(instance, **kwargs):
+    from sentry.debug_files.artifact_bundle_indexing import remove_artifact_bundle_from_indexes
+
+    try:
+        remove_artifact_bundle_from_indexes(instance)
+    except Exception as e:
+        sentry_sdk.capture_exception(e)
+
+
+pre_delete.connect(delete_bundle_from_index, sender=ArtifactBundle)
 post_delete.connect(delete_file_for_artifact_bundle, sender=ArtifactBundle)
 
 indexstore = LazyServiceWrapper(
@@ -148,7 +159,15 @@ class ArtifactBundleFlatFileIndex(Model):
         return f"bundle_index:{self.project_id}:{self.id}"
 
     def update_flat_file_index(self, data: str):
-        indexstore.set_bytes(self._indexstore_id(), data.encode())
+        encoded_data = data.encode()
+
+        metric_name = "debug_id_index" if self.dist_name == NULL_STRING else "url_index"
+        metrics.timing(
+            f"artifact_bundle_flat_file_indexing.{metric_name}.size_in_bytes",
+            value=len(encoded_data),
+        )
+
+        indexstore.set_bytes(self._indexstore_id(), encoded_data)
         self.update(date_added=timezone.now())
 
     def load_flat_file_index(self) -> Optional[bytes]:
@@ -169,17 +188,17 @@ class FlatFileIndexState(Model):
         db_table = "sentry_flatfileindexstate"
 
     @staticmethod
-    def compare_state_and_set(
+    def mark_as_indexed(
         flat_file_index_id: int,
         artifact_bundle_id: int,
-        indexing_state: ArtifactBundleIndexingState,
-        new_indexing_state: ArtifactBundleIndexingState,
     ) -> bool:
         updated_rows = FlatFileIndexState.objects.filter(
             flat_file_index_id=flat_file_index_id,
             artifact_bundle_id=artifact_bundle_id,
-            indexing_state=indexing_state.value,
-        ).update(indexing_state=new_indexing_state.value, date_added=timezone.now())
+            indexing_state=ArtifactBundleIndexingState.NOT_INDEXED.value,
+        ).update(
+            indexing_state=ArtifactBundleIndexingState.WAS_INDEXED.value, date_added=timezone.now()
+        )
 
         # If we had one row being updated, it means that the cas operation succeeded.
         return updated_rows == 1
@@ -314,6 +333,9 @@ class ArtifactBundleArchive:
     def _build_memory_maps(self):
         files = self.manifest.get("files", {})
         for file_path, info in files.items():
+            url = info.get("url")
+            if not url:
+                continue
             # Building the map for debug_id lookup.
             headers = self.normalize_headers(info.get("headers", {}))
             if (debug_id := headers.get("debug-id")) is not None:
@@ -327,18 +349,18 @@ class ArtifactBundleArchive:
                 ):
                     self._entries_by_debug_id[(debug_id, source_file_type)] = (
                         file_path,
-                        info.get("url"),
+                        url,
                         info,
                     )
 
             # Building the map for url lookup.
-            self._entries_by_url[info.get("url")] = (file_path, info)
+            self._entries_by_url[url] = (file_path, info)
 
-    def get_all_urls(self):
-        return self._entries_by_url.keys()
+    def get_all_urls(self) -> List[str]:
+        return [url for url in self._entries_by_url.keys()]
 
-    def get_all_debug_ids(self):
-        return self._entries_by_debug_id.keys()
+    def get_all_debug_ids(self) -> List[str]:
+        return list({debug_id for debug_id, _ty in self._entries_by_debug_id.keys()})
 
     def has_debug_ids(self):
         return len(self._entries_by_debug_id) > 0

+ 9 - 7
src/sentry/tasks/assemble.py

@@ -17,7 +17,7 @@ from sentry import analytics, features, options
 from sentry.api.serializers import serialize
 from sentry.cache import default_cache
 from sentry.debug_files.artifact_bundle_indexing import (
-    BundleMeta,
+    BundleManifest,
     mark_bundle_for_flat_file_indexing,
     update_artifact_bundle_index,
 )
@@ -763,15 +763,17 @@ class ArtifactBundlePostAssembler(PostAssembler):
             artifact_bundle, self.project_ids, self.release, self.dist
         )
 
-        bundle_meta = BundleMeta(
-            id=artifact_bundle.id,
-            # We give priority to the date last modified for total ordering.
-            timestamp=(artifact_bundle.date_last_modified or artifact_bundle.date_uploaded),
-        )
+        bundles_to_add = [BundleManifest.from_artifact_bundle(artifact_bundle, self.archive)]
 
         for identifier in identifiers:
             try:
-                update_artifact_bundle_index(bundle_meta, self.archive, identifier)
+                was_indexed = update_artifact_bundle_index(
+                    identifier, bundles_to_add=bundles_to_add
+                )
+                if not was_indexed:
+                    metrics.incr("artifact_bundle_flat_file_indexing.indexing.would_block")
+                    # TODO: spawn an async task to backfill the indexing
+                    pass
             except Exception as e:
                 metrics.incr("artifact_bundle_flat_file_indexing.error_when_indexing")
                 sentry_sdk.capture_exception(e)

+ 116 - 65
tests/sentry/debug_files/test_artifact_bundle_indexing.py

@@ -9,6 +9,7 @@ from django.utils import timezone
 from freezegun import freeze_time
 
 from sentry.debug_files.artifact_bundle_indexing import (
+    BundleManifest,
     BundleMeta,
     FlatFileIndex,
     mark_bundle_for_flat_file_indexing,
@@ -124,14 +125,12 @@ class FlatFileIndexingTest(FlatFileTestCase):
         identifiers = mark_bundle_for_flat_file_indexing(
             artifact_bundle, [self.project.id], release, dist
         )
-        bundle_meta = BundleMeta(
-            id=artifact_bundle.id,
-            timestamp=artifact_bundle.date_last_modified,
-        )
 
         with ArtifactBundleArchive(artifact_bundle.file.getfile()) as archive:
-            for identifier in identifiers:
-                update_artifact_bundle_index(bundle_meta, archive, identifier)
+            bundles_to_add = [BundleManifest.from_artifact_bundle(artifact_bundle, archive)]
+
+        for identifier in identifiers:
+            update_artifact_bundle_index(identifier, bundles_to_add=bundles_to_add)
 
         assert ArtifactBundleFlatFileIndex.objects.get(
             project_id=self.project.id, release_name=release, dist_name=dist
@@ -143,14 +142,12 @@ class FlatFileIndexingTest(FlatFileTestCase):
         identifiers = mark_bundle_for_flat_file_indexing(
             artifact_bundle, [self.project.id], None, None
         )
-        bundle_meta = BundleMeta(
-            id=artifact_bundle.id,
-            timestamp=artifact_bundle.date_last_modified,
-        )
 
         with ArtifactBundleArchive(artifact_bundle.file.getfile()) as archive:
-            for identifier in identifiers:
-                update_artifact_bundle_index(bundle_meta, archive, identifier)
+            bundles_to_add = [BundleManifest.from_artifact_bundle(artifact_bundle, archive)]
+
+        for identifier in identifiers:
+            update_artifact_bundle_index(identifier, bundles_to_add=bundles_to_add)
 
         assert ArtifactBundleFlatFileIndex.objects.get(
             project_id=self.project.id, release_name="", dist_name=""
@@ -165,14 +162,12 @@ class FlatFileIndexingTest(FlatFileTestCase):
         identifiers = mark_bundle_for_flat_file_indexing(
             artifact_bundle, [self.project.id], release, dist
         )
-        bundle_meta = BundleMeta(
-            id=artifact_bundle.id,
-            timestamp=artifact_bundle.date_last_modified,
-        )
 
         with ArtifactBundleArchive(artifact_bundle.file.getfile()) as archive:
-            for identifier in identifiers:
-                update_artifact_bundle_index(bundle_meta, archive, identifier)
+            bundles_to_add = [BundleManifest.from_artifact_bundle(artifact_bundle, archive)]
+
+        for identifier in identifiers:
+            update_artifact_bundle_index(identifier, bundles_to_add=bundles_to_add)
 
         assert ArtifactBundleFlatFileIndex.objects.get(
             project_id=self.project.id, release_name=release, dist_name=dist
@@ -190,17 +185,15 @@ class FlatFileIndexingTest(FlatFileTestCase):
         identifiers = mark_bundle_for_flat_file_indexing(
             artifact_bundle, [self.project.id], None, None
         )
-        bundle_meta = BundleMeta(
-            id=artifact_bundle.id,
-            timestamp=artifact_bundle.date_last_modified,
-        )
 
         with ArtifactBundleArchive(artifact_bundle.file.getfile()) as archive:
-            for identifier in identifiers:
-                try:
-                    update_artifact_bundle_index(bundle_meta, archive, identifier)
-                except Exception:
-                    pass
+            bundles_to_add = [BundleManifest.from_artifact_bundle(artifact_bundle, archive)]
+
+        for identifier in identifiers:
+            try:
+                update_artifact_bundle_index(identifier, bundles_to_add=bundles_to_add)
+            except Exception:
+                pass
 
         index = ArtifactBundleFlatFileIndex.objects.get(
             project_id=self.project.id, release_name="", dist_name=""
@@ -215,18 +208,16 @@ class FlatFileIndexingTest(FlatFileTestCase):
         identifiers = mark_bundle_for_flat_file_indexing(
             artifact_bundle, [self.project.id], release, dist
         )
-        bundle_meta = BundleMeta(
-            id=artifact_bundle.id,
-            timestamp=artifact_bundle.date_last_modified,
-        )
 
         with ArtifactBundleArchive(artifact_bundle.file.getfile()) as archive:
-            for identifier in identifiers:
-                update_artifact_bundle_index(bundle_meta, archive, identifier)
+            bundles_to_add = [BundleManifest.from_artifact_bundle(artifact_bundle, archive)]
 
-            # We run the indexing twice, it should still succeed
-            for identifier in identifiers:
-                update_artifact_bundle_index(bundle_meta, archive, identifier)
+        for identifier in identifiers:
+            update_artifact_bundle_index(identifier, bundles_to_add=bundles_to_add)
+
+        # We run the indexing twice, it should still succeed
+        for identifier in identifiers:
+            update_artifact_bundle_index(identifier, bundles_to_add=bundles_to_add)
 
         assert ArtifactBundleFlatFileIndex.objects.filter(
             project_id=self.project.id, release_name=release, dist_name=dist
@@ -235,6 +226,56 @@ class FlatFileIndexingTest(FlatFileTestCase):
             project_id=self.project.id, release_name="", dist_name=""
         ).exists()
 
+    def test_remove_bundle_from_index(self):
+        release = "1.0"
+        dist = "android"
+        artifact_bundle1 = self.mock_simple_artifact_bundle()
+        artifact_bundle2 = self.mock_simple_artifact_bundle()
+
+        identifiers1 = mark_bundle_for_flat_file_indexing(
+            artifact_bundle1, [self.project.id], release, dist
+        )
+        identifiers2 = mark_bundle_for_flat_file_indexing(
+            artifact_bundle2, [self.project.id], release, dist
+        )
+        assert identifiers1 == identifiers2
+
+        bundles_to_add = []
+        with ArtifactBundleArchive(artifact_bundle1.file.getfile()) as archive:
+            bundles_to_add.append(BundleManifest.from_artifact_bundle(artifact_bundle1, archive))
+        with ArtifactBundleArchive(artifact_bundle2.file.getfile()) as archive:
+            bundles_to_add.append(BundleManifest.from_artifact_bundle(artifact_bundle2, archive))
+
+        for identifier in identifiers1:
+            update_artifact_bundle_index(identifier, bundles_to_add=bundles_to_add)
+
+        identifier = identifiers1[0]
+        index = ArtifactBundleFlatFileIndex.objects.get(
+            project_id=identifier.project_id,
+            release_name=identifier.release,
+            dist_name=identifier.dist,
+        )
+
+        json_index = json.loads(index.load_flat_file_index() or b"")
+        assert len(json_index["bundles"]) == 2
+        assert json_index["bundles"][0]["bundle_id"] == f"artifact_bundle/{artifact_bundle1.id}"
+        assert json_index["files_by_url"]["~/app.js"] == [0, 1]
+
+        # remove explicitly
+        update_artifact_bundle_index(identifier, bundles_to_remove=[artifact_bundle1.id])
+
+        json_index = json.loads(index.load_flat_file_index() or b"")
+        assert len(json_index["bundles"]) == 1
+        assert json_index["bundles"][0]["bundle_id"] == f"artifact_bundle/{artifact_bundle2.id}"
+        assert json_index["files_by_url"]["~/app.js"] == [0]
+
+        # remove via delete hook
+        artifact_bundle2.delete()
+
+        json_index = json.loads(index.load_flat_file_index() or b"")
+        assert len(json_index["bundles"]) == 0
+        assert json_index["files_by_url"] == {}
+
 
 @freeze_time("2023-07-13T10:00:00.000Z")
 class FlatFileIndexTest(FlatFileTestCase):
@@ -255,11 +296,13 @@ class FlatFileIndexTest(FlatFileTestCase):
         )
 
         with ArtifactBundleArchive(artifact_bundle.file.getfile()) as bundle_archive:
-            flat_file_index = FlatFileIndex()
-            bundle_meta = BundleMeta(
-                id=artifact_bundle.id, timestamp=artifact_bundle.date_last_modified
-            )
-            flat_file_index.merge_urls(bundle_meta, bundle_archive)
+            urls = bundle_archive.get_all_urls()
+
+        flat_file_index = FlatFileIndex()
+        bundle_meta = BundleMeta(
+            id=artifact_bundle.id, timestamp=artifact_bundle.date_last_modified
+        )
+        flat_file_index.merge_urls(bundle_meta, urls)
 
         assert json.loads(flat_file_index.to_json()) == {
             "bundles": [
@@ -291,11 +334,13 @@ class FlatFileIndexTest(FlatFileTestCase):
         )
 
         with ArtifactBundleArchive(artifact_bundle.file.getfile()) as bundle_archive:
-            flat_file_index = FlatFileIndex()
-            bundle_meta = BundleMeta(
-                id=artifact_bundle.id, timestamp=artifact_bundle.date_last_modified
-            )
-            flat_file_index.merge_debug_ids(bundle_meta, bundle_archive)
+            debug_ids = bundle_archive.get_all_debug_ids()
+
+        flat_file_index = FlatFileIndex()
+        bundle_meta = BundleMeta(
+            id=artifact_bundle.id, timestamp=artifact_bundle.date_last_modified
+        )
+        flat_file_index.merge_debug_ids(bundle_meta, debug_ids)
 
         assert json.loads(flat_file_index.to_json()) == {
             "bundles": [
@@ -340,12 +385,14 @@ class FlatFileIndexTest(FlatFileTestCase):
         )
 
         with ArtifactBundleArchive(artifact_bundle.file.getfile()) as bundle_archive:
-            flat_file_index = FlatFileIndex()
-            flat_file_index.from_json(json.dumps(existing_json_index))
-            bundle_meta = BundleMeta(
-                id=artifact_bundle.id, timestamp=artifact_bundle.date_last_modified
-            )
-            flat_file_index.merge_urls(bundle_meta, bundle_archive)
+            urls = bundle_archive.get_all_urls()
+
+        flat_file_index = FlatFileIndex()
+        flat_file_index.from_json(json.dumps(existing_json_index))
+        bundle_meta = BundleMeta(
+            id=artifact_bundle.id, timestamp=artifact_bundle.date_last_modified
+        )
+        flat_file_index.merge_urls(bundle_meta, urls)
 
         assert json.loads(flat_file_index.to_json()) == {
             "bundles": [
@@ -393,12 +440,14 @@ class FlatFileIndexTest(FlatFileTestCase):
         )
 
         with ArtifactBundleArchive(artifact_bundle.file.getfile()) as bundle_archive:
-            flat_file_index = FlatFileIndex()
-            flat_file_index.from_json(json.dumps(existing_json_index))
-            bundle_meta = BundleMeta(
-                id=artifact_bundle.id, timestamp=artifact_bundle.date_last_modified
-            )
-            flat_file_index.merge_debug_ids(bundle_meta, bundle_archive)
+            debug_ids = bundle_archive.get_all_debug_ids()
+
+        flat_file_index = FlatFileIndex()
+        flat_file_index.from_json(json.dumps(existing_json_index))
+        bundle_meta = BundleMeta(
+            id=artifact_bundle.id, timestamp=artifact_bundle.date_last_modified
+        )
+        flat_file_index.merge_debug_ids(bundle_meta, debug_ids)
 
         assert json.loads(flat_file_index.to_json()) == {
             "bundles": [
@@ -488,13 +537,15 @@ class FlatFileIndexTest(FlatFileTestCase):
         )
 
         with ArtifactBundleArchive(artifact_bundle.file.getfile()) as bundle_archive:
-            flat_file_index = FlatFileIndex()
-            flat_file_index.from_json(json.dumps(existing_json_index))
-            # We use the id of the existing bundle.
-            bundle_meta = BundleMeta(
-                id=existing_bundle_id, timestamp=artifact_bundle.date_last_modified
-            )
-            flat_file_index.merge_urls(bundle_meta, bundle_archive)
+            urls = bundle_archive.get_all_urls()
+
+        flat_file_index = FlatFileIndex()
+        flat_file_index.from_json(json.dumps(existing_json_index))
+        # We use the id of the existing bundle.
+        bundle_meta = BundleMeta(
+            id=existing_bundle_id, timestamp=artifact_bundle.date_last_modified
+        )
+        flat_file_index.merge_urls(bundle_meta, urls)
 
         assert json.loads(flat_file_index.to_json()) == {
             "bundles": [