Skip to content
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
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
266 changes: 266 additions & 0 deletions core/src/test/java/kafka/server/InklessClusterTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import org.apache.kafka.clients.admin.AdminClient;
import org.apache.kafka.clients.admin.CreateTopicsResult;
import org.apache.kafka.clients.admin.NewTopic;
import org.apache.kafka.clients.admin.TopicDescription;
import org.apache.kafka.clients.consumer.Consumer;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.ConsumerRecord;
Expand All @@ -32,7 +33,9 @@
import org.apache.kafka.clients.producer.Producer;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.TopicPartitionInfo;
import org.apache.kafka.common.config.TopicConfig;
import org.apache.kafka.common.record.TimestampType;
import org.apache.kafka.common.serialization.ByteArrayDeserializer;
Expand All @@ -44,6 +47,7 @@

import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Nested;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.TestInfo;
import org.junit.jupiter.params.ParameterizedTest;
Expand All @@ -59,8 +63,10 @@
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Collectors;
import java.util.stream.Stream;

import io.aiven.inkless.config.InklessConfig;
Expand Down Expand Up @@ -283,4 +289,264 @@ private static int poll(Consumer<byte[], byte[]> consumer, TimestampType timesta
}
return recordsConsumed;
}

/**
* Tests for diskless topics with managed replicas (RF > 1).
* Uses a 3-broker cluster with rack assignments to verify rack-aware placement.
*/
@Nested
@Testcontainers
class ManagedReplicasCluster {
@Container
protected static InklessPostgreSQLContainer pgContainer = PostgreSQLTestContainer.container();
@Container
protected static MinioContainer s3Container = S3TestContainer.minio();

private KafkaClusterTestKit cluster;

@BeforeEach
public void setup(final TestInfo testInfo) throws Exception {
s3Container.createBucket(testInfo);
pgContainer.createDatabase(testInfo);

// Create 2-broker cluster with rack assignments (matching docker compose setup)
// Node IDs: 0, 1 for combined broker+controller nodes
Map<Integer, Map<String, String>> perServerProps = Map.of(
0, Map.of(ServerConfigs.BROKER_RACK_CONFIG, "az1"),
1, Map.of(ServerConfigs.BROKER_RACK_CONFIG, "az2")
);
final TestKitNodes nodes = new TestKitNodes.Builder()
.setCombined(true)
.setNumBrokerNodes(2)
.setNumControllerNodes(1)
.setPerServerProperties(perServerProps)
.build();
cluster = new KafkaClusterTestKit.Builder(nodes)
.setConfigProp(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "1")
.setConfigProp(ServerConfigs.DISKLESS_STORAGE_SYSTEM_ENABLE_CONFIG, "true")
// Enable managed replicas
.setConfigProp(ServerConfigs.DISKLESS_MANAGED_REPLICAS_ENABLE_CONFIG, "true")
// PG control plane config
.setConfigProp(InklessConfig.PREFIX + InklessConfig.CONTROL_PLANE_CLASS_CONFIG, PostgresControlPlane.class.getName())
.setConfigProp(InklessConfig.PREFIX + InklessConfig.CONTROL_PLANE_PREFIX + PostgresControlPlaneConfig.CONNECTION_STRING_CONFIG, pgContainer.getJdbcUrl())
.setConfigProp(InklessConfig.PREFIX + InklessConfig.CONTROL_PLANE_PREFIX + PostgresControlPlaneConfig.USERNAME_CONFIG, PostgreSQLTestContainer.USERNAME)
.setConfigProp(InklessConfig.PREFIX + InklessConfig.CONTROL_PLANE_PREFIX + PostgresControlPlaneConfig.PASSWORD_CONFIG, PostgreSQLTestContainer.PASSWORD)
// S3 storage config
.setConfigProp(InklessConfig.PREFIX + InklessConfig.STORAGE_BACKEND_CLASS_CONFIG, S3Storage.class.getName())
.setConfigProp(InklessConfig.PREFIX + InklessConfig.STORAGE_PREFIX + S3StorageConfig.S3_BUCKET_NAME_CONFIG, s3Container.getBucketName())
.setConfigProp(InklessConfig.PREFIX + InklessConfig.STORAGE_PREFIX + S3StorageConfig.S3_REGION_CONFIG, s3Container.getRegion())
.setConfigProp(InklessConfig.PREFIX + InklessConfig.STORAGE_PREFIX + S3StorageConfig.S3_ENDPOINT_URL_CONFIG, s3Container.getEndpoint())
.setConfigProp(InklessConfig.PREFIX + InklessConfig.STORAGE_PREFIX + S3StorageConfig.S3_PATH_STYLE_ENABLED_CONFIG, "true")
.setConfigProp(InklessConfig.PREFIX + InklessConfig.STORAGE_PREFIX + S3StorageConfig.AWS_ACCESS_KEY_ID_CONFIG, s3Container.getAccessKey())
.setConfigProp(InklessConfig.PREFIX + InklessConfig.STORAGE_PREFIX + S3StorageConfig.AWS_SECRET_ACCESS_KEY_CONFIG, s3Container.getSecretKey())
.build();
cluster.format();
cluster.startup();
cluster.waitForReadyBrokers();
}

@AfterEach
public void teardown() throws Exception {
cluster.close();
}

@Test
public void createDisklessTopicWithManagedReplicas() throws Exception {
Map<String, Object> clientConfigs = new HashMap<>();
clientConfigs.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers());
clientConfigs.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
clientConfigs.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
clientConfigs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
clientConfigs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
clientConfigs.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, AutoOffsetResetStrategy.EARLIEST.name());

