Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
37 commits
Select commit Hold shift + click to select a range
ed3050f
Increase segment.bytes configuration to 1MB in compaction and fetch a…
Ankith-Confluent Mar 9, 2026
fe06b56
Replace usleep with rd_usleep for better precision in session expiry …
Ankith-Confluent Mar 9, 2026
2280ad3
Update Kafka version to 4.2.0 in CI configurations and tests
Ankith-Confluent Mar 9, 2026
956927b
Refactor mock group member handling by removing common header and upd…
Ankith-Confluent Mar 9, 2026
d56e594
Add new mock tests for group heartbeat, fetch, and acknowledgment
Ankith-Confluent Mar 9, 2026
38439f6
Increase message size in compaction tests to exceed segment.bytes lim…
Ankith-Confluent Mar 9, 2026
04de4ed
Make TEST_SASL parameter required and add default value in CI configu…
Ankith-Confluent Mar 10, 2026
d307af8
Make TEST_SASL parameter required in CI configuration
Ankith-Confluent Mar 10, 2026
2378ca2
Add SASL test blocks for x86_64 and aarch64 architectures in CI confi…
Ankith-Confluent Mar 10, 2026
0d16299
Make TEST_SASL parameter optional in CI configuration
Ankith-Confluent Mar 10, 2026
2ebf2ea
Make TEST_SASL parameter optional in CI configuration
Ankith-Confluent Mar 10, 2026
2d8aa69
Refactor SASL test configuration and remove deprecated TEST_SASL para…
Ankith-Confluent Mar 10, 2026
94e079e
Refactor SASL test configuration: rename pipeline and add new promoti…
Ankith-Confluent Mar 11, 2026
06100fc
Remove SASL test configuration and related parameters from CI integra…
Ankith-Confluent Mar 12, 2026
04856a5
Refactor ShareFetch error handling and session validation in mock tests
Ankith-Confluent Mar 12, 2026
8932534
Test 155 fixes
Ankith-Confluent Mar 12, 2026
817b4a3
Add wait functions for assignment count and fatal error propagation i…
Ankith-Confluent Mar 12, 2026
26b5522
Enhance ShareFetch session validation to include Node ID for session …
Ankith-Confluent Mar 12, 2026
b31e289
Refactor tests to use SUB_TEST_QUICK for faster execution
Ankith-Confluent Mar 12, 2026
b3d1acd
Reduce wait times in share group tests for improved execution speed
Ankith-Confluent Mar 12, 2026
241fdf2
style fixes
Ankith-Confluent Mar 12, 2026
d8c7e3e
Add rdunittest_fetcher.c to the list of source files in CMakeLists.txt
Ankith-Confluent Mar 12, 2026
e2a57d3
Add share consumer tests to CI configuration and create test runner s…
Ankith-Confluent Mar 12, 2026
78df5c6
Increase test timeout for share group heartbeat mock and include new …
Ankith-Confluent Mar 12, 2026
fd32577
Refactor consumer leave rebalance test for clarity and efficiency
Ankith-Confluent Mar 12, 2026
500323f
Style fixes
Ankith-Confluent Mar 12, 2026
6802439
Reduce wait times in share group heartbeat tests and remove unused wa…
Ankith-Confluent Mar 13, 2026
9953507
Increase test timeout for share group heartbeat mock to accommodate l…
Ankith-Confluent Mar 13, 2026
c286fa8
tests: Replace SUB_TEST_QUICK with SUB_TEST in multiple test cases
Ankith-Confluent Mar 13, 2026
f0b9404
style fix
Ankith-Confluent Mar 13, 2026
f7b316e
tests: Initialize share_c1_initial and share_c2_initial to 0 in sessi…
Ankith-Confluent Mar 13, 2026
62a6637
tests: Update test flags for share group heartbeat and fetch mocks to…
Ankith-Confluent Mar 13, 2026
2aa0705
style: Fix spacing in test macro definition for consistency
Ankith-Confluent Mar 13, 2026
7a23d48
tests: Update test macros for consistency and improve readability
Ankith-Confluent Mar 13, 2026
965d65f
tests: Refactor subscription macros for improved readability and cons…
Ankith-Confluent Mar 13, 2026
7182cd8
tests: Add high volume and multi-topic tests for share consumer
Ankith-Confluent Mar 13, 2026
26dbf58
tests: Update share group mock tests to run manually until fixes are …
Ankith-Confluent Mar 13, 2026
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion .semaphore/run-all-tests.yml
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,7 @@ blocks:
value: "False"
commands:
- if [[ "$TEST_TYPE" != *"plaintext"* ]]; then exit 0; fi
- ./tests/run-all-tests.sh
- ./tests/run-all-tests.sh
- name: "SSL cluster (x86_64)"
env_vars:
- name: TEST_SSL
Expand Down
4 changes: 2 additions & 2 deletions .semaphore/semaphore-integration.yml
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ global_job_config:
- name: CI
value: 'true'
- name: KAFKA_VERSION
value: '4.0.0'
value: '4.2.0'
- name: CP_VERSION
value: '8.0.0'
# Shows plain output from docker build (no progress bars)
Expand Down Expand Up @@ -220,7 +220,7 @@ promotions:
env_vars:
- required: true
name: TEST_KAFKA_GIT_REF
default_value: 4.0.0
default_value: 4.2.0
- required: true
name: TEST_TYPE
default_value: plaintext,ssl
Expand Down
22 changes: 21 additions & 1 deletion .semaphore/semaphore.yml
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ global_job_config:
- name: CI
value: 'true'
- name: KAFKA_VERSION
value: '4.0.0'
value: '4.2.0'
- name: CP_VERSION
value: '8.0.0'
# Shows plain output from docker build (no progress bars)
Expand Down Expand Up @@ -139,6 +139,26 @@ blocks:
- packaging/tools/run-in-docker.sh test-runner-amd64-${CACHE_TAG} ./packaging/tools/build-configurations-checks.sh


