Skip to content

Commit

Permalink
ducky: make raft_availability run faster
Browse files Browse the repository at this point in the history
Replaces ad hoc ping-pong check with ping-pong client. The client
maintains a connection so when we invoke ping-pong 128 times it
initializes a producer (init_producer_id) only once.

The change reduces test_follower_isolation execution time from 6
minutes to 30 seconds.
  • Loading branch information
rystsov committed May 6, 2022
1 parent 814398d commit de9194c
Showing 1 changed file with 23 additions and 24 deletions.
47 changes: 23 additions & 24 deletions tests/rptest/tests/raft_availability_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
from rptest.clients.kafka_cat import KafkaCat
from rptest.clients.rpk import RpkTool, RpkException
from rptest.clients.types import TopicSpec
from rptest.clients.ping_pong import PingPong
from rptest.services.failure_injector import FailureInjector, FailureSpec
from rptest.tests.redpanda_test import RedpandaTest
from rptest.services.rpk_producer import RpkProducer
Expand Down Expand Up @@ -95,41 +96,36 @@ def check():
assert result[0] is not None
return result[0]

def _ping_pong(self, timeout=5):
kc = KafkaCat(self.redpanda)
rpk = RpkTool(self.redpanda)
def ping_pong_cold(self):
ping_pong = PingPong.cold(self.redpanda.brokers_list(), self.topic, 0,
self.logger)
return ping_pong

payload = str(random.randint(0, 1000))
start = time.time()
offset = rpk.produce(self.topic, "tkey", payload, timeout=timeout)
consumed = kc.consume_one(self.topic, 0, offset)
latency = time.time() - start
self.logger.info(
f"_ping_pong produced '{payload}' consumed '{consumed}' in {(latency)*1000.0:.2f} ms"
)
if consumed['payload'] != payload:
raise RuntimeError(f"expected '{payload}' got '{consumed}'")
def ping_pong_warmed(self):
ping_pong = PingPong.warmed(self.redpanda.brokers_list(), self.topic,
0, self.logger, 20)
return ping_pong

def _is_available(self, timeout=5):
def _is_available(self, timeout_s=5):
try:
# Should fail
self._ping_pong(timeout)
except RpkException:
self.ping_pong_cold().ping_pong(timeout_s)
except:
return False
else:
return True

def _expect_unavailable(self):
try:
# Should fail
self._ping_pong()
except RpkException:
self.logger.info("Cluster is unavailable as expected")
self.ping_pong_cold().ping_pong()
except:
self.logger.exception("Cluster is unavailable as expected")
else:
assert False, "ping_pong should not have worked "

def _expect_available(self):
self._ping_pong()
self.ping_pong_cold().ping_pong()
self.logger.info("Cluster is available as expected")

def _transfer_leadership(self, admin: Admin, namespace: str, topic: str,
Expand Down Expand Up @@ -276,7 +272,7 @@ def test_two_nodes_down(self):
# Find which node is the leader
initial_leader_id, replicas = self._wait_for_leader()

self._ping_pong()
self.ping_pong_cold().ping_pong()

leader_node = self.redpanda.get_node(initial_leader_id)
other_node_id = (set(replicas) - {initial_leader_id}).pop()
Expand Down Expand Up @@ -474,8 +470,9 @@ def test_follower_isolation(self):
self.redpanda.get_node(follower)))

# expect messages to be produced and consumed without a timeout
connection = self.ping_pong_warmed()
for i in range(0, 128):
self._ping_pong(ELECTION_TIMEOUT * 6)
connection.ping_pong()

@cluster(num_nodes=3, log_allow_list=RESTART_LOG_ALLOW_LIST)
def test_id_allocator_leader_isolation(self):
Expand Down Expand Up @@ -505,8 +502,9 @@ def test_id_allocator_leader_isolation(self):
self.redpanda.get_node(initial_leader_id)))

# expect messages to be produced and consumed without a timeout
connection = self.ping_pong_warmed()
for i in range(0, 128):
self._ping_pong(ELECTION_TIMEOUT * 6)
connection.ping_pong()

@cluster(num_nodes=3)
def test_initial_leader_stability(self):
Expand Down Expand Up @@ -580,5 +578,6 @@ def controller_available():
hosts=hosts,
check=lambda node_id: node_id != controller_id)

connection = self.ping_pong_warmed()
for i in range(0, 128):
self._ping_pong()
connection.ping_pong()

0 comments on commit de9194c

Please sign in to comment.