String topicName = "managed-replicas-topic";
int numPartitions = 3;
int numRecords = 100;

// Create diskless topic - should get RF=2 (one per rack) with managed replicas enabled
try (Admin admin = AdminClient.create(clientConfigs)) {
final NewTopic topic = new NewTopic(topicName, numPartitions, (short) -1)
.configs(Map.of(TopicConfig.DISKLESS_ENABLE_CONFIG, "true"));
CreateTopicsResult result = admin.createTopics(Collections.singletonList(topic));
result.all().get(30, TimeUnit.SECONDS);

// Verify managed replicas behavior: RF = rack_count
TopicDescription description = admin.describeTopics(Collections.singletonList(topicName))
.allTopicNames().get(30, TimeUnit.SECONDS).get(topicName);
log.info("Topic {} created with {} partitions", topicName, description.partitions().size());
assertEquals(numPartitions, description.partitions().size());

// Node 0 is in az1, Node 1 is in az2
Set<Integer> expectedBrokers = Set.of(0, 1);

for (TopicPartitionInfo partition : description.partitions()) {
log.info("Partition {}: leader={}, replicas={}, isr={}",
partition.partition(), partition.leader().id(),
partition.replicas().stream().map(n -> n.id()).toList(),
partition.isr().stream().map(n -> n.id()).toList());

// With 2 racks, RF should be 2
assertEquals(2, partition.replicas().size(),
"Expected RF=2 for managed replicas with 2 racks");

// Verify replicas contain brokers from both racks (nodes 0 and 1)
Set<Integer> replicaIds = partition.replicas().stream()
.map(Node::id)
.collect(Collectors.toSet());
assertEquals(expectedBrokers, replicaIds,
"Replicas should include one broker from each rack (az1=node0, az2=node1)");

// Verify all replicas are in ISR initially
Set<Integer> isrIds = partition.isr().stream()
.map(Node::id)
.collect(Collectors.toSet());
assertEquals(replicaIds, isrIds,
"All replicas should be in ISR initially");

// Verify leader is one of the replicas
assertTrue(replicaIds.contains(partition.leader().id()),
"Leader should be one of the replicas");
}
}

// Produce records
AtomicInteger recordsProduced = new AtomicInteger();
final long now = System.currentTimeMillis();
try (Producer<byte[], byte[]> producer = new KafkaProducer<>(clientConfigs)) {
for (int i = 0; i < numRecords; i++) {
byte[] value = ("message-" + i).getBytes();
// Distribute across partitions
int partition = i % numPartitions;
final ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(topicName, partition, now, null, value);
producer.send(record, (metadata, exception) -> {
if (exception != null) {
log.error("Failed to send record", exception);
} else {
recordsProduced.incrementAndGet();
}
});
}
producer.flush();
}
assertEquals(numRecords, recordsProduced.get());

// Consume records
int recordsConsumed = 0;
try (Consumer<byte[], byte[]> consumer = new KafkaConsumer<>(clientConfigs)) {
List<TopicPartition> partitions = new java.util.ArrayList<>();
for (int i = 0; i < numPartitions; i++) {
partitions.add(new TopicPartition(topicName, i));
}
consumer.assign(partitions);
for (int i = 0; i < 10; i++) {
ConsumerRecords<byte[], byte[]> poll = consumer.poll(Duration.ofSeconds(5));
recordsConsumed += poll.count();
if (recordsConsumed >= numRecords) break;
}
}
assertEquals(numRecords, recordsConsumed);
}

@Test
public void produceAndConsumeWithClientAZ() throws Exception {
Map<String, Object> clientConfigs = new HashMap<>();
clientConfigs.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers());
clientConfigs.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
clientConfigs.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
clientConfigs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
clientConfigs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
clientConfigs.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, AutoOffsetResetStrategy.EARLIEST.name());
// Set client AZ via client.id prefix
clientConfigs.put(CommonClientConfigs.CLIENT_ID_CONFIG, "diskless_az=az1");

