Skip to content

Commit

Permalink
Merge pull request #23916 from ztlpn/fix-CORE-3104
Browse files Browse the repository at this point in the history
Fix ghost node handling in raft heartbeats
  • Loading branch information
ztlpn authored Oct 28, 2024
2 parents 294705b + 1c23c35 commit 40e0dca
Show file tree
Hide file tree
Showing 4 changed files with 24 additions and 15 deletions.
4 changes: 2 additions & 2 deletions src/v/raft/consensus.cc
Original file line number Diff line number Diff line change
Expand Up @@ -3959,7 +3959,7 @@ reply_result consensus::lightweight_heartbeat(
target_node,
_self,
source_node);
return reply_result::failure;
return reply_result::group_unavailable;
}

/**
Expand Down Expand Up @@ -4014,7 +4014,7 @@ ss::future<full_heartbeat_reply> consensus::full_heartbeat(
target_vnode,
_self,
source_vnode);
reply.result = reply_result::failure;
reply.result = reply_result::group_unavailable;
co_return reply;
}
/**
Expand Down
11 changes: 11 additions & 0 deletions src/v/raft/heartbeat_manager.cc
Original file line number Diff line number Diff line change
Expand Up @@ -340,6 +340,17 @@ void heartbeat_manager::process_reply(
return;
}
auto& reply = r.value();

if (reply.source() != n) {
vlog(
raftlog.warn,
"got heartbeat reply from a different node id {} (expected {}), "
"ignoring",
reply.source(),
n);
return;
}

reply.for_each_lw_reply([this, n, target = reply.target(), &groups](
group_id group, reply_result result) {
auto it = _consensus_groups.find(group);
Expand Down
17 changes: 6 additions & 11 deletions tests/rptest/tests/admin_uuid_operations_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -295,14 +295,12 @@ def test_force_uuid_override(self, mode):
backoff_sec=2,
err_msg=f"{to_stop.name} did not take the UUID override")

self.logger.debug(f"Wait for the cluster to become healthy...")
self.logger.debug(f"Decommission ghost node [{ghost_node_id}]...")
self._decommission(ghost_node_id)

self.logger.debug(f"...and wait for the cluster to become healthy.")
self.wait_until_cluster_healthy(timeout_sec=30)

self.logger.debug(
f".. and decommission ghost node [{ghost_node_id}]...")
self._decommission(ghost_node_id)

self.logger.debug(
"Check that all this state sticks across a rolling restart")

Expand Down Expand Up @@ -393,14 +391,11 @@ def test_force_uuid_override_multinode(self, mode):
auto_assign_node_id=True,
)

self.logger.debug("Wait for the cluster to become healthy...")
self.logger.debug(f"Decommission ghost node [{ghost_node_id}]...")
self._decommission(ghost_node_id)

self.logger.debug("...and wait for the cluster to become healthy.")
controller_leader = self.wait_until_cluster_healthy(timeout_sec=30)

assert controller_leader is not None, "Didn't elect a controller leader"
assert controller_leader not in to_stop, f"Unexpected controller leader {controller_leader.account.hostname}"

self.logger.debug(
f"...and decommission ghost node [{ghost_node_id}]...")

self._decommission(ghost_node_id)
7 changes: 5 additions & 2 deletions tests/rptest/tests/license_enforcement_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@

from rptest.services.cluster import cluster
from rptest.clients.rpk import RpkTool
from rptest.services.admin import Admin
from rptest.services.redpanda import LoggingConfig
from rptest.tests.redpanda_test import RedpandaTest
from rptest.services.redpanda_installer import RedpandaInstaller
Expand Down Expand Up @@ -132,15 +133,17 @@ def test_escape_hatch_license_variable(self, clean_node_before_upgrade):
{"partition_autobalancing_mode": "continuous"})

first_upgraded = self.redpanda.nodes[0]
first_upgraded_id = self.redpanda.node_id(first_upgraded)
self.logger.info(
f"Upgrading node {first_upgraded} with an license injected via the environment variable escape hatch expecting it to succeed"
f"Upgrading node {first_upgraded.name} with an license injected via the environment variable escape hatch expecting it to succeed"
)
installer.install([first_upgraded], latest_version)
self.redpanda.stop_node(first_upgraded)

if clean_node_before_upgrade:
self.logger.info(f"Cleaning node {first_upgraded}")
self.logger.info(f"Cleaning node {first_upgraded.name}")
self.redpanda.remove_local_data(first_upgraded)
Admin(self.redpanda).decommission_broker(first_upgraded_id)

license = sample_license(assert_exists=True)
self.redpanda.set_environment(
Expand Down

0 comments on commit 40e0dca

Please sign in to comment.