- name: 'Linux Ubuntu amd64: share consumer tests'
dependencies: []
skip:
when: "tag =~ '^v[0-9]\\.'"
task:
agent:
machine:
type: s1-prod-ubuntu24-04-amd64-1
prologue:
commands:
- '[[ -z $DOCKERHUB_APIKEY ]] || docker login --username $DOCKERHUB_USER --password $DOCKERHUB_APIKEY'
jobs:
- name: 'Build and run share consumer tests (0170, 0171)'
commands:
- cache restore trivup-kafka-${KAFKA_VERSION}-${CACHE_TAG}
- packaging/tools/run-in-docker.sh test-runner-amd64-${CACHE_TAG}
./packaging/tools/run-share-consumer-tests.sh
${KAFKA_VERSION} ${CP_VERSION}
- cache store trivup-kafka-${KAFKA_VERSION}-${CACHE_TAG} tests/tmp-KafkaCluster/KafkaCluster/KafkaBrokerApp/kafka/${KAFKA_VERSION}

- name: 'Linux x64: release artifact docker builds'
dependencies: []
run:
Expand Down
19 changes: 19 additions & 0 deletions packaging/tools/run-share-consumer-tests.sh
Original file line number Diff line number Diff line change
@@ -0,0 +1,19 @@
#!/bin/bash
set -e

if [ "$#" -lt 2 ]; then
echo "Usage: $0 <kafka-version> <cp-version>"
echo "Example: $0 4.2.0 8.0.0"
exit 1
fi

KAFKA_VERSION=$1
CP_VERSION=$2

