Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

MV update may crash during bootstrap #18709

Closed
gleb-cloudius opened this issue May 16, 2024 · 4 comments
Closed

MV update may crash during bootstrap #18709

gleb-cloudius opened this issue May 16, 2024 · 4 comments

Comments

@gleb-cloudius
Copy link
Contributor

gleb-cloudius commented May 16, 2024

Reproducer:

diff --git a/test/topology_custom/test_mv_decommission.py b/test/topology_custom/test_mv_decommission.py
new file mode 100644
index 0000000000..caff758b71
--- /dev/null
+++ b/test/topology_custom/test_mv_decommission.py
@@ -0,0 +1,74 @@
+import asyncio
+import pytest
+import time
+import logging
+
+from cassandra.cluster import ConnectionException, ConsistencyLevel, NoHostAvailable, Session, SimpleStatement  # type: ignore
+
+from test.pylib.manager_client import ManagerClient
+from test.topology.conftest import skip_mode
+
+
+logger = logging.getLogger(__name__)
+
+
+@pytest.mark.asyncio
+@skip_mode('release', 'error injections are not supported in release mode')
+async def test_mv_decommission(manager: ManagerClient):
+    cfg = {'force_gossip_topology_changes': True}
+
+    servers = [await manager.server_add(config=cfg, timeout=60) for _ in range(3)]
+
+    cql = manager.get_cql()
+    await cql.run_async("CREATE KEYSPACE ks WITH replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 3};")
+    await cql.run_async("CREATE TABLE ks.t (pk int primary key, v int)")
+    await cql.run_async("CREATE materialized view ks.t_view AS select pk, v from ks.t where v is not null primary key (v, pk)")
+
+    stop_event = asyncio.Event()
+    concurrency = 10
+    async def do_writes() -> int:
+        iteration = 0
+        while not stop_event.is_set():
+            start_time = time.time()
+            try:
+                await cql.run_async(f"insert into ks.t (pk, v) values ({iteration}, {iteration})")
+            except NoHostAvailable as e:
+                for _, err in e.errors.items():
+                    # ConnectionException can be raised when the node is shutting down.
+                    if not isinstance(err, ConnectionException):
+                        logger.error(f"Write started {time.time() - start_time}s ago failed: {e}")
+                        raise
+            except Exception as e:
+                logger.error(f"Write started {time.time() - start_time}s ago failed: {e}")
+                raise
+            iteration += 1
+            await asyncio.sleep(0.01)
+
+        return iteration
+
+    tasks = [asyncio.create_task(do_writes()) for _ in range(concurrency)]
+
+    async def finish() -> list[int]:
+        logger.info("Stopping write workers")
+        stop_event.set()
+        nums = await asyncio.gather(*tasks)
+        return nums
+
+    await manager.server_add()
+    await manager.server_add()
+    await manager.server_add()
+
+    nums = await finish()
+
+    await asyncio.sleep(2)
+
+    logger.info(nums)
+    logger.info(sum(nums))
+
+    res = await cql.run_async("select pk, v from ks.t");
+    for r in res:
+        logger.info(f"pk: {r.pk}, v: {r.v}")
+
+    res = await cql.run_async("select v, pk from ks.t_view");
+    for r in res:
+        logger.info(f"v: {r.v}, pk: {r.pk}")
diff --git a/db/view/view.cc b/db/view/view.cc
index 8807a6748d..6a2d419ad8 100644
--- a/db/view/view.cc
+++ b/db/view/view.cc
@@ -1714,7 +1714,8 @@ future<> view_update_generator::mutate_MV(
         wait_for_all_updates wait_for_all)
 {
     auto base_ermp = base->table().get_effective_replication_map();
-    static constexpr size_t max_concurrent_updates = 128;
+    static constexpr size_t max_concurrent_updates = 2;
+    co_await seastar::sleep(std::chrono::milliseconds{1000});
     co_await max_concurrent_for_each(view_updates, max_concurrent_updates,
             [this, base_token, &stats, &cf_stats, tr_state, &pending_view_updates, allow_hints, wait_for_all, base_ermp] (frozen_mutation_and_schema mut) mutable -> future<> {
         auto view_token = dht::get_token(*mut.s, mut.fm.key());

The reason is that taking of a base table and view tables erm is not atomic (max_concurrent_for_each may preempt). The code in get_view_natural_endpoint though assumes that typologies are the same since it has code like:

            } else if (!network_topology || topology.get_datacenter(view_endpoint) == my_datacenter) {
                view_endpoints.push_back(view_endpoint);
            }

Where it looks up DC for view endpoint in base table topology. This may fail if view topology contain new node, while base table does not.

@gleb-cloudius
Copy link
Contributor Author

Similar issue: #17786

@kbr-scylla
Copy link
Contributor

kbr-scylla commented May 16, 2024

The reason is that taking of a base table and view tables erm is not atomic (max_concurrent_for_each may preempt). The code in get_view_natural_endpoint though assumes that typologies are the same since it has code like:

So it's a bug in MV code. But seems easy to fix, e.g. by ensuring we don't yield between taking the erms, or perhaps by taking the locations from the same erm? (locations for base endpoints from view erm?)

Anyway since it's MV code bug, reassigning to @nyh (although not sure, maybe it should be @piodul with the recent changes cc @kostja? Dunno -- please reassign if necessary)

Also, since this looks pretty serious -- it could potentially cause a full cluster crash when we have MV workload during bootstrap -- marking as release blocker

@mykaul
Copy link
Contributor

mykaul commented May 20, 2024

@kostja , @piodul - who's handling it?

@piodul
Copy link
Contributor

piodul commented May 20, 2024

@wmitros please take a look at it

wmitros added a commit to wmitros/scylla that referenced this issue May 22, 2024
When calculating the base-view mapping while the topology
is changing, we may encounter a situation where the base
table noticed the change in its effective replication map
while the view table hasn't, or vice-versa. This can happen
because the ERM update may be performed during the preemption
between taking the base ERM and view ERM, or, due to f2ff701,
the update may have just been performed partially when we are
taking the ERMs.

Until now, we assumed that the ERMs are synchronized while calling
finding the base-view endpoint mapping, so in particular, we were
using the topology from the base's ERM to check the datacenters of
all endpoints. Now that the ERMs are more likely to not be the same,
we may try to get the datacenter of a view endpoint that doesn't
exist in the base's topology, causing us to crash.

This is fixed in the following patch by using the view table's
topology for endpoints coming from the view ERM. The mapping
resulting from the call might now be a temporary mapping
between endpoints in different topologies, but it still maps
base and view replicas 1-to-1.

Fixes: scylladb#17786
Fixes: scylladb#18709
wmitros added a commit to wmitros/scylla that referenced this issue May 26, 2024
When calculating the base-view mapping while the topology
is changing, we may encounter a situation where the base
table noticed the change in its effective replication map
while the view table hasn't, or vice-versa. This can happen
because the ERM update may be performed during the preemption
between taking the base ERM and view ERM, or, due to f2ff701,
the update may have just been performed partially when we are
taking the ERMs.

Until now, we assumed that the ERMs are synchronized while calling
finding the base-view endpoint mapping, so in particular, we were
using the topology from the base's ERM to check the datacenters of
all endpoints. Now that the ERMs are more likely to not be the same,
we may try to get the datacenter of a view endpoint that doesn't
exist in the base's topology, causing us to crash.

This is fixed in this patch by using the view table's topology for
endpoints coming from the view ERM. The mapping resulting from the
call might now be a temporary mapping between endpoints in different
topologies, but it still maps base and view replicas 1-to-1.

Fixes: scylladb#17786
Fixes: scylladb#18709
wmitros added a commit to wmitros/scylla that referenced this issue May 27, 2024
When calculating the base-view mapping while the topology
is changing, we may encounter a situation where the base
table noticed the change in its effective replication map
while the view table hasn't, or vice-versa. This can happen
because the ERM update may be performed during the preemption
between taking the base ERM and view ERM, or, due to f2ff701,
the update may have just been performed partially when we are
taking the ERMs.

Until now, we assumed that the ERMs are synchronized while calling
finding the base-view endpoint mapping, so in particular, we were
using the topology from the base's ERM to check the datacenters of
all endpoints. Now that the ERMs are more likely to not be the same,
we may try to get the datacenter of a view endpoint that doesn't
exist in the base's topology, causing us to crash.

This is fixed in this patch by using the view table's topology for
endpoints coming from the view ERM. The mapping resulting from the
call might now be a temporary mapping between endpoints in different
topologies, but it still maps base and view replicas 1-to-1.

Fixes: scylladb#17786
Fixes: scylladb#18709
wmitros added a commit to wmitros/scylla that referenced this issue May 27, 2024
When calculating the base-view mapping while the topology
is changing, we may encounter a situation where the base
table noticed the change in its effective replication map
while the view table hasn't, or vice-versa. This can happen
because the ERM update may be performed during the preemption
between taking the base ERM and view ERM, or, due to f2ff701,
the update may have just been performed partially when we are
taking the ERMs.

Until now, we assumed that the ERMs are synchronized while calling
finding the base-view endpoint mapping, so in particular, we were
using the topology from the base's ERM to check the datacenters of
all endpoints. Now that the ERMs are more likely to not be the same,
we may try to get the datacenter of a view endpoint that doesn't
exist in the base's topology, causing us to crash.

This is fixed in this patch by using the view table's topology for
endpoints coming from the view ERM. The mapping resulting from the
call might now be a temporary mapping between endpoints in different
topologies, but it still maps base and view replicas 1-to-1.

Fixes: scylladb#17786
Fixes: scylladb#18709
mergify bot pushed a commit that referenced this issue May 28, 2024
When calculating the base-view mapping while the topology
is changing, we may encounter a situation where the base
table noticed the change in its effective replication map
while the view table hasn't, or vice-versa. This can happen
because the ERM update may be performed during the preemption
between taking the base ERM and view ERM, or, due to f2ff701,
the update may have just been performed partially when we are
taking the ERMs.

Until now, we assumed that the ERMs are synchronized while calling
finding the base-view endpoint mapping, so in particular, we were
using the topology from the base's ERM to check the datacenters of
all endpoints. Now that the ERMs are more likely to not be the same,
we may try to get the datacenter of a view endpoint that doesn't
exist in the base's topology, causing us to crash.

This is fixed in this patch by using the view table's topology for
endpoints coming from the view ERM. The mapping resulting from the
call might now be a temporary mapping between endpoints in different
topologies, but it still maps base and view replicas 1-to-1.

Fixes: #17786
Fixes: #18709
(cherry picked from commit ed95782)

# Conflicts:
#	db/view/view.cc
#	service/storage_service.cc
mergify bot pushed a commit that referenced this issue May 28, 2024
When calculating the base-view mapping while the topology
is changing, we may encounter a situation where the base
table noticed the change in its effective replication map
while the view table hasn't, or vice-versa. This can happen
because the ERM update may be performed during the preemption
between taking the base ERM and view ERM, or, due to f2ff701,
the update may have just been performed partially when we are
taking the ERMs.

Until now, we assumed that the ERMs are synchronized while calling
finding the base-view endpoint mapping, so in particular, we were
using the topology from the base's ERM to check the datacenters of
all endpoints. Now that the ERMs are more likely to not be the same,
we may try to get the datacenter of a view endpoint that doesn't
exist in the base's topology, causing us to crash.

This is fixed in this patch by using the view table's topology for
endpoints coming from the view ERM. The mapping resulting from the
call might now be a temporary mapping between endpoints in different
topologies, but it still maps base and view replicas 1-to-1.

Fixes: #17786
Fixes: #18709
(cherry picked from commit ed95782)

# Conflicts:
#	db/view/view.cc
#	service/storage_service.cc
mergify bot pushed a commit that referenced this issue May 28, 2024
When calculating the base-view mapping while the topology
is changing, we may encounter a situation where the base
table noticed the change in its effective replication map
while the view table hasn't, or vice-versa. This can happen
because the ERM update may be performed during the preemption
between taking the base ERM and view ERM, or, due to f2ff701,
the update may have just been performed partially when we are
taking the ERMs.

Until now, we assumed that the ERMs are synchronized while calling
finding the base-view endpoint mapping, so in particular, we were
using the topology from the base's ERM to check the datacenters of
all endpoints. Now that the ERMs are more likely to not be the same,
we may try to get the datacenter of a view endpoint that doesn't
exist in the base's topology, causing us to crash.

This is fixed in this patch by using the view table's topology for
endpoints coming from the view ERM. The mapping resulting from the
call might now be a temporary mapping between endpoints in different
topologies, but it still maps base and view replicas 1-to-1.

Fixes: #17786
Fixes: #18709
(cherry picked from commit ed95782)
wmitros added a commit to wmitros/scylla that referenced this issue May 29, 2024
When calculating the base-view mapping while the topology
is changing, we may encounter a situation where the base
table noticed the change in its effective replication map
while the view table hasn't, or vice-versa. This can happen
because the ERM update may be performed during the preemption
between taking the base ERM and view ERM, or, due to f2ff701,
the update may have just been performed partially when we are
taking the ERMs.

Until now, we assumed that the ERMs are synchronized while calling
finding the base-view endpoint mapping, so in particular, we were
using the topology from the base's ERM to check the datacenters of
all endpoints. Now that the ERMs are more likely to not be the same,
we may try to get the datacenter of a view endpoint that doesn't
exist in the base's topology, causing us to crash.

This is fixed in this patch by using the view table's topology for
endpoints coming from the view ERM. The mapping resulting from the
call might now be a temporary mapping between endpoints in different
topologies, but it still maps base and view replicas 1-to-1.

Fixes: scylladb#17786
Fixes: scylladb#18709

(cherry-picked from 519317d)

This commit also includes the follow-up patch that removes the
flakiness from the test that is introduced by the commit above.
The flakiness was caused by enabling the
delay_before_get_view_natural_endpoint injection on a node
and not disabling it before the node is shut down. The patch
removes the enabling of the injection on the node in the first
place.
By squashing the commits, we won't introduce a place in the
commit history where a potential bisect could mistakenly fail.

Fixes: scylladb#18941

(cherry-picked from 0de3a5f)
piodul pushed a commit that referenced this issue May 30, 2024
When calculating the base-view mapping while the topology
is changing, we may encounter a situation where the base
table noticed the change in its effective replication map
while the view table hasn't, or vice-versa. This can happen
because the ERM update may be performed during the preemption
between taking the base ERM and view ERM, or, due to f2ff701,
the update may have just been performed partially when we are
taking the ERMs.

Until now, we assumed that the ERMs are synchronized while calling
finding the base-view endpoint mapping, so in particular, we were
using the topology from the base's ERM to check the datacenters of
all endpoints. Now that the ERMs are more likely to not be the same,
we may try to get the datacenter of a view endpoint that doesn't
exist in the base's topology, causing us to crash.

This is fixed in this patch by using the view table's topology for
endpoints coming from the view ERM. The mapping resulting from the
call might now be a temporary mapping between endpoints in different
topologies, but it still maps base and view replicas 1-to-1.

Fixes: #17786
Fixes: #18709

(cherry-picked from 519317d)

This commit also includes the follow-up patch that removes the
flakiness from the test that is introduced by the commit above.
The flakiness was caused by enabling the
delay_before_get_view_natural_endpoint injection on a node
and not disabling it before the node is shut down. The patch
removes the enabling of the injection on the node in the first
place.
By squashing the commits, we won't introduce a place in the
commit history where a potential bisect could mistakenly fail.

Fixes: #18941

(cherry-picked from 0de3a5f)

Closes #18974
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging a pull request may close this issue.

8 participants