String topicName = "az-aware-topic";
int numRecords = 50;

// Create diskless topic with managed replicas
try (Admin admin = AdminClient.create(clientConfigs)) {
final NewTopic topic = new NewTopic(topicName, 1, (short) -1)
.configs(Map.of(TopicConfig.DISKLESS_ENABLE_CONFIG, "true"));
CreateTopicsResult result = admin.createTopics(Collections.singletonList(topic));
result.all().get(30, TimeUnit.SECONDS);

// Verify managed replicas: RF=2 with replicas on both brokers
TopicDescription description = admin.describeTopics(Collections.singletonList(topicName))
.allTopicNames().get(30, TimeUnit.SECONDS).get(topicName);
TopicPartitionInfo partition = description.partitions().get(0);
assertEquals(2, partition.replicas().size(),
"Expected RF=2 for managed replicas with 2 racks");
Set<Integer> replicaIds = partition.replicas().stream()
.map(Node::id)
.collect(Collectors.toSet());
assertEquals(Set.of(0, 1), replicaIds,
"Replicas should include one broker from each rack");
}

// Produce records with AZ-aware client (client hints az1)
AtomicInteger recordsProduced = new AtomicInteger();
try (Producer<byte[], byte[]> producer = new KafkaProducer<>(clientConfigs)) {
for (int i = 0; i < numRecords; i++) {
byte[] value = ("az-message-" + i).getBytes();
final ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(topicName, value);
producer.send(record, (metadata, exception) -> {
if (exception != null) {
log.error("Failed to send record", exception);
} else {
log.info("Produced to partition {} at offset {}", metadata.partition(), metadata.offset());
recordsProduced.incrementAndGet();
}
});
}
producer.flush();
}
assertEquals(numRecords, recordsProduced.get());

// Consume records with AZ-aware client (same az1 hint)
Map<String, Object> consumerConfigs = new HashMap<>(clientConfigs);
consumerConfigs.put(ConsumerConfig.GROUP_ID_CONFIG, java.util.UUID.randomUUID().toString());
int recordsConsumed = 0;
try (Consumer<byte[], byte[]> consumer = new KafkaConsumer<>(consumerConfigs)) {
consumer.subscribe(Collections.singletonList(topicName));
for (int i = 0; i < 10; i++) {
ConsumerRecords<byte[], byte[]> poll = consumer.poll(Duration.ofSeconds(5));
recordsConsumed += poll.count();
if (recordsConsumed >= numRecords) break;
}
}
assertEquals(numRecords, recordsConsumed);
}

@Test
public void nonDisklessTopicDoesNotGetManagedReplicas() throws Exception {
Map<String, Object> clientConfigs = new HashMap<>();
clientConfigs.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers());

String topicName = "regular-topic";
short explicitRF = 1;

// Create regular (non-diskless) topic with explicit RF=1
try (Admin admin = AdminClient.create(clientConfigs)) {
final NewTopic topic = new NewTopic(topicName, 2, explicitRF);
// Explicitly NOT setting diskless.enable=true
CreateTopicsResult result = admin.createTopics(Collections.singletonList(topic));
result.all().get(30, TimeUnit.SECONDS);

// Verify the topic was created with the explicit RF=1, not managed replicas RF=2
TopicDescription description = admin.describeTopics(Collections.singletonList(topicName))
.allTopicNames().get(30, TimeUnit.SECONDS).get(topicName);
log.info("Regular topic {} created", topicName);

for (TopicPartitionInfo partition : description.partitions()) {
log.info("Partition {}: leader={}, replicas={}",
partition.partition(), partition.leader().id(),
partition.replicas().stream().map(n -> n.id()).toList());

// Non-diskless topic should keep the explicit RF=1
assertEquals(1, partition.replicas().size(),
"Non-diskless topic should use explicit RF, not managed replicas");
}
}
}
}
}
6 changes: 6 additions & 0 deletions docker/examples/docker-compose-files/inkless/Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,12 @@ s3-aws:
KAFKA_VERSION=$(KAFKA_VERSION) $(DOCKER) compose -f docker-compose.yml -f docker-compose.monitoring.yml -f docker-compose.demo.yml -f docker-compose.s3-aws.yml up
$(MAKE) destroy

# Multi-AZ cluster with managed replicas (RF=rack_count)
.PHONY: managed-replicas
managed-replicas:
KAFKA_VERSION=$(KAFKA_VERSION) $(DOCKER) compose -f docker-compose.yml -f docker-compose.s3-local.yml -f docker-compose.managed-replicas.yml up
$(MAKE) destroy

.PHONY: destroy
destroy:
$(DOCKER) compose down --remove-orphans
Expand Down
Loading