source /home/user/venv/bin/activate
./configure --install-deps --enable-werror --enable-devel
make -j all
make -j -C tests build
(cd tests && python3 -m trivup.clusters.KafkaCluster --kraft \
--version ${KAFKA_VERSION} \
--cpversion ${CP_VERSION} --cmd 'TESTS_SKIP_BEFORE=0170 ./run-test.sh')
1 change: 1 addition & 0 deletions src/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -72,6 +72,7 @@ set(
rdregex.c
rdstring.c
rdunittest.c
rdunittest_fetcher.c
rdvarint.c
rdmap.c
snappy.c
Expand Down
51 changes: 36 additions & 15 deletions src/rdkafka_mock_cgrp.c
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,6 @@
#include "rdkafka_int.h"
#include "rdbuf.h"
#include "rdkafka_mock_int.h"
#include "rdkafka_mock_group_common.h"


static const char *rd_kafka_mock_cgrp_classic_state_names[] = {
Expand Down Expand Up @@ -73,9 +72,10 @@ rd_kafka_mock_cgrp_classic_set_state(rd_kafka_mock_cgrp_classic_t *mcgrp,
void rd_kafka_mock_cgrp_classic_member_active(
rd_kafka_mock_cgrp_classic_t *mcgrp,
rd_kafka_mock_cgrp_classic_member_t *member) {
rd_kafka_mock_group_member_mark_active(mcgrp->cluster->rk, "classic",
member->id,
&member->ts_last_activity);
rd_kafka_dbg(mcgrp->cluster->rk, MOCK, "MOCK",
"Marking mock consumer group member %s as active",
member->id);
member->ts_last_activity = rd_clock();
}


Expand Down Expand Up @@ -537,8 +537,13 @@ static void rd_kafka_mock_cgrp_classic_member_destroy(
rd_kafka_mock_cgrp_classic_member_t *rd_kafka_mock_cgrp_classic_member_find(
const rd_kafka_mock_cgrp_classic_t *mcgrp,
const rd_kafkap_str_t *MemberId) {
return RD_KAFKA_MOCK_MEMBER_FIND(&mcgrp->members, MemberId,
rd_kafka_mock_cgrp_classic_member_t);
const rd_kafka_mock_cgrp_classic_member_t *member;
TAILQ_FOREACH(member, &mcgrp->members, link) {
if (!rd_kafkap_str_cmp_str(MemberId, member->id))
return (rd_kafka_mock_cgrp_classic_member_t *)member;
}

return NULL;
}


Expand Down Expand Up @@ -654,8 +659,13 @@ void rd_kafka_mock_cgrp_classic_destroy(rd_kafka_mock_cgrp_classic_t *mcgrp) {
rd_kafka_mock_cgrp_classic_t *
rd_kafka_mock_cgrp_classic_find(rd_kafka_mock_cluster_t *mcluster,
const rd_kafkap_str_t *GroupId) {
return RD_KAFKA_MOCK_GROUP_FIND(&mcluster->cgrps_classic, GroupId,
rd_kafka_mock_cgrp_classic_t);
rd_kafka_mock_cgrp_classic_t *mcgrp;
TAILQ_FOREACH(mcgrp, &mcluster->cgrps_classic, link) {
if (!rd_kafkap_str_cmp_str(GroupId, mcgrp->id))
return mcgrp;
}

return NULL;
}


Expand Down Expand Up @@ -1325,9 +1335,10 @@ rd_kafka_mock_cgrp_consumer_member_next_assignment(
void rd_kafka_mock_cgrp_consumer_member_active(
rd_kafka_mock_cgrp_consumer_t *mcgrp,
rd_kafka_mock_cgrp_consumer_member_t *member) {
rd_kafka_mock_group_member_mark_active(mcgrp->cluster->rk, "consumer",
member->id,
&member->ts_last_activity);
rd_kafka_dbg(mcgrp->cluster->rk, MOCK, "MOCK",
"Marking mock consumer group member %s as active",
member->id);
member->ts_last_activity = rd_clock();
}

/**
Expand All @@ -1342,8 +1353,13 @@ void rd_kafka_mock_cgrp_consumer_member_active(
rd_kafka_mock_cgrp_consumer_member_t *rd_kafka_mock_cgrp_consumer_member_find(
const rd_kafka_mock_cgrp_consumer_t *mcgrp,
const rd_kafkap_str_t *MemberId) {
return RD_KAFKA_MOCK_MEMBER_FIND(&mcgrp->members, MemberId,
rd_kafka_mock_cgrp_consumer_member_t);
const rd_kafka_mock_cgrp_consumer_member_t *member;
TAILQ_FOREACH(member, &mcgrp->members, link) {
if (!rd_kafkap_str_cmp_str(MemberId, member->id))
return (rd_kafka_mock_cgrp_consumer_member_t *)member;
}

return NULL;
}

/**
Expand Down Expand Up @@ -1684,8 +1700,13 @@ void rd_kafka_mock_cgrp_consumer_member_fenced(
rd_kafka_mock_cgrp_consumer_t *
rd_kafka_mock_cgrp_consumer_find(const rd_kafka_mock_cluster_t *mcluster,
const rd_kafkap_str_t *GroupId) {
return RD_KAFKA_MOCK_GROUP_FIND(&mcluster->cgrps_consumer, GroupId,
rd_kafka_mock_cgrp_consumer_t);
rd_kafka_mock_cgrp_consumer_t *mcgrp;
TAILQ_FOREACH(mcgrp, &mcluster->cgrps_consumer, link) {
if (!rd_kafkap_str_cmp_str(GroupId, mcgrp->id))
return mcgrp;
}

return NULL;
}

/**
Expand Down
98 changes: 0 additions & 98 deletions src/rdkafka_mock_group_common.h

This file was deleted.

Loading