Skip to content
Merged
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
Original file line number Diff line number Diff line change
Expand Up @@ -77,13 +77,13 @@
import static org.junit.jupiter.api.Assertions.assertTrue;

@Testcontainers
public class InklessClusterTest {
public class InklessLegacyClusterTest {
@Container
protected static InklessPostgreSQLContainer pgContainer = PostgreSQLTestContainer.container();
@Container
protected static MinioContainer s3Container = S3TestContainer.minio();

private static final Logger log = LoggerFactory.getLogger(InklessClusterTest.class);
private static final Logger log = LoggerFactory.getLogger(InklessLegacyClusterTest.class);

private KafkaClusterTestKit cluster;

Expand Down
343 changes: 343 additions & 0 deletions core/src/test/java/kafka/server/InklessManagedReplicasClusterTest.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,343 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.server;

import org.apache.kafka.clients.CommonClientConfigs;
import org.apache.kafka.clients.admin.Admin;
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.ConsumerRecords;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.clients.consumer.internals.AutoOffsetResetStrategy;
import org.apache.kafka.clients.producer.KafkaProducer;
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.serialization.ByteArrayDeserializer;
import org.apache.kafka.common.serialization.ByteArraySerializer;
import org.apache.kafka.common.test.KafkaClusterTestKit;
import org.apache.kafka.common.test.TestKitNodes;
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig;
import org.apache.kafka.server.config.ReplicationConfigs;
import org.apache.kafka.server.config.ServerConfigs;

import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.TestInfo;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.testcontainers.junit.jupiter.Container;
import org.testcontainers.junit.jupiter.Testcontainers;

import java.nio.charset.StandardCharsets;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.IntFunction;
import java.util.stream.Collectors;

import io.aiven.inkless.config.InklessConfig;
import io.aiven.inkless.control_plane.postgres.PostgresControlPlane;
import io.aiven.inkless.control_plane.postgres.PostgresControlPlaneConfig;
import io.aiven.inkless.storage_backend.s3.S3Storage;
import io.aiven.inkless.storage_backend.s3.S3StorageConfig;
import io.aiven.inkless.test_utils.InklessPostgreSQLContainer;
import io.aiven.inkless.test_utils.MinioContainer;
import io.aiven.inkless.test_utils.PostgreSQLTestContainer;
import io.aiven.inkless.test_utils.S3TestContainer;

import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;

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

private static final Logger log = LoggerFactory.getLogger(InklessManagedReplicasClusterTest.class);

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 0: combined broker+controller, Node 1: broker-only
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 with RF=2 (one replica per AZ)
.setConfigProp(ServerConfigs.DISKLESS_MANAGED_REPLICAS_ENABLE_CONFIG, "true")
.setConfigProp(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, "2")
// 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 with RF=-1 → resolves to default.replication.factor=2
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=2 from default.replication.factor
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(Node::id).toList(),
partition.isr().stream().map(Node::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 and consume records
final long now = System.currentTimeMillis();
produceRecords(clientConfigs, numRecords, i -> {
byte[] value = ("message-" + i).getBytes(StandardCharsets.UTF_8);
return new ProducerRecord<>(topicName, i % numPartitions, now, null, value);
});

List<TopicPartition> partitions = new ArrayList<>();
for (int i = 0; i < numPartitions; i++) {
partitions.add(new TopicPartition(topicName, i));
}
consumeWithAssign(clientConfigs, partitions, numRecords);
}

@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");

// Verify AZ-aware routing: client hints az1, so transformer should select node 0 (az1) as leader
assertEquals(0, partition.leader().id(),
"Leader should be the az1 broker (node 0) for a client with diskless_az=az1");
}

// Produce and consume records with AZ-aware client
produceRecords(clientConfigs, numRecords, i -> {
byte[] value = ("az-message-" + i).getBytes(StandardCharsets.UTF_8);
return new ProducerRecord<>(topicName, value);
});

consumeWithSubscribe(clientConfigs, topicName, numRecords);
}

@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(Node::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");
}
}
}

private void produceRecords(Map<String, Object> configs, int numRecords,
IntFunction<ProducerRecord<byte[], byte[]>> recordFactory) {
AtomicInteger recordsProduced = new AtomicInteger();
try (Producer<byte[], byte[]> producer = new KafkaProducer<>(configs)) {
for (int i = 0; i < numRecords; i++) {
producer.send(recordFactory.apply(i), (metadata, exception) -> {
if (exception != null) {
log.error("Failed to send record", exception);
} else {
recordsProduced.incrementAndGet();
}
});
}
producer.flush();
}
assertEquals(numRecords, recordsProduced.get());
}

private void consumeWithAssign(Map<String, Object> configs, List<TopicPartition> partitions, int numRecords) {
int recordsConsumed = 0;
try (Consumer<byte[], byte[]> consumer = new KafkaConsumer<>(configs)) {
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);
}

private void consumeWithSubscribe(Map<String, Object> configs, String topic, int numRecords) {
Map<String, Object> consumerConfigs = new HashMap<>(configs);
consumerConfigs.put(ConsumerConfig.GROUP_ID_CONFIG, UUID.randomUUID().toString());
int recordsConsumed = 0;
try (Consumer<byte[], byte[]> consumer = new KafkaConsumer<>(consumerConfigs)) {
consumer.subscribe(Collections.singletonList(topic));
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);
}
}
Loading