Browse Source

KAFKA-15167: Tiered Storage Test Harness Framework (#14116)

`TieredStorageTestHarness` is a base class for integration tests exercising the tiered storage functionality. This uses  `LocalTieredStorage` instance as the second-tier storage system and `TopicBasedRemoteLogMetadataManager` as the remote log metadata manager.

Co-authored-by: Alexandre Dupriez <alexandre.dupriez@gmail.com>
Co-authored-by: Kamal Chandraprakash <kamal.chandraprakash@gmail.com>
pull/14268/head
Kamal Chandraprakash 1 year ago committed by GitHub
parent
commit
6492164d9c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 9
      build.gradle
  2. 132
      checkstyle/import-control-storage.xml
  3. 29
      checkstyle/import-control.xml
  4. 2
      core/src/main/scala/kafka/server/KafkaBroker.scala
  5. 9
      core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala
  6. 4
      storage/src/test/java/org/apache/kafka/server/log/remote/storage/LocalTieredStorageTraverser.java
  7. 11
      storage/src/test/java/org/apache/kafka/tiered/storage/README.md
  8. 36
      storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestAction.java
  9. 395
      storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestBuilder.java
  10. 315
      storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestContext.java
  11. 219
      storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestHarness.java
  12. 72
      storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestReport.java
  13. 41
      storage/src/test/java/org/apache/kafka/tiered/storage/actions/BounceBrokerAction.java
  14. 143
      storage/src/test/java/org/apache/kafka/tiered/storage/actions/ConsumeAction.java
  15. 43
      storage/src/test/java/org/apache/kafka/tiered/storage/actions/CreatePartitionsAction.java
  16. 61
      storage/src/test/java/org/apache/kafka/tiered/storage/actions/CreateTopicAction.java
  17. 51
      storage/src/test/java/org/apache/kafka/tiered/storage/actions/DeleteRecordsAction.java
  18. 86
      storage/src/test/java/org/apache/kafka/tiered/storage/actions/DeleteTopicAction.java
  19. 42
      storage/src/test/java/org/apache/kafka/tiered/storage/actions/EraseBrokerStorageAction.java
  20. 58
      storage/src/test/java/org/apache/kafka/tiered/storage/actions/ExpectBrokerInISRAction.java
  21. 48
      storage/src/test/java/org/apache/kafka/tiered/storage/actions/ExpectEmptyRemoteStorageAction.java
  22. 109
      storage/src/test/java/org/apache/kafka/tiered/storage/actions/ExpectLeaderAction.java
  23. 74
      storage/src/test/java/org/apache/kafka/tiered/storage/actions/ExpectLeaderEpochCheckpointAction.java
  24. 66
      storage/src/test/java/org/apache/kafka/tiered/storage/actions/ExpectListOffsetsAction.java
  25. 61
      storage/src/test/java/org/apache/kafka/tiered/storage/actions/ExpectTopicIdToMatchInRemoteStorageAction.java
  26. 74
      storage/src/test/java/org/apache/kafka/tiered/storage/actions/ExpectUserTopicMappedToMetadataPartitionsAction.java
  27. 162
      storage/src/test/java/org/apache/kafka/tiered/storage/actions/ProduceAction.java
  28. 78
      storage/src/test/java/org/apache/kafka/tiered/storage/actions/ReassignReplicaAction.java
  29. 83
      storage/src/test/java/org/apache/kafka/tiered/storage/actions/ShrinkReplicaAction.java
  30. 41
      storage/src/test/java/org/apache/kafka/tiered/storage/actions/StartBrokerAction.java
  31. 41
      storage/src/test/java/org/apache/kafka/tiered/storage/actions/StopBrokerAction.java
  32. 53
      storage/src/test/java/org/apache/kafka/tiered/storage/actions/UpdateBrokerConfigAction.java
  33. 52
      storage/src/test/java/org/apache/kafka/tiered/storage/actions/UpdateTopicConfigAction.java
  34. 136
      storage/src/test/java/org/apache/kafka/tiered/storage/integration/OffloadAndConsumeFromLeaderTest.java
  35. 70
      storage/src/test/java/org/apache/kafka/tiered/storage/specs/ConsumableSpec.java
  36. 72
      storage/src/test/java/org/apache/kafka/tiered/storage/specs/DeletableSpec.java
  37. 74
      storage/src/test/java/org/apache/kafka/tiered/storage/specs/ExpandPartitionCountSpec.java
  38. 60
      storage/src/test/java/org/apache/kafka/tiered/storage/specs/FetchableSpec.java
  39. 75
      storage/src/test/java/org/apache/kafka/tiered/storage/specs/KeyValueSpec.java
  40. 73
      storage/src/test/java/org/apache/kafka/tiered/storage/specs/OffloadableSpec.java
  41. 87
      storage/src/test/java/org/apache/kafka/tiered/storage/specs/OffloadedSegmentSpec.java
  42. 81
      storage/src/test/java/org/apache/kafka/tiered/storage/specs/ProducableSpec.java
  43. 89
      storage/src/test/java/org/apache/kafka/tiered/storage/specs/RemoteDeleteSegmentSpec.java
  44. 77
      storage/src/test/java/org/apache/kafka/tiered/storage/specs/RemoteFetchSpec.java
  45. 106
      storage/src/test/java/org/apache/kafka/tiered/storage/specs/TopicSpec.java
  46. 72
      storage/src/test/java/org/apache/kafka/tiered/storage/utils/ActionUtils.java
  47. 162
      storage/src/test/java/org/apache/kafka/tiered/storage/utils/BrokerLocalStorage.java
  48. 29
      storage/src/test/java/org/apache/kafka/tiered/storage/utils/DumpLocalTieredStorage.java
  49. 121
      storage/src/test/java/org/apache/kafka/tiered/storage/utils/LocalTieredStorageOutput.java
  50. 180
      storage/src/test/java/org/apache/kafka/tiered/storage/utils/RecordsKeyValueMatcher.java
  51. 6
      storage/src/test/resources/log4j.properties

9
build.gradle

@ -1695,6 +1695,10 @@ project(':storage:api') { @@ -1695,6 +1695,10 @@ project(':storage:api') {
javadoc {
include "**/org/apache/kafka/server/log/remote/storage/*"
}
checkstyle {
configProperties = checkstyleConfigProperties("import-control-storage.xml")
}
}
project(':storage') {
@ -1719,6 +1723,7 @@ project(':storage') { @@ -1719,6 +1723,7 @@ project(':storage') {
testImplementation project(':core').sourceSets.test.output
testImplementation project(':server-common')
testImplementation project(':server-common').sourceSets.test.output
testImplementation libs.hamcrest
testImplementation libs.junitJupiter
testImplementation libs.mockitoCore
testImplementation libs.bcpkix
@ -1797,6 +1802,10 @@ project(':storage') { @@ -1797,6 +1802,10 @@ project(':storage') {
javadoc {
enabled = false
}
checkstyle {
configProperties = checkstyleConfigProperties("import-control-storage.xml")
}
}
project(':tools:tools-api') {

132
checkstyle/import-control-storage.xml

@ -0,0 +1,132 @@ @@ -0,0 +1,132 @@
<!DOCTYPE import-control PUBLIC
"-//Puppy Crawl//DTD Import Control 1.1//EN"
"http://www.puppycrawl.com/dtds/import_control_1_1.dtd">
<!--
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.
-->
<import-control pkg="org.apache.kafka">
<!-- THINK HARD ABOUT THE LAYERING OF THE PROJECT BEFORE CHANGING THIS FILE -->
<!-- common library dependencies -->
<allow pkg="java" />
<allow pkg="javax.management" />
<allow pkg="org.slf4j" />
<allow pkg="org.junit" />
<allow pkg="org.opentest4j" />
<allow pkg="org.hamcrest" />
<allow pkg="org.mockito" />
<allow pkg="org.easymock" />
<allow pkg="org.powermock" />
<allow pkg="java.security" />
<allow pkg="javax.net.ssl" />
<allow pkg="javax.security" />
<allow pkg="org.ietf.jgss" />
<allow pkg="net.jqwik.api" />
<!-- no one depends on the server -->
<disallow pkg="kafka" />
<!-- anyone can use public classes -->
<allow pkg="org.apache.kafka.common" exact-match="true" />
<allow pkg="org.apache.kafka.common.security" />
<allow pkg="org.apache.kafka.common.serialization" />
<allow pkg="org.apache.kafka.common.utils" />
<allow pkg="org.apache.kafka.common.errors" exact-match="true" />
<allow pkg="org.apache.kafka.common.memory" />
<subpackage name="server">
<allow pkg="org.apache.kafka.common" />
<subpackage name="log">
<allow pkg="com.fasterxml.jackson" />
<allow pkg="kafka.api" />
<allow pkg="kafka.utils" />
<allow pkg="org.apache.kafka.clients" />
<allow pkg="org.apache.kafka.server.common" />
<allow pkg="org.apache.kafka.server.config" />
<allow pkg="org.apache.kafka.server.log" />
<allow pkg="org.apache.kafka.server.record" />
<allow pkg="org.apache.kafka.test" />
<allow pkg="org.apache.kafka.storage"/>
<subpackage name="remote">
<allow pkg="scala.collection" />
<subpackage name="storage">
<allow pkg="com.yammer.metrics.core" />
<allow pkg="org.apache.kafka.server.metrics" />
</subpackage>
</subpackage>
</subpackage>
</subpackage>
<subpackage name="storage.internals">
<allow pkg="com.yammer.metrics.core" />
<allow pkg="org.apache.kafka.server"/>
<allow pkg="org.apache.kafka.storage.internals"/>
<allow pkg="org.apache.kafka.common" />
<allow pkg="com.github.benmanes.caffeine.cache" />
</subpackage>
<!-- START OF TIERED STORAGE INTEGRATION TEST IMPORT DEPENDENCIES -->
<subpackage name="tiered.storage">
<allow pkg="scala" />
<allow pkg="org.apache.kafka.tiered.storage" />
<allow pkg="org.apache.kafka.tiered.storage.actions" />
<allow pkg="org.apache.kafka.tiered.storage.specs" />
<allow pkg="org.apache.kafka.tiered.storage.utils" />
<allow pkg="kafka.api" />
<allow pkg="kafka.log" />
<allow pkg="kafka.server" />
<allow pkg="kafka.utils" />
<allow pkg="org.apache.kafka.common.config" />
<allow pkg="org.apache.kafka.common.record" />
<allow pkg="org.apache.kafka.common.replica" />
<allow pkg="org.apache.kafka.common.network" />
<allow pkg="org.apache.kafka.clients" />
<allow pkg="org.apache.kafka.clients.admin" />
<allow pkg="org.apache.kafka.clients.consumer" />
<allow pkg="org.apache.kafka.clients.producer" />
<allow pkg="org.apache.kafka.metadata" />
<allow pkg="org.apache.kafka.storage"/>
<allow pkg="org.apache.kafka.storage.internals.log" />
<allow pkg="org.apache.kafka.server.log" />
<allow pkg="org.apache.kafka.server.log.remote" />
<allow pkg="org.apache.kafka.server.log.remote.storage" />
<allow pkg="org.apache.kafka.test" />
<subpackage name="actions">
</subpackage>
<subpackage name="specs">
</subpackage>
<subpackage name="utils">
</subpackage>
<subpackage name="integration">
</subpackage>
</subpackage>
<!-- END OF TIERED STORAGE INTEGRATION TEST IMPORT DEPENDENCIES -->
</import-control>

29
checkstyle/import-control.xml

@ -247,35 +247,6 @@ @@ -247,35 +247,6 @@
<!-- This is required to make AlterConfigPolicyTest work. -->
<allow pkg="org.apache.kafka.server.policy" />
<subpackage name="log">
<allow pkg="com.fasterxml.jackson" />
<allow pkg="kafka.api" />
<allow pkg="kafka.utils" />
<allow pkg="org.apache.kafka.clients" />
<allow pkg="org.apache.kafka.server.common" />
<allow pkg="org.apache.kafka.server.config" />
<allow pkg="org.apache.kafka.server.log" />
<allow pkg="org.apache.kafka.server.record" />
<allow pkg="org.apache.kafka.test" />
<allow pkg="org.apache.kafka.storage"/>
<subpackage name="remote">
<allow pkg="scala.collection" />
<subpackage name="storage">
<allow pkg="com.yammer.metrics.core" />
<allow pkg="org.apache.kafka.server.metrics" />
</subpackage>
</subpackage>
</subpackage>
</subpackage>
<subpackage name="storage.internals">
<allow pkg="com.yammer.metrics.core" />
<allow pkg="org.apache.kafka.server"/>
<allow pkg="org.apache.kafka.storage.internals"/>
<allow pkg="org.apache.kafka.common" />
<allow pkg="com.github.benmanes.caffeine.cache" />
</subpackage>
<subpackage name="shell">

2
core/src/main/scala/kafka/server/KafkaBroker.scala

@ -19,6 +19,7 @@ package kafka.server @@ -19,6 +19,7 @@ package kafka.server
import com.yammer.metrics.core.MetricName
import kafka.log.LogManager
import kafka.log.remote.RemoteLogManager
import kafka.metrics.LinuxIoMetricsCollector
import kafka.network.SocketServer
import kafka.security.CredentialProvider
@ -80,6 +81,7 @@ trait KafkaBroker extends Logging { @@ -80,6 +81,7 @@ trait KafkaBroker extends Logging {
def kafkaScheduler: Scheduler
def kafkaYammerMetrics: KafkaYammerMetrics
def logManager: LogManager
def remoteLogManagerOpt: Option[RemoteLogManager]
def metrics: Metrics
def quotaManagers: QuotaFactory.QuotaManagers
def replicaManager: ReplicaManager

9
core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala

@ -258,13 +258,20 @@ abstract class KafkaServerTestHarness extends QuorumTestHarness { @@ -258,13 +258,20 @@ abstract class KafkaServerTestHarness extends QuorumTestHarness {
}
def killBroker(index: Int): Unit = {
if(alive(index)) {
if (alive(index)) {
_brokers(index).shutdown()
_brokers(index).awaitShutdown()
alive(index) = false
}
}
def startBroker(index: Int): Unit = {
if (!alive(index)) {
_brokers(index).startup()
alive(index) = true
}
}
/**
* Restart any dead brokers
*/

4
storage/src/test/java/org/apache/kafka/server/log/remote/storage/LocalTieredStorageTraverser.java

@ -26,9 +26,9 @@ public interface LocalTieredStorageTraverser { @@ -26,9 +26,9 @@ public interface LocalTieredStorageTraverser {
/**
* Called when a new topic-partition stored on the remote storage is discovered.
* @param topicPartition The new topic-partition discovered.
* @param topicIdPartition The new topic-partition discovered.
*/
void visitTopicIdPartition(TopicIdPartition topicPartition);
void visitTopicIdPartition(TopicIdPartition topicIdPartition);
/**
* Called when a new segment is discovered for a given topic-partition.

11
storage/src/test/java/org/apache/kafka/tiered/storage/README.md

@ -0,0 +1,11 @@ @@ -0,0 +1,11 @@
# The Test Flow
Step 1: For every test, setup is done via TieredStorageTestHarness which extends IntegrationTestHarness and sets up a cluster with TS enabled on it.
Step 2: The test is written as a specification consisting of sequential actions and assertions. The spec for the complete test is written down first which creates "actions" to be executed.
Step 3: Once we have the test spec in-place (which includes assertion actions), we execute the test which will execute each action sequentially.
Step 4: The test execution stops when any of the action throws an exception (or an assertion error).
Step 5: Clean-up for the test is performed on test exit

36
storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestAction.java

@ -0,0 +1,36 @@ @@ -0,0 +1,36 @@
/*
* 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 org.apache.kafka.tiered.storage;
import java.io.PrintStream;
public interface TieredStorageTestAction {
default void execute(TieredStorageTestContext context) throws Exception {
try {
doExecute(context);
context.succeed(this);
} catch (Exception e) {
context.fail(this);
throw e;
}
}
void doExecute(TieredStorageTestContext context) throws Exception;
void describe(PrintStream output);
}

395
storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestBuilder.java

@ -0,0 +1,395 @@ @@ -0,0 +1,395 @@
/*
* 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 org.apache.kafka.tiered.storage;
import org.apache.kafka.tiered.storage.actions.BounceBrokerAction;
import org.apache.kafka.tiered.storage.actions.ConsumeAction;
import org.apache.kafka.tiered.storage.actions.CreatePartitionsAction;
import org.apache.kafka.tiered.storage.actions.CreateTopicAction;
import org.apache.kafka.tiered.storage.actions.DeleteRecordsAction;
import org.apache.kafka.tiered.storage.actions.DeleteTopicAction;
import org.apache.kafka.tiered.storage.actions.EraseBrokerStorageAction;
import org.apache.kafka.tiered.storage.actions.ExpectBrokerInISRAction;
import org.apache.kafka.tiered.storage.actions.ExpectEmptyRemoteStorageAction;
import org.apache.kafka.tiered.storage.actions.ExpectLeaderAction;
import org.apache.kafka.tiered.storage.actions.ExpectLeaderEpochCheckpointAction;
import org.apache.kafka.tiered.storage.actions.ExpectListOffsetsAction;
import org.apache.kafka.tiered.storage.actions.ExpectTopicIdToMatchInRemoteStorageAction;
import org.apache.kafka.tiered.storage.actions.ExpectUserTopicMappedToMetadataPartitionsAction;
import org.apache.kafka.tiered.storage.actions.ProduceAction;
import org.apache.kafka.tiered.storage.actions.ReassignReplicaAction;
import org.apache.kafka.tiered.storage.actions.ShrinkReplicaAction;
import org.apache.kafka.tiered.storage.actions.StartBrokerAction;
import org.apache.kafka.tiered.storage.actions.StopBrokerAction;
import org.apache.kafka.tiered.storage.actions.UpdateBrokerConfigAction;
import org.apache.kafka.tiered.storage.actions.UpdateTopicConfigAction;
import org.apache.kafka.tiered.storage.specs.ConsumableSpec;
import org.apache.kafka.tiered.storage.specs.DeletableSpec;
import org.apache.kafka.tiered.storage.specs.ExpandPartitionCountSpec;
import org.apache.kafka.tiered.storage.specs.FetchableSpec;
import org.apache.kafka.tiered.storage.specs.KeyValueSpec;
import org.apache.kafka.tiered.storage.specs.OffloadableSpec;
import org.apache.kafka.tiered.storage.specs.OffloadedSegmentSpec;
import org.apache.kafka.tiered.storage.specs.ProducableSpec;
import org.apache.kafka.tiered.storage.specs.RemoteDeleteSegmentSpec;
import org.apache.kafka.tiered.storage.specs.RemoteFetchSpec;
import org.apache.kafka.tiered.storage.specs.TopicSpec;
import org.apache.kafka.clients.admin.OffsetSpec;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.config.TopicConfig;
import org.apache.kafka.server.log.remote.storage.LocalTieredStorageEvent;
import org.apache.kafka.storage.internals.log.EpochEntry;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;
@SuppressWarnings("ClassDataAbstractionCoupling")
public final class TieredStorageTestBuilder {
private final int defaultProducedBatchSize = 1;
private final long defaultEarliestLocalOffsetExpectedInLogDirectory = 0;
private Map<TopicPartition, ProducableSpec> producables = new HashMap<>();
private Map<TopicPartition, List<OffloadableSpec>> offloadables = new HashMap<>();
private Map<TopicPartition, ConsumableSpec> consumables = new HashMap<>();
private Map<TopicPartition, FetchableSpec> fetchables = new HashMap<>();
private Map<TopicPartition, List<DeletableSpec>> deletables = new HashMap<>();
private List<TieredStorageTestAction> actions = new ArrayList<>();
public TieredStorageTestBuilder() {
}
public TieredStorageTestBuilder createTopic(String topic,
Integer partitionCount,
Integer replicationFactor,
Integer maxBatchCountPerSegment,
Map<Integer, List<Integer>> replicaAssignment,
Boolean enableRemoteLogStorage) {
assertTrue(maxBatchCountPerSegment >= 1, "Segments size for topic " + topic + " needs to be >= 1");
assertTrue(partitionCount >= 1, "Partition count for topic " + topic + " needs to be >= 1");
assertTrue(replicationFactor >= 1, "Replication factor for topic " + topic + " needs to be >= 1");
Map<String, String> properties = new HashMap<>();
properties.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, enableRemoteLogStorage.toString());
TopicSpec topicSpec = new TopicSpec(topic, partitionCount, replicationFactor, maxBatchCountPerSegment,
replicaAssignment, properties);
actions.add(new CreateTopicAction(topicSpec));
return this;
}
public TieredStorageTestBuilder createPartitions(String topic,
Integer partitionCount,
Map<Integer, List<Integer>> replicaAssignment) {
assertTrue(partitionCount >= 1, "Partition count for topic " + topic + " needs to be >= 1");
ExpandPartitionCountSpec spec = new ExpandPartitionCountSpec(topic, partitionCount, replicaAssignment);
actions.add(new CreatePartitionsAction(spec));
return this;
}
public TieredStorageTestBuilder updateTopicConfig(String topic,
Map<String, String> configsToBeAdded,
List<String> configsToBeDeleted) {
assertTrue(!configsToBeAdded.isEmpty() || !configsToBeDeleted.isEmpty(),
"Topic " + topic + " configs shouldn't be empty");
actions.add(new UpdateTopicConfigAction(topic, configsToBeAdded, configsToBeDeleted));
return this;
}
public TieredStorageTestBuilder updateBrokerConfig(Integer brokerId,
Map<String, String> configsToBeAdded,
List<String> configsToBeDeleted) {
assertTrue(!configsToBeAdded.isEmpty() || !configsToBeDeleted.isEmpty(),
"Broker " + brokerId + " configs shouldn't be empty");
actions.add(new UpdateBrokerConfigAction(brokerId, configsToBeAdded, configsToBeDeleted));
return this;
}
public TieredStorageTestBuilder deleteTopic(List<String> topics) {
topics.forEach(topic -> actions.add(buildDeleteTopicAction(topic, true)));
return this;
}
public TieredStorageTestBuilder produce(String topic,
Integer partition,
KeyValueSpec... keyValues) {
assertTrue(partition >= 0, "Partition must be >= 0");
ProducableSpec spec = getOrCreateProducable(topic, partition);
for (KeyValueSpec kv : keyValues) {
spec.getRecords().add(new ProducerRecord<>(topic, partition, kv.getKey(), kv.getValue()));
}
createProduceAction();
return this;
}
public TieredStorageTestBuilder produceWithTimestamp(String topic,
Integer partition,
KeyValueSpec... keyValues) {
assertTrue(partition >= 0, "Partition must be >= 0");
ProducableSpec spec = getOrCreateProducable(topic, partition);
for (KeyValueSpec kv : keyValues) {
spec.getRecords()
.add(new ProducerRecord<>(topic, partition, kv.getTimestamp(), kv.getKey(), kv.getValue()));
}
createProduceAction();
return this;
}
public TieredStorageTestBuilder withBatchSize(String topic,
Integer partition,
Integer batchSize) {
assertTrue(batchSize >= 1, "The size of a batch of produced records must >= 1");
getOrCreateProducable(topic, partition).setBatchSize(batchSize);
return this;
}
public TieredStorageTestBuilder expectEarliestLocalOffsetInLogDirectory(String topic,
Integer partition,
Long earliestLocalOffset) {
assertTrue(earliestLocalOffset >= 0, "Record offset must be >= 0");
getOrCreateProducable(topic, partition).setEarliestLocalLogOffset(earliestLocalOffset);
return this;
}
public TieredStorageTestBuilder expectSegmentToBeOffloaded(Integer fromBroker,
String topic,
Integer partition,
Integer baseOffset,
KeyValueSpec... keyValues) {
TopicPartition topicPartition = new TopicPartition(topic, partition);
List<ProducerRecord<String, String>> records = new ArrayList<>();
for (KeyValueSpec kv: keyValues) {
records.add(new ProducerRecord<>(topic, partition, kv.getKey(), kv.getValue()));
}
offloadables.computeIfAbsent(topicPartition, k -> new ArrayList<>())
.add(new OffloadableSpec(fromBroker, baseOffset, records));
return this;
}
public TieredStorageTestBuilder expectTopicIdToMatchInRemoteStorage(String topic) {
actions.add(new ExpectTopicIdToMatchInRemoteStorageAction(topic));
return this;
}
public TieredStorageTestBuilder consume(String topic,
Integer partition,
Long fetchOffset,
Integer expectedTotalRecord,
Integer expectedRecordsFromSecondTier) {
TopicPartition topicPartition = new TopicPartition(topic, partition);
assertTrue(partition >= 0, "Partition must be >= 0");
assertTrue(fetchOffset >= 0, "Fetch offset must be >=0");
assertTrue(expectedTotalRecord >= 1, "Must read at least one record");
assertTrue(expectedRecordsFromSecondTier >= 0, "Expected read cannot be < 0");
assertTrue(expectedRecordsFromSecondTier <= expectedTotalRecord, "Cannot fetch more records than consumed");
assertFalse(consumables.containsKey(topicPartition), "Consume already in progress for " + topicPartition);
consumables.put(
topicPartition, new ConsumableSpec(fetchOffset, expectedTotalRecord, expectedRecordsFromSecondTier));
createConsumeAction();
return this;
}
public TieredStorageTestBuilder expectLeader(String topic,
Integer partition,
Integer brokerId,
Boolean electLeader) {
actions.add(new ExpectLeaderAction(new TopicPartition(topic, partition), brokerId, electLeader));
return this;
}
public TieredStorageTestBuilder expectInIsr(String topic,
Integer partition,
Integer brokerId) {
actions.add(new ExpectBrokerInISRAction(new TopicPartition(topic, partition), brokerId));
return this;
}
public TieredStorageTestBuilder expectFetchFromTieredStorage(Integer fromBroker,
String topic,
Integer partition,
Integer remoteFetchRequestCount) {
TopicPartition topicPartition = new TopicPartition(topic, partition);
assertTrue(partition >= 0, "Partition must be >= 0");
assertTrue(remoteFetchRequestCount >= 0, "Expected fetch count from tiered storage must be >= 0");
assertFalse(fetchables.containsKey(topicPartition), "Consume already in progress for " + topicPartition);
fetchables.put(topicPartition, new FetchableSpec(fromBroker, remoteFetchRequestCount));
return this;
}
public TieredStorageTestBuilder expectDeletionInRemoteStorage(Integer fromBroker,
String topic,
Integer partition,
LocalTieredStorageEvent.EventType eventType,
Integer eventCount) {
TopicPartition topicPartition = new TopicPartition(topic, partition);
deletables.computeIfAbsent(topicPartition, k -> new ArrayList<>())
.add(new DeletableSpec(fromBroker, eventType, eventCount));
return this;
}
public TieredStorageTestBuilder waitForRemoteLogSegmentDeletion(String topic) {
actions.add(buildDeleteTopicAction(topic, false));
return this;
}
public TieredStorageTestBuilder expectLeaderEpochCheckpoint(Integer brokerId,
String topic,
Integer partition,
Integer beginEpoch,
Long startOffset) {
TopicPartition topicPartition = new TopicPartition(topic, partition);
actions.add(new ExpectLeaderEpochCheckpointAction(brokerId, topicPartition, beginEpoch, startOffset));
return this;
}
public TieredStorageTestBuilder expectListOffsets(String topic,
Integer partition,
OffsetSpec offsetSpec,
EpochEntry epochEntry) {
TopicPartition topicPartition = new TopicPartition(topic, partition);
actions.add(new ExpectListOffsetsAction(topicPartition, offsetSpec, epochEntry));
return this;
}
public TieredStorageTestBuilder bounce(Integer brokerId) {
actions.add(new BounceBrokerAction(brokerId));
return this;
}
public TieredStorageTestBuilder stop(Integer brokerId) {
actions.add(new StopBrokerAction(brokerId));
return this;
}
public TieredStorageTestBuilder start(Integer brokerId) {
actions.add(new StartBrokerAction(brokerId));
return this;
}
public TieredStorageTestBuilder eraseBrokerStorage(Integer brokerId) {
actions.add(new EraseBrokerStorageAction(brokerId));
return this;
}
public TieredStorageTestBuilder expectEmptyRemoteStorage(String topic,
Integer partition) {
TopicPartition topicPartition = new TopicPartition(topic, partition);
actions.add(new ExpectEmptyRemoteStorageAction(topicPartition));
return this;
}
public TieredStorageTestBuilder shrinkReplica(String topic,
Integer partition,
List<Integer> replicaIds) {
TopicPartition topicPartition = new TopicPartition(topic, partition);
actions.add(new ShrinkReplicaAction(topicPartition, replicaIds));
return this;
}
public TieredStorageTestBuilder reassignReplica(String topic,
Integer partition,
List<Integer> replicaIds) {
TopicPartition topicPartition = new TopicPartition(topic, partition);
actions.add(new ReassignReplicaAction(topicPartition, replicaIds));
return this;
}
public TieredStorageTestBuilder expectUserTopicMappedToMetadataPartitions(String topic,
List<Integer> metadataPartitions) {
actions.add(new ExpectUserTopicMappedToMetadataPartitionsAction(topic, metadataPartitions));
return this;
}
public TieredStorageTestBuilder deleteRecords(String topic,
Integer partition,
Long beforeOffset) {
TopicPartition topicPartition = new TopicPartition(topic, partition);
actions.add(new DeleteRecordsAction(topicPartition, beforeOffset));
return this;
}
public List<TieredStorageTestAction> complete() {
return actions;
}
private void createProduceAction() {
if (!producables.isEmpty()) {
producables.forEach((topicPartition, producableSpec) -> {
List<ProducerRecord<String, String>> recordsToProduce = new ArrayList<>(producableSpec.getRecords());
List<OffloadedSegmentSpec> offloadedSegmentSpecs =
offloadables.computeIfAbsent(topicPartition, k -> new ArrayList<>())
.stream()
.map(spec ->
new OffloadedSegmentSpec(spec.getSourceBrokerId(), topicPartition, spec.getBaseOffset(),
spec.getRecords()))
.collect(Collectors.toList());
ProduceAction action = new ProduceAction(topicPartition, offloadedSegmentSpecs, recordsToProduce,
producableSpec.getBatchSize(), producableSpec.getEarliestLocalLogOffset());
actions.add(action);
});
producables = new HashMap<>();
offloadables = new HashMap<>();
}
}
private void createConsumeAction() {
if (!consumables.isEmpty()) {
consumables.forEach((topicPartition, consumableSpec) -> {
FetchableSpec fetchableSpec = fetchables.computeIfAbsent(topicPartition, k -> new FetchableSpec(0, 0));
RemoteFetchSpec remoteFetchSpec = new RemoteFetchSpec(fetchableSpec.getSourceBrokerId(), topicPartition,
fetchableSpec.getFetchCount());
ConsumeAction action = new ConsumeAction(topicPartition, consumableSpec.getFetchOffset(),
consumableSpec.getExpectedTotalCount(), consumableSpec.getExpectedFromSecondTierCount(),
remoteFetchSpec);
actions.add(action);
});
consumables = new HashMap<>();
fetchables = new HashMap<>();
}
}
private ProducableSpec getOrCreateProducable(String topic,
Integer partition) {
TopicPartition topicPartition = new TopicPartition(topic, partition);
return producables.computeIfAbsent(topicPartition,
k -> new ProducableSpec(new ArrayList<>(), defaultProducedBatchSize,
defaultEarliestLocalOffsetExpectedInLogDirectory));
}
private DeleteTopicAction buildDeleteTopicAction(String topic,
Boolean shouldDelete) {
List<RemoteDeleteSegmentSpec> deleteSegmentSpecList = deletables.entrySet()
.stream()
.filter(e -> e.getKey().topic().equals(topic))
.flatMap(e -> {
TopicPartition partition = e.getKey();
List<DeletableSpec> deletableSpecs = e.getValue();
return deletableSpecs.stream()
.map(spec -> new RemoteDeleteSegmentSpec(spec.getSourceBrokerId(), partition,
spec.getEventType(), spec.getEventCount()));
})
.collect(Collectors.toList());
deleteSegmentSpecList.forEach(spec -> deletables.remove(spec.getTopicPartition()));
return new DeleteTopicAction(topic, deleteSegmentSpecList, shouldDelete);
}
}

315
storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestContext.java

@ -0,0 +1,315 @@ @@ -0,0 +1,315 @@
/*
* 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 org.apache.kafka.tiered.storage;
import org.apache.kafka.clients.consumer.Consumer;
import org.apache.kafka.tiered.storage.specs.ExpandPartitionCountSpec;
import org.apache.kafka.tiered.storage.specs.TopicSpec;
import org.apache.kafka.tiered.storage.utils.BrokerLocalStorage;
import kafka.log.UnifiedLog;
import kafka.utils.TestUtils;
import org.apache.kafka.clients.admin.Admin;
import org.apache.kafka.clients.admin.AlterConfigOp;
import org.apache.kafka.clients.admin.AlterConfigsOptions;
import org.apache.kafka.clients.admin.ConfigEntry;
import org.apache.kafka.clients.admin.NewPartitions;
import org.apache.kafka.clients.admin.NewTopic;
import org.apache.kafka.clients.admin.TopicDescription;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.ConsumerRecords;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.TopicPartitionInfo;
import org.apache.kafka.common.config.ConfigResource;
import org.apache.kafka.common.serialization.Deserializer;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.common.serialization.Serializer;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.server.log.remote.storage.LocalTieredStorage;
import org.apache.kafka.server.log.remote.storage.LocalTieredStorageHistory;
import org.apache.kafka.server.log.remote.storage.LocalTieredStorageSnapshot;
import scala.Function0;
import scala.Function1;
import java.io.IOException;
import java.io.PrintStream;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Properties;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;
import scala.Option;
import scala.collection.JavaConverters;
import static org.apache.kafka.clients.producer.ProducerConfig.LINGER_MS_CONFIG;
public final class TieredStorageTestContext implements AutoCloseable {
private final TieredStorageTestHarness harness;
private final Serializer<String> ser = Serdes.String().serializer();
private final Deserializer<String> de = Serdes.String().deserializer();
private final Map<String, TopicSpec> topicSpecs = new HashMap<>();
private final TieredStorageTestReport testReport;
private volatile KafkaProducer<String, String> producer;
private volatile Consumer<String, String> consumer;
private volatile Admin admin;
private volatile List<LocalTieredStorage> remoteStorageManagers;
private volatile List<BrokerLocalStorage> localStorages;
public TieredStorageTestContext(TieredStorageTestHarness harness) {
this.harness = harness;
this.testReport = new TieredStorageTestReport(this);
initClients();
initContext();
}
@SuppressWarnings("deprecation")
private void initClients() {
// Set a producer linger of 60 seconds, in order to optimistically generate batches of
// records with a pre-determined size.
Properties producerOverrideProps = new Properties();
producerOverrideProps.put(LINGER_MS_CONFIG, String.valueOf(TimeUnit.SECONDS.toMillis(60)));
producer = harness.createProducer(ser, ser, producerOverrideProps);
consumer = harness.createConsumer(de, de, new Properties(),
JavaConverters.asScalaBuffer(Collections.<String>emptyList()).toList());
admin = harness.createAdminClient(harness.listenerName(), new Properties());
}
private void initContext() {
remoteStorageManagers = TieredStorageTestHarness.remoteStorageManagers(harness.aliveBrokers());
localStorages = TieredStorageTestHarness.localStorages(harness.aliveBrokers());
}
public void createTopic(TopicSpec spec) throws ExecutionException, InterruptedException {
NewTopic newTopic;
if (spec.getAssignment() == null || spec.getAssignment().isEmpty()) {
newTopic = new NewTopic(spec.getTopicName(), spec.getPartitionCount(), (short) spec.getReplicationFactor());
} else {
Map<Integer, List<Integer>> replicasAssignments = spec.getAssignment();
newTopic = new NewTopic(spec.getTopicName(), replicasAssignments);
}
newTopic.configs(spec.getProperties());
admin.createTopics(Collections.singletonList(newTopic)).all().get();
TestUtils.waitForAllPartitionsMetadata(harness.brokers(), spec.getTopicName(), spec.getPartitionCount());
synchronized (this) {
topicSpecs.put(spec.getTopicName(), spec);
}
}
public void createPartitions(ExpandPartitionCountSpec spec) throws ExecutionException, InterruptedException {
NewPartitions newPartitions;
if (spec.getAssignment() == null || spec.getAssignment().isEmpty()) {
newPartitions = NewPartitions.increaseTo(spec.getPartitionCount());
} else {
Map<Integer, List<Integer>> assignment = spec.getAssignment();
List<List<Integer>> newAssignments = assignment.entrySet().stream()
.sorted(Map.Entry.comparingByKey())
.map(Map.Entry::getValue)
.collect(Collectors.toList());
newPartitions = NewPartitions.increaseTo(spec.getPartitionCount(), newAssignments);
}
Map<String, NewPartitions> partitionsMap = Collections.singletonMap(spec.getTopicName(), newPartitions);
admin.createPartitions(partitionsMap).all().get();
TestUtils.waitForAllPartitionsMetadata(harness.brokers(), spec.getTopicName(), spec.getPartitionCount());
}
public void updateTopicConfig(String topic,
Map<String, String> configsToBeAdded,
List<String> configsToBeDeleted)
throws ExecutionException, InterruptedException, TimeoutException {
ConfigResource configResource = new ConfigResource(ConfigResource.Type.TOPIC, topic);
updateResource(configResource, configsToBeAdded, configsToBeDeleted);
}
public void updateBrokerConfig(Integer brokerId,
Map<String, String> configsToBeAdded,
List<String> configsToBeDeleted)
throws ExecutionException, InterruptedException, TimeoutException {
ConfigResource configResource = new ConfigResource(ConfigResource.Type.BROKER, brokerId.toString());
updateResource(configResource, configsToBeAdded, configsToBeDeleted);
}
private void updateResource(ConfigResource configResource,
Map<String, String> configsToBeAdded,
List<String> configsToBeDeleted)
throws ExecutionException, InterruptedException, TimeoutException {
List<AlterConfigOp> alterEntries = new ArrayList<>();
configsToBeDeleted.forEach(k ->
alterEntries.add(new AlterConfigOp(new ConfigEntry(k, ""), AlterConfigOp.OpType.DELETE)));
configsToBeAdded.forEach((k, v) ->
alterEntries.add(new AlterConfigOp(new ConfigEntry(k, v), AlterConfigOp.OpType.SET)));
AlterConfigsOptions alterOptions = new AlterConfigsOptions().timeoutMs(30000);
Map<ConfigResource, Collection<AlterConfigOp>> configsMap =
Collections.singletonMap(configResource, alterEntries);
admin.incrementalAlterConfigs(configsMap, alterOptions).all().get(30, TimeUnit.SECONDS);
}
public void deleteTopic(String topic) {
TestUtils.deleteTopicWithAdmin(admin, topic, harness.brokers());
}
/**
* Send the given records trying to honor the batch size. This is attempted
* with a large producer linger and the use of an explicit flush every time
* the number of a "group" of records reaches the batch size.
* @param recordsToProduce the records to produce
* @param batchSize the batch size
*/
public void produce(List<ProducerRecord<String, String>> recordsToProduce, Integer batchSize) {
int counter = 0;
for (ProducerRecord<String, String> record : recordsToProduce) {
producer.send(record);
if (counter++ % batchSize == 0) {
producer.flush();
}
}
}
public List<ConsumerRecord<String, String>> consume(TopicPartition topicPartition,
Integer expectedTotalCount,
Long fetchOffset) {
consumer.assign(Collections.singletonList(topicPartition));
consumer.seek(topicPartition, fetchOffset);
long timeoutMs = 60_000L;
String sep = System.lineSeparator();
List<ConsumerRecord<String, String>> records = new ArrayList<>();
Function1<ConsumerRecords<String, String>, Object> pollAction = polledRecords -> {
polledRecords.forEach(records::add);
return records.size() >= expectedTotalCount;
};
Function0<String> messageSupplier = () ->
String.format("Could not consume %d records of %s from offset %d in %d ms. %d message(s) consumed:%s%s",
expectedTotalCount, topicPartition, fetchOffset, timeoutMs, records.size(), sep,
Utils.join(records, sep));
TestUtils.pollRecordsUntilTrue(consumer, pollAction, messageSupplier, timeoutMs);
return records;
}
public Long nextOffset(TopicPartition topicPartition) {
List<TopicPartition> partitions = Collections.singletonList(topicPartition);
consumer.assign(partitions);
consumer.seekToEnd(partitions);
return consumer.position(topicPartition);
}
public Long beginOffset(TopicPartition topicPartition) {
List<TopicPartition> partitions = Collections.singletonList(topicPartition);
consumer.assign(partitions);
consumer.seekToBeginning(partitions);
return consumer.position(topicPartition);
}
public void bounce(int brokerId) {
harness.killBroker(brokerId);
harness.startBroker(brokerId);
initContext();
}
public void stop(int brokerId) {
harness.killBroker(brokerId);
initContext();
}
public void start(int brokerId) {
harness.startBroker(brokerId);
initContext();
}
public void eraseBrokerStorage(int brokerId) throws IOException {
localStorages.get(brokerId).eraseStorage();
}
public TopicSpec topicSpec(String topicName) {
synchronized (topicSpecs) {
return topicSpecs.get(topicName);
}
}
public LocalTieredStorageSnapshot takeTieredStorageSnapshot() {
int aliveBrokerId = harness.aliveBrokers().head().config().brokerId();
return LocalTieredStorageSnapshot.takeSnapshot(remoteStorageManagers.get(aliveBrokerId));
}
public LocalTieredStorageHistory tieredStorageHistory(int brokerId) {
return remoteStorageManagers.get(brokerId).getHistory();
}
public List<LocalTieredStorage> remoteStorageManagers() {
return remoteStorageManagers;
}
public List<BrokerLocalStorage> localStorages() {
return localStorages;
}
public Deserializer<String> de() {
return de;
}
public Admin admin() {
return admin;
}
public boolean isActive(Integer brokerId) {
return harness.aliveBrokers().exists(b -> b.config().brokerId() == brokerId);
}
public boolean isAssignedReplica(TopicPartition topicPartition, Integer replicaId)
throws ExecutionException, InterruptedException {
String topic = topicPartition.topic();
int partition = topicPartition.partition();
TopicDescription description = admin.describeTopics(Collections.singletonList(topicPartition.topic()))
.allTopicNames().get().get(topic);
TopicPartitionInfo partitionInfo = description.partitions().get(partition);
return partitionInfo.replicas().stream().anyMatch(node -> node.id() == replicaId);
}
public Optional<UnifiedLog> log(Integer brokerId, TopicPartition partition) {
Option<UnifiedLog> log = harness.brokers().apply(brokerId).logManager().getLog(partition, false);
return log.isDefined() ? Optional.of(log.get()) : Optional.empty();
}
public void succeed(TieredStorageTestAction action) {
testReport.addSucceeded(action);
}
public void fail(TieredStorageTestAction action) {
testReport.addFailed(action);
}
public void printReport(PrintStream output) {
testReport.print(output);
}
@Override
public void close() throws IOException {
Utils.closeAll(producer, consumer);
Utils.closeQuietly(admin, "Admin client");
}
}

219
storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestHarness.java

@ -0,0 +1,219 @@ @@ -0,0 +1,219 @@
/*
* 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 org.apache.kafka.tiered.storage;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.test.TestUtils;
import org.apache.kafka.tiered.storage.utils.BrokerLocalStorage;
import kafka.api.IntegrationTestHarness;
import kafka.log.remote.RemoteLogManager;
import kafka.server.KafkaBroker;
import kafka.server.KafkaConfig;
import org.apache.kafka.common.replica.ReplicaSelector;
import org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManager;
import org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig;
import org.apache.kafka.server.log.remote.storage.ClassLoaderAwareRemoteStorageManager;
import org.apache.kafka.server.log.remote.storage.LocalTieredStorage;
import org.apache.kafka.server.log.remote.storage.RemoteStorageManager;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Disabled;
import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.TestInfo;
import scala.collection.Seq;
import java.util.ArrayList;
import java.util.List;
import java.util.Locale;
import java.util.Optional;
import java.util.Properties;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import scala.collection.JavaConverters;
import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP;
import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.REMOTE_STORAGE_MANAGER_CLASS_NAME_PROP;
import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.REMOTE_LOG_METADATA_MANAGER_CLASS_NAME_PROP;
import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.REMOTE_LOG_MANAGER_TASK_INTERVAL_MS_PROP;
import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.REMOTE_LOG_METADATA_MANAGER_CONFIG_PREFIX_PROP;
import static org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig.REMOTE_STORAGE_MANAGER_CONFIG_PREFIX_PROP;
import static org.apache.kafka.server.log.remote.storage.LocalTieredStorage.DELETE_ON_CLOSE_CONFIG;
import static org.apache.kafka.server.log.remote.storage.LocalTieredStorage.STORAGE_DIR_CONFIG;
/**
* Base class for integration tests exercising the tiered storage functionality in Apache Kafka.
* This uses a {@link LocalTieredStorage} instance as the second-tier storage system and
* {@link TopicBasedRemoteLogMetadataManager} as the remote log metadata manager.
*/
@Tag("integration")
public abstract class TieredStorageTestHarness extends IntegrationTestHarness {
/**
* InitialTaskDelayMs is set to 30 seconds for the delete-segment scheduler in Apache Kafka.
* Hence, we need to wait at least that amount of time before segments eligible for deletion
* gets physically removed.
*/
private static final Integer STORAGE_WAIT_TIMEOUT_SEC = 35;
// The default value of log cleanup interval is 30 secs, and it increases the test execution time.
private static final Integer LOG_CLEANUP_INTERVAL_MS = 500;
private static final Integer RLM_TASK_INTERVAL_MS = 500;
protected int numRemoteLogMetadataPartitions = 5;
private TieredStorageTestContext context;
private String testClassName = "";
@SuppressWarnings("deprecation")
@Override
public void modifyConfigs(Seq<Properties> props) {
for (Properties p : JavaConverters.seqAsJavaList(props)) {
p.putAll(overridingProps());
}
}
public Properties overridingProps() {
Assertions.assertTrue(STORAGE_WAIT_TIMEOUT_SEC > TimeUnit.MILLISECONDS.toSeconds(RLM_TASK_INTERVAL_MS),
"STORAGE_WAIT_TIMEOUT_SEC should be greater than RLM_TASK_INTERVAL_MS");
Properties overridingProps = new Properties();
// Configure the tiered storage in Kafka. Set an interval of 1 second for the remote log manager background
// activity to ensure the tiered storage has enough room to be exercised within the lifetime of a test.
//
// The replication factor of the remote log metadata topic needs to be chosen so that in resiliency
// tests, metadata can survive the loss of one replica for its topic-partitions.
//
// The second-tier storage system is mocked via the LocalTieredStorage instance which persists transferred
// data files on the local file system.
overridingProps.setProperty(REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, "true");
overridingProps.setProperty(REMOTE_STORAGE_MANAGER_CLASS_NAME_PROP, LocalTieredStorage.class.getName());
overridingProps.setProperty(REMOTE_LOG_METADATA_MANAGER_CLASS_NAME_PROP,
TopicBasedRemoteLogMetadataManager.class.getName());
overridingProps.setProperty(REMOTE_LOG_MANAGER_TASK_INTERVAL_MS_PROP, RLM_TASK_INTERVAL_MS.toString());
overridingProps.setProperty(REMOTE_STORAGE_MANAGER_CONFIG_PREFIX_PROP, storageConfigPrefix(""));
overridingProps.setProperty(REMOTE_LOG_METADATA_MANAGER_CONFIG_PREFIX_PROP, metadataConfigPrefix(""));
overridingProps.setProperty(
metadataConfigPrefix(TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_TOPIC_PARTITIONS_PROP),
String.valueOf(numRemoteLogMetadataPartitions));
overridingProps.setProperty(
metadataConfigPrefix(TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_TOPIC_REPLICATION_FACTOR_PROP),
String.valueOf(brokerCount()));
// This configuration ensures inactive log segments are deleted fast enough so that
// the integration tests can confirm a given log segment is present only in the second-tier storage.
// Note that this does not impact the eligibility of a log segment to be offloaded to the
// second-tier storage.
overridingProps.setProperty(KafkaConfig.LogCleanupIntervalMsProp(), LOG_CLEANUP_INTERVAL_MS.toString());
// This can be customized to read remote log segments from followers.
readReplicaSelectorClass()
.ifPresent(c -> overridingProps.put(KafkaConfig.ReplicaSelectorClassProp(), c.getName()));
// The directory of the second-tier storage needs to be constant across all instances of storage managers
// in every broker and throughout the test. Indeed, as brokers are restarted during the test.
// You can override this property with a fixed path of your choice if you wish to use a non-temporary
// directory to access its content after a test terminated.
overridingProps.setProperty(storageConfigPrefix(STORAGE_DIR_CONFIG),
TestUtils.tempDirectory("kafka-remote-tier-" + testClassName).getAbsolutePath());
// This configuration will remove all the remote files when close is called in remote storage manager.
// Storage manager close is being called while the server is actively processing the socket requests,
// so enabling this config can break the existing tests.
// NOTE: When using TestUtils#tempDir(), the folder gets deleted when VM terminates.
overridingProps.setProperty(storageConfigPrefix(DELETE_ON_CLOSE_CONFIG), "false");
return overridingProps;
}
protected Optional<Class<ReplicaSelector>> readReplicaSelectorClass() {
return Optional.empty();
}
protected abstract void writeTestSpecifications(TieredStorageTestBuilder builder);
@BeforeEach
@Override
public void setUp(TestInfo testInfo) {
testClassName = testInfo.getTestClass().get().getSimpleName().toLowerCase(Locale.getDefault());
super.setUp(testInfo);
context = new TieredStorageTestContext(this);
}
@Disabled("Disabled until the trunk build is stable to test tiered storage")
@Test
public void executeTieredStorageTest() {
TieredStorageTestBuilder builder = new TieredStorageTestBuilder();
writeTestSpecifications(builder);
try {
for (TieredStorageTestAction action : builder.complete()) {
action.execute(context);
}
} catch (Exception ex) {
throw new AssertionError("Could not build test specifications. No test was executed.", ex);
}
}
@AfterEach
@Override
public void tearDown() {
try {
Utils.closeQuietly(context, "TieredStorageTestContext");
super.tearDown();
context.printReport(System.out);
} catch (Exception ex) {
throw new AssertionError("Failed to close the tear down the test harness.", ex);
}
}
private String storageConfigPrefix(String key) {
return "rsm.config." + testClassName + "." + key;
}
private String metadataConfigPrefix(String key) {
return "rlmm.config." + testClassName + "." + key;
}
@SuppressWarnings("deprecation")
public static List<LocalTieredStorage> remoteStorageManagers(Seq<KafkaBroker> brokers) {
List<LocalTieredStorage> storages = new ArrayList<>();
JavaConverters.seqAsJavaList(brokers).forEach(broker -> {
if (broker.remoteLogManagerOpt().isDefined()) {
RemoteLogManager remoteLogManager = broker.remoteLogManagerOpt().get();
RemoteStorageManager storageManager = remoteLogManager.storageManager();
if (storageManager instanceof ClassLoaderAwareRemoteStorageManager) {
ClassLoaderAwareRemoteStorageManager loaderAwareRSM =
(ClassLoaderAwareRemoteStorageManager) storageManager;
if (loaderAwareRSM.delegate() instanceof LocalTieredStorage) {
storages.add((LocalTieredStorage) loaderAwareRSM.delegate());
}
}
} else {
throw new AssertionError("Broker " + broker.config().brokerId()
+ " does not have a remote log manager.");
}
});
return storages;
}
@SuppressWarnings("deprecation")
public static List<BrokerLocalStorage> localStorages(Seq<KafkaBroker> brokers) {
return JavaConverters.seqAsJavaList(brokers).stream()
.map(b -> new BrokerLocalStorage(b.config().brokerId(), b.config().logDirs().head(),
STORAGE_WAIT_TIMEOUT_SEC))
.collect(Collectors.toList());
}
}

72
storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestReport.java

@ -0,0 +1,72 @@ @@ -0,0 +1,72 @@
/*
* 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 org.apache.kafka.tiered.storage;
import org.apache.kafka.tiered.storage.utils.DumpLocalTieredStorage;
import org.apache.kafka.server.log.remote.storage.LocalTieredStorage;
import java.io.PrintStream;
import java.util.ArrayList;
import java.util.List;
public final class TieredStorageTestReport {
private final TieredStorageTestContext context;
private final List<TieredStorageTestAction> successfulActions = new ArrayList<>();
private final List<TieredStorageTestAction> failedActions = new ArrayList<>();
public TieredStorageTestReport(TieredStorageTestContext context) {
this.context = context;
}
public synchronized void addSucceeded(TieredStorageTestAction action) {
successfulActions.add(action);
}
public synchronized void addFailed(TieredStorageTestAction action) {
failedActions.add(action);
}
public void print(PrintStream output) {
output.println();
int seqNo = 0;
List<List<TieredStorageTestAction>> actionsLists = new ArrayList<>();
actionsLists.add(successfulActions);
actionsLists.add(failedActions);
List<String> statusList = new ArrayList<>();
statusList.add("SUCCESS");
statusList.add("FAILURE");
for (int i = 0; i < actionsLists.size(); i++) {
List<TieredStorageTestAction> actions = actionsLists.get(i);
String ident = statusList.get(i);
for (TieredStorageTestAction action : actions) {
seqNo++;
output.print("[" + ident + "] (" + seqNo + ") ");
action.describe(output);
output.println();
}
}
String lts = "";
if (!context.remoteStorageManagers().isEmpty()) {
LocalTieredStorage tieredStorage = context.remoteStorageManagers().get(0);
lts = DumpLocalTieredStorage.dump(tieredStorage, context.de(), context.de());
}
output.printf("Content of local tiered storage:%n%n%s%n", lts);
}
}

41
storage/src/test/java/org/apache/kafka/tiered/storage/actions/BounceBrokerAction.java

@ -0,0 +1,41 @@ @@ -0,0 +1,41 @@
/*
* 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 org.apache.kafka.tiered.storage.actions;
import org.apache.kafka.tiered.storage.TieredStorageTestAction;
import org.apache.kafka.tiered.storage.TieredStorageTestContext;
import java.io.PrintStream;
public final class BounceBrokerAction implements TieredStorageTestAction {
private final int brokerId;
public BounceBrokerAction(int brokerId) {
this.brokerId = brokerId;
}
@Override
public void doExecute(TieredStorageTestContext context) {
context.bounce(brokerId);
}
@Override
public void describe(PrintStream output) {
output.println("bounce-broker: " + brokerId);
}
}

143
storage/src/test/java/org/apache/kafka/tiered/storage/actions/ConsumeAction.java

@ -0,0 +1,143 @@ @@ -0,0 +1,143 @@
/*
* 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 org.apache.kafka.tiered.storage.actions;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.record.Record;
import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.server.log.remote.storage.LocalTieredStorageEvent;
import org.apache.kafka.server.log.remote.storage.LocalTieredStorageHistory;
import org.apache.kafka.tiered.storage.TieredStorageTestAction;
import org.apache.kafka.tiered.storage.TieredStorageTestContext;
import org.apache.kafka.tiered.storage.specs.RemoteFetchSpec;
import java.io.PrintStream;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.ExecutionException;
import java.util.stream.Collectors;
import static org.apache.kafka.server.log.remote.storage.LocalTieredStorageEvent.EventType.FETCH_SEGMENT;
import static org.apache.kafka.tiered.storage.utils.ActionUtils.tieredStorageRecords;
import static org.apache.kafka.tiered.storage.utils.RecordsKeyValueMatcher.correspondTo;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.fail;
public final class ConsumeAction implements TieredStorageTestAction {
private final TopicPartition topicPartition;
private final Long fetchOffset;
private final Integer expectedTotalCount;
private final Integer expectedFromSecondTierCount;
private final RemoteFetchSpec remoteFetchSpec;
private final Serde<String> serde = Serdes.String();
public ConsumeAction(TopicPartition topicPartition,
Long fetchOffset,
Integer expectedTotalCount,
Integer expectedFromSecondTierCount,
RemoteFetchSpec remoteFetchSpec) {
this.topicPartition = topicPartition;
this.fetchOffset = fetchOffset;
this.expectedTotalCount = expectedTotalCount;
this.expectedFromSecondTierCount = expectedFromSecondTierCount;
this.remoteFetchSpec = remoteFetchSpec;
}
@Override
public void doExecute(TieredStorageTestContext context) throws InterruptedException, ExecutionException {
// Retrieve the history (which stores the chronological sequence of interactions with the second-tier
// storage) for the expected broker. Note that while the second-tier storage is unique, each broker
// maintains a local instance of LocalTieredStorage, which is the server-side plug-in interface which
// allows Kafka to interact with that storage. These instances record the interactions (or events)
// between the broker which they belong to and the second-tier storage.
//
// The latest event at the time of invocation for the interaction of type "FETCH_SEGMENT" between the
// given broker and the second-tier storage is retrieved. It can be empty if an interaction of this
// type has yet to happen.
LocalTieredStorageHistory history = context.tieredStorageHistory(remoteFetchSpec.getSourceBrokerId());
Optional<LocalTieredStorageEvent> latestEventSoFar = history.latestEvent(FETCH_SEGMENT, topicPartition);
// Records are consumed here
List<ConsumerRecord<String, String>> consumedRecords =
context.consume(topicPartition, expectedTotalCount, fetchOffset);
// (A) Comparison of records consumed with records in the second-tier storage.
// Reads all records physically found in the second-tier storage ∂for the given topic-partition.
// The resulting sequence is sorted by records offset, as there is no guarantee on ordering from
// the LocalTieredStorageSnapshot.
List<Record> tieredStorageRecords = tieredStorageRecords(context, topicPartition);
Optional<Record> firstExpectedRecordOpt = tieredStorageRecords
.stream()
.filter(record -> record.offset() >= fetchOffset)
.findFirst();
if (!firstExpectedRecordOpt.isPresent()) {
// If no records could be found in the second-tier storage or their offset are less
// than the consumer fetch offset, no record would be consumed from that storage.
if (expectedFromSecondTierCount > 0) {
fail("Could not find any record with offset >= " + fetchOffset + " from tier storage.");
}
return;
}
int indexOfFetchOffsetInTieredStorage = tieredStorageRecords.indexOf(firstExpectedRecordOpt.get());
int recordsCountFromFirstIndex = tieredStorageRecords.size() - indexOfFetchOffsetInTieredStorage;
assertFalse(expectedFromSecondTierCount > recordsCountFromFirstIndex,
"Not enough records found in tiered storage from offset " + fetchOffset + " for "
+ topicPartition + ". Expected: " + expectedFromSecondTierCount
+ ", Was: " + recordsCountFromFirstIndex);
assertFalse(expectedFromSecondTierCount < recordsCountFromFirstIndex,
"Too many records found in tiered storage from offset " + fetchOffset + " for "
+ topicPartition + ". Expected: " + expectedFromSecondTierCount
+ ", Was: " + recordsCountFromFirstIndex);
List<Record> storedRecords =
tieredStorageRecords.subList(indexOfFetchOffsetInTieredStorage, tieredStorageRecords.size());
List<ConsumerRecord<String, String>> readRecords = consumedRecords.subList(0, expectedFromSecondTierCount);
assertThat(storedRecords, correspondTo(readRecords, topicPartition, serde, serde));
// (B) Assessment of the interactions between the source broker and the second-tier storage.
List<LocalTieredStorageEvent> events = history.getEvents(FETCH_SEGMENT, topicPartition);
List<LocalTieredStorageEvent> eventsInScope = latestEventSoFar
.map(latestEvent ->
events.stream().filter(event -> event.isAfter(latestEvent)).collect(Collectors.toList()))
.orElse(events);
assertEquals(remoteFetchSpec.getCount(), eventsInScope.size(),
"Number of fetch requests from broker " + remoteFetchSpec.getSourceBrokerId() + " to the " +
"tier storage does not match the expected value for topic-partition "
+ remoteFetchSpec.getTopicPartition());
}
@Override
public void describe(PrintStream output) {
output.println("consume-action:");
output.println(" topic-partition = " + topicPartition);
output.println(" fetch-offset = " + fetchOffset);
output.println(" expected-record-count = " + expectedTotalCount);
output.println(" expected-record-from-tiered-storage = " + expectedFromSecondTierCount);
}
}

43
storage/src/test/java/org/apache/kafka/tiered/storage/actions/CreatePartitionsAction.java

@ -0,0 +1,43 @@ @@ -0,0 +1,43 @@
/*
* 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 org.apache.kafka.tiered.storage.actions;
import org.apache.kafka.tiered.storage.TieredStorageTestAction;
import org.apache.kafka.tiered.storage.TieredStorageTestContext;
import org.apache.kafka.tiered.storage.specs.ExpandPartitionCountSpec;
import java.io.PrintStream;
import java.util.concurrent.ExecutionException;
public final class CreatePartitionsAction implements TieredStorageTestAction {
private final ExpandPartitionCountSpec spec;
public CreatePartitionsAction(ExpandPartitionCountSpec spec) {
this.spec = spec;
}
@Override
public void doExecute(TieredStorageTestContext context) throws InterruptedException, ExecutionException {
context.createPartitions(spec);
}
@Override
public void describe(PrintStream output) {
output.println("create-partitions: " + spec);
}
}

61
storage/src/test/java/org/apache/kafka/tiered/storage/actions/CreateTopicAction.java

@ -0,0 +1,61 @@ @@ -0,0 +1,61 @@
/*
* 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 org.apache.kafka.tiered.storage.actions;
import org.apache.kafka.tiered.storage.TieredStorageTestAction;
import org.apache.kafka.tiered.storage.TieredStorageTestContext;
import org.apache.kafka.tiered.storage.specs.TopicSpec;
import org.apache.kafka.common.config.TopicConfig;
import java.io.PrintStream;
import java.util.concurrent.ExecutionException;
public final class CreateTopicAction implements TieredStorageTestAction {
private final TopicSpec spec;
public CreateTopicAction(TopicSpec spec) {
this.spec = spec;
}
@Override
public void doExecute(TieredStorageTestContext context) throws ExecutionException, InterruptedException {
// Ensure offset and time indexes are generated for every record.
spec.getProperties().put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, "1");
// Leverage the use of the segment index size to create a log-segment accepting one and only one record.
// The minimum size of the indexes is that of an entry, which is 8 for the offset index and 12 for the
// time index. Hence, since the topic is configured to generate index entries for every record with, for
// a "small" number of records (i.e. such that the average record size times the number of records is
// much less than the segment size), the number of records which hold in a segment is the multiple of 12
// defined below.
if (spec.getMaxBatchCountPerSegment() != -1) {
spec.getProperties().put(
TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, String.valueOf(12 * spec.getMaxBatchCountPerSegment()));
}
// To verify records physically absent from Kafka's storage can be consumed via the second tier storage, we
// want to delete log segments as soon as possible. When tiered storage is active, an inactive log
// segment is not eligible for deletion until it has been offloaded, which guarantees all segments
// should be offloaded before deletion, and their consumption is possible thereafter.
spec.getProperties().put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, "1");
context.createTopic(spec);
}
@Override
public void describe(PrintStream output) {
output.println("create topic: " + spec);
}
}

51
storage/src/test/java/org/apache/kafka/tiered/storage/actions/DeleteRecordsAction.java

@ -0,0 +1,51 @@ @@ -0,0 +1,51 @@
/*
* 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 org.apache.kafka.tiered.storage.actions;
import org.apache.kafka.tiered.storage.TieredStorageTestAction;
import org.apache.kafka.tiered.storage.TieredStorageTestContext;
import org.apache.kafka.clients.admin.RecordsToDelete;
import org.apache.kafka.common.TopicPartition;
import java.io.PrintStream;
import java.util.Collections;
import java.util.Map;
import java.util.concurrent.ExecutionException;
public final class DeleteRecordsAction implements TieredStorageTestAction {
private final TopicPartition partition;
private final Long beforeOffset;
public DeleteRecordsAction(TopicPartition partition,
Long beforeOffset) {
this.partition = partition;
this.beforeOffset = beforeOffset;
}
@Override
public void doExecute(TieredStorageTestContext context) throws InterruptedException, ExecutionException {
Map<TopicPartition, RecordsToDelete> recordsToDeleteMap =
Collections.singletonMap(partition, RecordsToDelete.beforeOffset(beforeOffset));
context.admin().deleteRecords(recordsToDeleteMap).all().get();
}
@Override
public void describe(PrintStream output) {
output.printf("delete-records partition: %s, before-offset: %d%n", partition, beforeOffset);
}
}

86
storage/src/test/java/org/apache/kafka/tiered/storage/actions/DeleteTopicAction.java

@ -0,0 +1,86 @@ @@ -0,0 +1,86 @@
/*
* 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 org.apache.kafka.tiered.storage.actions;
import org.apache.kafka.tiered.storage.TieredStorageTestAction;
import org.apache.kafka.tiered.storage.TieredStorageTestContext;
import org.apache.kafka.tiered.storage.specs.RemoteDeleteSegmentSpec;
import org.apache.kafka.server.log.remote.storage.LocalTieredStorage;
import org.apache.kafka.server.log.remote.storage.LocalTieredStorageCondition;
import java.io.PrintStream;
import java.util.List;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;
import static org.apache.kafka.server.log.remote.storage.LocalTieredStorageCondition.expectEvent;
import static org.apache.kafka.server.log.remote.storage.LocalTieredStorageEvent.EventType.DELETE_SEGMENT;
import static org.apache.kafka.server.log.remote.storage.LocalTieredStorageEvent.EventType.DELETE_PARTITION;
public final class DeleteTopicAction implements TieredStorageTestAction {
private static final int DELETE_WAIT_TIMEOUT_SEC = 10;
private final String topic;
private final List<RemoteDeleteSegmentSpec> deleteSegmentSpecs;
private final Boolean shouldDelete;
public DeleteTopicAction(String topic,
List<RemoteDeleteSegmentSpec> deleteSegmentSpecs,
Boolean shouldDelete) {
this.topic = topic;
this.deleteSegmentSpecs = deleteSegmentSpecs;
this.shouldDelete = shouldDelete;
}
@Override
public void doExecute(TieredStorageTestContext context)
throws ExecutionException, InterruptedException, TimeoutException {
List<LocalTieredStorage> tieredStorages = context.remoteStorageManagers();
List<LocalTieredStorageCondition> tieredStorageConditions = deleteSegmentSpecs.stream()
.filter(spec -> spec.getEventType() == DELETE_SEGMENT || spec.getEventType() == DELETE_PARTITION)
.map(spec -> expectEvent(
tieredStorages,
spec.getEventType(),
spec.getSourceBrokerId(),
spec.getTopicPartition(),
false,
spec.getEventCount()))
.collect(Collectors.toList());
if (shouldDelete) {
context.deleteTopic(topic);
}
if (!tieredStorageConditions.isEmpty()) {
// In stop partitions call, leader tries to delete the remote log partition. Once the partition deletion is
// successful, then the leader sends DELETE_SEGMENT_FINISHED event to __remote_log_metadata topic.
// And, the replica's which listens to the internal topic, updates it's internal cache and drops the segment
// metadata from the cache.
tieredStorageConditions.stream()
.reduce(LocalTieredStorageCondition::and)
.get()
.waitUntilTrue(DELETE_WAIT_TIMEOUT_SEC, TimeUnit.SECONDS);
}
}
@Override
public void describe(PrintStream output) {
String action = shouldDelete ? "delete-topic" : "wait-for-segment-deletion";
output.println(action + ": " + topic);
deleteSegmentSpecs.forEach(spec -> output.println(" " + spec));
}
}

42
storage/src/test/java/org/apache/kafka/tiered/storage/actions/EraseBrokerStorageAction.java

@ -0,0 +1,42 @@ @@ -0,0 +1,42 @@
/*
* 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 org.apache.kafka.tiered.storage.actions;
import org.apache.kafka.tiered.storage.TieredStorageTestAction;
import org.apache.kafka.tiered.storage.TieredStorageTestContext;
import java.io.IOException;
import java.io.PrintStream;
public final class EraseBrokerStorageAction implements TieredStorageTestAction {
private final int brokerId;
public EraseBrokerStorageAction(int brokerId) {
this.brokerId = brokerId;
}
@Override
public void doExecute(TieredStorageTestContext context) throws IOException {
context.eraseBrokerStorage(brokerId);
}
@Override
public void describe(PrintStream output) {
output.println("erase-broker-storage: " + brokerId);
}
}

58
storage/src/test/java/org/apache/kafka/tiered/storage/actions/ExpectBrokerInISRAction.java

@ -0,0 +1,58 @@ @@ -0,0 +1,58 @@
/*
* 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 org.apache.kafka.tiered.storage.actions;
import org.apache.kafka.clients.admin.TopicDescription;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.TopicPartitionInfo;
import org.apache.kafka.test.TestUtils;
import org.apache.kafka.tiered.storage.TieredStorageTestAction;
import org.apache.kafka.tiered.storage.TieredStorageTestContext;
import java.io.PrintStream;
import static org.apache.kafka.tiered.storage.utils.ActionUtils.describeTopic;
public final class ExpectBrokerInISRAction implements TieredStorageTestAction {
private final TopicPartition topicPartition;
private final Integer replicaId;
public ExpectBrokerInISRAction(TopicPartition topicPartition,
Integer replicaId) {
this.topicPartition = topicPartition;
this.replicaId = replicaId;
}
@Override
public void doExecute(TieredStorageTestContext context) throws InterruptedException {
TestUtils.waitForCondition(() -> {
TopicDescription description = describeTopic(context, topicPartition.topic());
TopicPartitionInfo partitionInfo = description.partitions()
.get(topicPartition.partition());
if (partitionInfo != null) {
return partitionInfo.isr().stream().anyMatch(node -> node.id() == replicaId);
}
return false;
}, "Expected broker " + replicaId + " to be in ISR for " + topicPartition);
}
@Override
public void describe(PrintStream output) {
output.printf("expect-broker-in-isr topic-partition: %s broker-id: %d%n", topicPartition, replicaId);
}
}

48
storage/src/test/java/org/apache/kafka/tiered/storage/actions/ExpectEmptyRemoteStorageAction.java

@ -0,0 +1,48 @@ @@ -0,0 +1,48 @@
/*
* 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 org.apache.kafka.tiered.storage.actions;
import org.apache.kafka.tiered.storage.TieredStorageTestAction;
import org.apache.kafka.tiered.storage.TieredStorageTestContext;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.server.log.remote.storage.LocalTieredStorageSnapshot;
import org.apache.kafka.test.TestUtils;
import java.io.PrintStream;
public final class ExpectEmptyRemoteStorageAction implements TieredStorageTestAction {
private final TopicPartition topicPartition;
public ExpectEmptyRemoteStorageAction(TopicPartition topicPartition) {
this.topicPartition = topicPartition;
}
@Override
public void doExecute(TieredStorageTestContext context) throws InterruptedException {
TestUtils.waitForCondition(() -> {
LocalTieredStorageSnapshot snapshot = context.takeTieredStorageSnapshot();
return !snapshot.getTopicPartitions().contains(topicPartition) &&
snapshot.getFilesets(topicPartition).isEmpty();
}, 2000L, "Remote storage is not empty for " + topicPartition);
}
@Override
public void describe(PrintStream output) {
output.println("expect-empty-remote-storage topic-partition: " + topicPartition);
}
}

109
storage/src/test/java/org/apache/kafka/tiered/storage/actions/ExpectLeaderAction.java

@ -0,0 +1,109 @@ @@ -0,0 +1,109 @@
/*
* 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 org.apache.kafka.tiered.storage.actions;
import org.apache.kafka.tiered.storage.TieredStorageTestAction;
import org.apache.kafka.tiered.storage.TieredStorageTestContext;
import org.apache.kafka.clients.admin.NewPartitionReassignment;
import org.apache.kafka.clients.admin.TopicDescription;
import org.apache.kafka.common.ElectionType;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.TopicPartitionInfo;
import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
import org.apache.kafka.test.TestUtils;
import java.io.PrintStream;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.atomic.AtomicInteger;
import static org.apache.kafka.tiered.storage.utils.ActionUtils.describeTopic;
public final class ExpectLeaderAction implements TieredStorageTestAction {
private final TopicPartition topicPartition;
private final int replicaId;
private final Boolean electLeader;
public ExpectLeaderAction(TopicPartition topicPartition,
int replicaId,
Boolean electLeader) {
this.topicPartition = topicPartition;
this.replicaId = replicaId;
this.electLeader = electLeader;
}
@Override
public void doExecute(TieredStorageTestContext context) throws InterruptedException, ExecutionException {
String topic = topicPartition.topic();
int partition = topicPartition.partition();
TestUtils.waitForCondition(() -> {
TopicDescription description = describeTopic(context, topic);
List<Node> isr = description.partitions().get(partition).isr();
return isr != null && isr.stream().anyMatch(node -> node.id() == replicaId);
}, "Broker " + replicaId + " is out of sync for " + partition + ". Cannot be elected as leader.");
reassignPartition(context);
if (electLeader) {
context.admin().electLeaders(ElectionType.PREFERRED, Collections.singleton(topicPartition));
}
AtomicInteger actualLeader = new AtomicInteger(-1);
TestUtils.waitForCondition(() -> {
try {
TopicDescription description = describeTopic(context, topic);
actualLeader.set(
Optional.of(description.partitions().get(partition).leader()).map(Node::id).orElse(-1));
return replicaId == actualLeader.get();
} catch (ExecutionException ex) {
if (ex.getCause() instanceof UnknownTopicOrPartitionException) {
return false;
}
throw new RuntimeException(ex);
}
}, "Leader of " + topicPartition + " was not " + replicaId + ". Actual leader: " + actualLeader);
}
@Override
public void describe(PrintStream output) {
output.printf("expect-leader: topic-partition: %s, replicaId: %d, electLeader: %s%n",
topicPartition, replicaId, electLeader);
}
private void reassignPartition(TieredStorageTestContext context) throws ExecutionException, InterruptedException {
String topic = topicPartition.topic();
TopicPartitionInfo partitionInfo = describeTopic(context, topic)
.partitions()
.get(topicPartition.partition());
List<Integer> targetReplicas = new ArrayList<>();
targetReplicas.add(replicaId);
partitionInfo.replicas().forEach(replica -> {
if (replica.id() != replicaId) {
targetReplicas.add(replica.id());
}
});
Map<TopicPartition, Optional<NewPartitionReassignment>> proposed =
Collections.singletonMap(topicPartition, Optional.of(new NewPartitionReassignment(targetReplicas)));
context.admin().alterPartitionReassignments(proposed);
}
}

74
storage/src/test/java/org/apache/kafka/tiered/storage/actions/ExpectLeaderEpochCheckpointAction.java

@ -0,0 +1,74 @@ @@ -0,0 +1,74 @@
/*
* 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 org.apache.kafka.tiered.storage.actions;
import org.apache.kafka.tiered.storage.TieredStorageTestAction;
import org.apache.kafka.tiered.storage.TieredStorageTestContext;
import kafka.log.UnifiedLog;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache;
import org.apache.kafka.storage.internals.log.EpochEntry;
import org.apache.kafka.test.TestUtils;
import scala.Option;
import java.io.PrintStream;
import java.util.Optional;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.atomic.AtomicReference;
public final class ExpectLeaderEpochCheckpointAction implements TieredStorageTestAction {
private final Integer brokerId;
private final TopicPartition partition;
private final Integer beginEpoch;
private final Long startOffset;
public ExpectLeaderEpochCheckpointAction(Integer brokerId,
TopicPartition partition,
Integer beginEpoch,
Long startOffset) {
this.brokerId = brokerId;
this.partition = partition;
this.beginEpoch = beginEpoch;
this.startOffset = startOffset;
}
@Override
public void doExecute(TieredStorageTestContext context) throws InterruptedException, ExecutionException {
AtomicReference<EpochEntry> earliestEntryOpt = new AtomicReference<>();
TestUtils.waitForCondition(() -> {
EpochEntry earliestEntry = null;
Optional<UnifiedLog> log = context.log(brokerId, partition);
if (log.isPresent()) {
Option<LeaderEpochFileCache> leaderEpochCache = log.get().leaderEpochCache();
if (leaderEpochCache.isDefined()) {
earliestEntry = leaderEpochCache.get().earliestEntry().orElse(null);
}
}
earliestEntryOpt.set(earliestEntry);
return earliestEntry != null && beginEpoch == earliestEntry.epoch
&& startOffset == earliestEntry.startOffset;
}, 2000L, "leader-epoch-checkpoint begin-epoch: " + beginEpoch + " and start-offset: "
+ startOffset + " doesn't match with actual: " + earliestEntryOpt.get());
}
@Override
public void describe(PrintStream output) {
output.printf("expect-leader-epoch-checkpoint broker-id: %d, partition: %s, beginEpoch: %d, startOffset: %d%n",
brokerId, partition, beginEpoch, startOffset);
}
}

66
storage/src/test/java/org/apache/kafka/tiered/storage/actions/ExpectListOffsetsAction.java

@ -0,0 +1,66 @@ @@ -0,0 +1,66 @@
/*
* 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 org.apache.kafka.tiered.storage.actions;
import org.apache.kafka.tiered.storage.TieredStorageTestAction;
import org.apache.kafka.tiered.storage.TieredStorageTestContext;
import org.apache.kafka.clients.admin.ListOffsetsResult;
import org.apache.kafka.clients.admin.OffsetSpec;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.storage.internals.log.EpochEntry;
import java.io.PrintStream;
import java.util.Collections;
import java.util.concurrent.ExecutionException;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;
public final class ExpectListOffsetsAction implements TieredStorageTestAction {
private final TopicPartition partition;
private final OffsetSpec spec;
private final EpochEntry expected;
public ExpectListOffsetsAction(TopicPartition partition,
OffsetSpec spec,
EpochEntry expected) {
this.partition = partition;
this.spec = spec;
this.expected = expected;
}
@Override
public void doExecute(TieredStorageTestContext context) throws InterruptedException, ExecutionException {
ListOffsetsResult.ListOffsetsResultInfo listOffsetsResult = context.admin()
.listOffsets(Collections.singletonMap(partition, spec))
.all()
.get()
.get(partition);
assertEquals(expected.startOffset, listOffsetsResult.offset());
if (expected.epoch != -1) {
assertTrue(listOffsetsResult.leaderEpoch().isPresent());
assertEquals(expected.epoch, listOffsetsResult.leaderEpoch().get());
}
}
@Override
public void describe(PrintStream output) {
output.printf("expect-list-offsets partition: %s, spec: %s, expected-epoch-and-offset: %s%n",
partition, spec, expected);
}
}

61
storage/src/test/java/org/apache/kafka/tiered/storage/actions/ExpectTopicIdToMatchInRemoteStorageAction.java

@ -0,0 +1,61 @@ @@ -0,0 +1,61 @@
/*
* 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 org.apache.kafka.tiered.storage.actions;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.Uuid;
import org.apache.kafka.server.log.remote.storage.LocalTieredStorageSnapshot;
import org.apache.kafka.tiered.storage.TieredStorageTestAction;
import org.apache.kafka.tiered.storage.TieredStorageTestContext;
import java.io.PrintStream;
import java.util.List;
import java.util.concurrent.ExecutionException;
import java.util.stream.Collectors;
import static org.apache.kafka.tiered.storage.utils.ActionUtils.describeTopic;
import static org.junit.jupiter.api.Assertions.assertEquals;
public final class ExpectTopicIdToMatchInRemoteStorageAction implements TieredStorageTestAction {
private final String topic;
public ExpectTopicIdToMatchInRemoteStorageAction(String topic) {
this.topic = topic;
}
@Override
public void doExecute(TieredStorageTestContext context) throws InterruptedException, ExecutionException {
Uuid topicId = describeTopic(context, topic).topicId();
context.remoteStorageManagers().forEach(rsm -> {
LocalTieredStorageSnapshot snapshot = LocalTieredStorageSnapshot.takeSnapshot(rsm);
List<TopicPartition> partitions = snapshot.getTopicPartitions()
.stream()
.filter(tp -> tp.topic().equals(topic))
.collect(Collectors.toList());
partitions.forEach(partition ->
snapshot.getFilesets(partition)
.forEach(fileset -> assertEquals(topicId, fileset.getRemoteLogSegmentId().id()))
);
});
}
@Override
public void describe(PrintStream output) {
output.println("expect-topic-id-to-match-in-remote-storage: " + topic);
}
}

74
storage/src/test/java/org/apache/kafka/tiered/storage/actions/ExpectUserTopicMappedToMetadataPartitionsAction.java

@ -0,0 +1,74 @@ @@ -0,0 +1,74 @@
/*
* 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 org.apache.kafka.tiered.storage.actions;
import org.apache.kafka.tiered.storage.TieredStorageTestAction;
import org.apache.kafka.tiered.storage.TieredStorageTestContext;
import org.apache.kafka.clients.admin.TopicDescription;
import org.apache.kafka.common.TopicIdPartition;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.Uuid;
import org.apache.kafka.server.log.remote.metadata.storage.RemoteLogMetadataTopicPartitioner;
import org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig;
import java.io.PrintStream;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ExecutionException;
import java.util.stream.Collectors;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.apache.kafka.tiered.storage.utils.ActionUtils.describeTopics;
public final class ExpectUserTopicMappedToMetadataPartitionsAction implements TieredStorageTestAction {
private final String topic;
private final List<Integer> metadataPartitions;
public ExpectUserTopicMappedToMetadataPartitionsAction(String topic,
List<Integer> metadataPartitions) {
this.topic = topic;
this.metadataPartitions = metadataPartitions;
}
@Override
public void doExecute(TieredStorageTestContext context) throws InterruptedException, ExecutionException {
String metadataTopic = TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_TOPIC_NAME;
Map<String, TopicDescription> descriptions = describeTopics(context, Arrays.asList(topic, metadataTopic));
int metadataTopicPartitionCount = descriptions.get(metadataTopic).partitions().size();
RemoteLogMetadataTopicPartitioner partitioner =
new RemoteLogMetadataTopicPartitioner(metadataTopicPartitionCount);
Uuid topicId = descriptions.get(topic).topicId();
Set<Integer> actualMetadataPartitions = descriptions.get(topic).partitions()
.stream()
.map(info -> new TopicIdPartition(topicId, new TopicPartition(topic, info.partition())))
.map(partitioner::metadataPartition)
.collect(Collectors.toSet());
assertTrue(actualMetadataPartitions.containsAll(metadataPartitions),
() -> "metadata-partition distribution expected: " + metadataPartitions + ", actual: "
+ actualMetadataPartitions);
}
@Override
public void describe(PrintStream output) {
output.printf("expect-user-topic-mapped-to-metadata-partitions topic: %s metadata-partitions: %s%n",
topic, metadataPartitions);
}
}

162
storage/src/test/java/org/apache/kafka/tiered/storage/actions/ProduceAction.java

@ -0,0 +1,162 @@ @@ -0,0 +1,162 @@
/*
* 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 org.apache.kafka.tiered.storage.actions;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.record.Record;
import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.server.log.remote.storage.LocalTieredStorage;
import org.apache.kafka.server.log.remote.storage.LocalTieredStorageCondition;
import org.apache.kafka.tiered.storage.TieredStorageTestAction;
import org.apache.kafka.tiered.storage.TieredStorageTestContext;
import org.apache.kafka.tiered.storage.specs.OffloadedSegmentSpec;
import org.apache.kafka.tiered.storage.specs.TopicSpec;
import org.apache.kafka.tiered.storage.utils.BrokerLocalStorage;
import java.io.PrintStream;
import java.util.List;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;
import static org.apache.kafka.server.log.remote.storage.LocalTieredStorageCondition.expectEvent;
import static org.apache.kafka.server.log.remote.storage.LocalTieredStorageEvent.EventType.COPY_SEGMENT;
import static org.apache.kafka.tiered.storage.utils.ActionUtils.tieredStorageRecords;
import static org.apache.kafka.tiered.storage.utils.RecordsKeyValueMatcher.correspondTo;
import static org.hamcrest.MatcherAssert.assertThat;
public final class ProduceAction implements TieredStorageTestAction {
// How much time to wait for all remote log segments of a topic-partition to be offloaded
// to the second-tier storage.
private static final int OFFLOAD_WAIT_TIMEOUT_SEC = 20;
private final TopicPartition topicPartition;
private final List<OffloadedSegmentSpec> offloadedSegmentSpecs;
private final List<ProducerRecord<String, String>> recordsToProduce;
private final Integer batchSize;
private final Long expectedEarliestLocalOffset;
private final Serde<String> serde = Serdes.String();
public ProduceAction(TopicPartition topicPartition,
List<OffloadedSegmentSpec> offloadedSegmentSpecs,
List<ProducerRecord<String, String>> recordsToProduce,
Integer batchSize,
Long expectedEarliestLocalOffset) {
this.topicPartition = topicPartition;
this.offloadedSegmentSpecs = offloadedSegmentSpecs;
this.recordsToProduce = recordsToProduce;
this.batchSize = batchSize;
this.expectedEarliestLocalOffset = expectedEarliestLocalOffset;
}
@Override
public void doExecute(TieredStorageTestContext context)
throws InterruptedException, ExecutionException, TimeoutException {
List<LocalTieredStorage> tieredStorages = context.remoteStorageManagers();
List<BrokerLocalStorage> localStorages = context.localStorages();
List<LocalTieredStorageCondition> tieredStorageConditions = offloadedSegmentSpecs.stream()
.map(spec -> expectEvent(
tieredStorages,
COPY_SEGMENT,
spec.getSourceBrokerId(),
spec.getTopicPartition(),
false))
.collect(Collectors.toList());
// Retrieve the offset of the next record which would be consumed from the topic-partition
// before records are produced. This allows consuming only the newly produced records afterwards.
long startOffset = context.nextOffset(topicPartition);
long beginOffset = context.beginOffset(topicPartition);
// Records are produced here.
context.produce(recordsToProduce, batchSize);
if (!tieredStorageConditions.isEmpty()) {
tieredStorageConditions.stream()
.reduce(LocalTieredStorageCondition::and)
.get()
.waitUntilTrue(OFFLOAD_WAIT_TIMEOUT_SEC, TimeUnit.SECONDS);
}
// At this stage, records were produced, and the expected remote log segments found in the second-tier storage.
// Further steps are:
// 1) Verify the local (first-tier) storages contain only the expected log segments - that is to say,
// in the special case of these integration tests, only the active segment.
// 2) Consume the records and verify they match the produced records.
TopicSpec topicSpec = context.topicSpec(topicPartition.topic());
long earliestLocalOffset = expectedEarliestLocalOffset != -1L ? expectedEarliestLocalOffset
: startOffset + recordsToProduce.size()
- (recordsToProduce.size() % topicSpec.getMaxBatchCountPerSegment()) - 1;
for (BrokerLocalStorage localStorage : localStorages) {
// Select brokers which are assigned a replica of the topic-partition
boolean isAssignedReplica = context.isAssignedReplica(topicPartition, localStorage.getBrokerId());
if (isAssignedReplica) {
// Filter out inactive brokers, which may still contain log segments we would expect
// to be deleted based on the retention configuration.
boolean isActive = context.isActive(localStorage.getBrokerId());
if (isActive) {
// Wait until the brokers local storage has been cleared from the inactive log segments.
localStorage.waitForEarliestLocalOffset(topicPartition, earliestLocalOffset);
}
}
}
// Verify that the produced records can be consumed from the topic-partition.
List<ConsumerRecord<String, String>> consumedRecords =
context.consume(topicPartition, recordsToProduce.size(), startOffset);
assertThat(consumedRecords, correspondTo(recordsToProduce, topicPartition, serde, serde));
// Take a physical snapshot of the second-tier storage, and compare the records found with
// those of the expected log segments.
List<Record> tieredStorageRecords = tieredStorageRecords(context, topicPartition);
// Don't include the records which were stored before our records were produced.
List<Record> discoveredRecords =
tieredStorageRecords.subList((int) (startOffset - beginOffset), tieredStorageRecords.size());
List<ProducerRecord<String, String>> producerRecords = offloadedSegmentSpecs.stream()
.flatMap(spec -> spec.getRecords().stream())
.collect(Collectors.toList());
compareRecords(discoveredRecords, producerRecords, topicPartition);
}
@Override
public void describe(PrintStream output) {
output.println("produce-records: " + topicPartition);
recordsToProduce.forEach(record -> output.println(" " + record));
offloadedSegmentSpecs.forEach(spec -> output.println(" " + spec));
}
/**
* Compare the records found in the local tiered storage with the expected records.
*
* @param discoveredRecords The records found in the local tiered storage.
* @param producerRecords The records expected to be found, based on what was sent by the producer.
* @param topicPartition The topic-partition of the records.
*/
private void compareRecords(List<Record> discoveredRecords,
List<ProducerRecord<String, String>> producerRecords,
TopicPartition topicPartition) {
assertThat(discoveredRecords, correspondTo(producerRecords, topicPartition, serde, serde));
}
}

78
storage/src/test/java/org/apache/kafka/tiered/storage/actions/ReassignReplicaAction.java

@ -0,0 +1,78 @@ @@ -0,0 +1,78 @@
/*
* 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 org.apache.kafka.tiered.storage.actions;
import org.apache.kafka.tiered.storage.TieredStorageTestAction;
import org.apache.kafka.tiered.storage.TieredStorageTestContext;
import org.apache.kafka.clients.admin.NewPartitionReassignment;
import org.apache.kafka.clients.admin.TopicDescription;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
import org.apache.kafka.test.TestUtils;
import java.io.PrintStream;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.ExecutionException;
import java.util.stream.Collectors;
import static org.apache.kafka.tiered.storage.utils.ActionUtils.describeTopic;
public final class ReassignReplicaAction implements TieredStorageTestAction {
private final TopicPartition topicPartition;
private final List<Integer> replicaIds;
public ReassignReplicaAction(TopicPartition topicPartition,
List<Integer> replicaIds) {
this.topicPartition = topicPartition;
this.replicaIds = replicaIds;
}
@Override
public void doExecute(TieredStorageTestContext context) throws InterruptedException, ExecutionException {
String topic = topicPartition.topic();
int partition = topicPartition.partition();
Map<TopicPartition, Optional<NewPartitionReassignment>> proposed =
Collections.singletonMap(topicPartition, Optional.of(new NewPartitionReassignment(replicaIds)));
context.admin().alterPartitionReassignments(proposed);
TestUtils.waitForCondition(() -> {
try {
TopicDescription description = describeTopic(context, topic);
List<Integer> actualReplicaIds = description.partitions().get(partition).replicas()
.stream()
.map(Node::id)
.collect(Collectors.toList());
return replicaIds.equals(actualReplicaIds);
} catch (ExecutionException e) {
if (e.getCause() instanceof UnknownTopicOrPartitionException) {
return false;
}
throw new RuntimeException(e);
}
}, "Unable to reassign the replicas of " + topicPartition + ", replica-ids: " + replicaIds);
}
@Override
public void describe(PrintStream output) {
output.printf("reassign-replica topic-partition: %s replica-ids: %s%n", topicPartition, replicaIds);
}
}

83
storage/src/test/java/org/apache/kafka/tiered/storage/actions/ShrinkReplicaAction.java

@ -0,0 +1,83 @@ @@ -0,0 +1,83 @@
/*
* 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 org.apache.kafka.tiered.storage.actions;
import org.apache.kafka.tiered.storage.TieredStorageTestAction;
import org.apache.kafka.tiered.storage.TieredStorageTestContext;
import org.apache.kafka.clients.admin.NewPartitionReassignment;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.TopicPartitionInfo;
import org.apache.kafka.test.TestUtils;
import java.io.PrintStream;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.ExecutionException;
import static org.apache.kafka.tiered.storage.utils.ActionUtils.describeTopic;
public final class ShrinkReplicaAction implements TieredStorageTestAction {
private final TopicPartition topicPartition;
private final List<Integer> replicaIds;
public ShrinkReplicaAction(TopicPartition topicPartition,
List<Integer> replicaIds) {
this.topicPartition = topicPartition;
this.replicaIds = replicaIds;
}
@Override
public void doExecute(TieredStorageTestContext context) throws InterruptedException, ExecutionException {
String topic = topicPartition.topic();
int partition = topicPartition.partition();
TopicPartitionInfo partitionInfo = describeTopic(context, topic).partitions().get(partition);
List<Integer> currentReplicaIds = new ArrayList<>();
List<Integer> targetReplicaIds = new ArrayList<>();
if (partitionInfo != null) {
partitionInfo.replicas().forEach(node -> currentReplicaIds.add(node.id()));
partitionInfo.replicas().stream().filter(node -> replicaIds.contains(node.id()))
.forEach(node -> targetReplicaIds.add(node.id()));
}
Map<TopicPartition, Optional<NewPartitionReassignment>> proposed =
Collections.singletonMap(topicPartition, Optional.of(new NewPartitionReassignment(targetReplicaIds)));
context.admin().alterPartitionReassignments(proposed).all().get();
TestUtils.waitForCondition(() -> {
Set<Integer> actualReplicaIds = new HashSet<>();
TopicPartitionInfo topicPartitionInfo = describeTopic(context, topic).partitions().get(partition);
if (topicPartitionInfo != null) {
topicPartitionInfo.replicas().forEach(node -> actualReplicaIds.add(node.id()));
}
return targetReplicaIds.size() == actualReplicaIds.size()
&& targetReplicaIds.containsAll(actualReplicaIds)
&& actualReplicaIds.containsAll(targetReplicaIds);
}, "Unable to shrink the replicas of " + topicPartition + ", replica-ids: " + replicaIds
+ ", actual-replica-ids: " + currentReplicaIds);
}
@Override
public void describe(PrintStream output) {
output.printf("shrink-replica topic-partition: %s replica-ids: %s%n", topicPartition, replicaIds);
}
}

41
storage/src/test/java/org/apache/kafka/tiered/storage/actions/StartBrokerAction.java

@ -0,0 +1,41 @@ @@ -0,0 +1,41 @@
/*
* 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 org.apache.kafka.tiered.storage.actions;
import org.apache.kafka.tiered.storage.TieredStorageTestAction;
import org.apache.kafka.tiered.storage.TieredStorageTestContext;
import java.io.PrintStream;
public final class StartBrokerAction implements TieredStorageTestAction {
private final int brokerId;
public StartBrokerAction(int brokerId) {
this.brokerId = brokerId;
}
@Override
public void doExecute(TieredStorageTestContext context) {
context.start(brokerId);
}
@Override
public void describe(PrintStream output) {
output.println("start-broker: " + brokerId);
}
}

41
storage/src/test/java/org/apache/kafka/tiered/storage/actions/StopBrokerAction.java

@ -0,0 +1,41 @@ @@ -0,0 +1,41 @@
/*
* 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 org.apache.kafka.tiered.storage.actions;
import org.apache.kafka.tiered.storage.TieredStorageTestAction;
import org.apache.kafka.tiered.storage.TieredStorageTestContext;
import java.io.PrintStream;
public final class StopBrokerAction implements TieredStorageTestAction {
private final int brokerId;
public StopBrokerAction(int brokerId) {
this.brokerId = brokerId;
}
@Override
public void doExecute(TieredStorageTestContext context) {
context.stop(brokerId);
}
@Override
public void describe(PrintStream output) {
output.println("stop-broker: " + brokerId);
}
}

53
storage/src/test/java/org/apache/kafka/tiered/storage/actions/UpdateBrokerConfigAction.java

@ -0,0 +1,53 @@ @@ -0,0 +1,53 @@
/*
* 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 org.apache.kafka.tiered.storage.actions;
import org.apache.kafka.tiered.storage.TieredStorageTestAction;
import org.apache.kafka.tiered.storage.TieredStorageTestContext;
import java.io.PrintStream;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeoutException;
public final class UpdateBrokerConfigAction implements TieredStorageTestAction {
private final int brokerId;
private final Map<String, String> configsToBeAdded;
private final List<String> configsToBeDeleted;
public UpdateBrokerConfigAction(int brokerId,
Map<String, String> configsToBeAdded,
List<String> configsToBeDeleted) {
this.brokerId = brokerId;
this.configsToBeAdded = configsToBeAdded;
this.configsToBeDeleted = configsToBeDeleted;
}
@Override
public void doExecute(TieredStorageTestContext context)
throws ExecutionException, InterruptedException, TimeoutException {
context.updateBrokerConfig(brokerId, configsToBeAdded, configsToBeDeleted);
}
@Override
public void describe(PrintStream output) {
output.printf("Update broker config: %d, configs-to-be-added: %s, configs-to-be-deleted: %s%n",
brokerId, configsToBeAdded, configsToBeDeleted);
}
}

52
storage/src/test/java/org/apache/kafka/tiered/storage/actions/UpdateTopicConfigAction.java

@ -0,0 +1,52 @@ @@ -0,0 +1,52 @@
/*
* 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 org.apache.kafka.tiered.storage.actions;
import org.apache.kafka.tiered.storage.TieredStorageTestAction;
import org.apache.kafka.tiered.storage.TieredStorageTestContext;
import java.io.PrintStream;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeoutException;
public final class UpdateTopicConfigAction implements TieredStorageTestAction {
private final String topic;
private final Map<String, String> configsToBeAdded;
private final List<String> configsToBeDeleted;
public UpdateTopicConfigAction(String topic,
Map<String, String> configsToBeAdded,
List<String> configsToBeDeleted) {
this.topic = topic;
this.configsToBeAdded = configsToBeAdded;
this.configsToBeDeleted = configsToBeDeleted;
}
@Override
public void doExecute(TieredStorageTestContext context)
throws ExecutionException, InterruptedException, TimeoutException {
context.updateTopicConfig(topic, configsToBeAdded, configsToBeDeleted);
}
@Override
public void describe(PrintStream output) {
output.printf("update topic config: %s, configs-to-be-added: %s, configs-to-be-deleted: %s%n",
topic, configsToBeAdded, configsToBeDeleted);
}
}

136
storage/src/test/java/org/apache/kafka/tiered/storage/integration/OffloadAndConsumeFromLeaderTest.java

@ -0,0 +1,136 @@ @@ -0,0 +1,136 @@
/*
* 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 org.apache.kafka.tiered.storage.integration;
import org.apache.kafka.tiered.storage.TieredStorageTestBuilder;
import org.apache.kafka.tiered.storage.TieredStorageTestHarness;
import org.apache.kafka.tiered.storage.specs.KeyValueSpec;
import java.util.List;
import java.util.Map;
/**
* Test Cases (A):
* Elementary offloads and fetches from tiered storage.
*/
public final class OffloadAndConsumeFromLeaderTest extends TieredStorageTestHarness {
/**
* Cluster of one broker
* @return number of brokers in the cluster
*/
@Override
public int brokerCount() {
return 1;
}
@Override
protected void writeTestSpecifications(TieredStorageTestBuilder builder) {
final Integer broker = 0;
final String topicA = "topicA";
final String topicB = "topicB";
final Integer p0 = 0;
final Integer partitionCount = 1;
final Integer replicationFactor = 1;
final Integer maxBatchCountPerSegment = 1;
final Map<Integer, List<Integer>> replicaAssignment = null;
final boolean enableRemoteLogStorage = true;
final Integer batchSize = 1;
builder
/*
* (A.1) Create a topic which segments contain only one batch and produce three records
* with a batch size of 1.
*
* The topic and broker are configured so that the two rolled segments are picked from
* the offloaded to the tiered storage and not present in the first-tier broker storage.
*
* Acceptance:
* -----------
* State of the storages after production of the records and propagation of the log
* segment lifecycles to peer subsystems (log cleaner, remote log manager).
*
* - First-tier storage - - Second-tier storage -
* Log tA-p0 Log tA-p0
* *-------------------* *-------------------*
* | base offset = 2 | | base offset = 0 |
* | (k3, v3) | | (k1, v1) |
* *-------------------* *-------------------*
* *-------------------*
* | base offset = 1 |
* | (k2, v2) |
* *-------------------*
*/
.createTopic(topicA, partitionCount, replicationFactor, maxBatchCountPerSegment, replicaAssignment,
enableRemoteLogStorage)
.withBatchSize(topicA, p0, batchSize)
.expectSegmentToBeOffloaded(broker, topicA, p0, 0, new KeyValueSpec("k1", "v1"))
.expectSegmentToBeOffloaded(broker, topicA, p0, 0, new KeyValueSpec("k2", "v2"))
.produce(topicA, p0, new KeyValueSpec("k1", "v1"), new KeyValueSpec("k2", "v2"),
new KeyValueSpec("k3", "v3"))
/*
* (A.2) Similar scenario as above, but with segments of two records.
*
* Acceptance:
* -----------
* State of the storages after production of the records and propagation of the log
* segment lifecycles to peer subsystems (log cleaner, remote log manager).
*
* - First-tier storage - - Second-tier storage -
* Log tB-p0 Log tB-p0
* *-------------------* *-------------------*
* | base offset = 4 | | base offset = 0 |
* | (k5, v5) | | (k1, v1) |
* *-------------------* | (k2, v2) |
* *-------------------*
* *-------------------*
* | base offset = 2 |
* | (k3, v3) |
* | (k4, v4) |
* *-------------------*
*/
.createTopic(topicB, partitionCount, replicationFactor, 2, replicaAssignment,
enableRemoteLogStorage)
.withBatchSize(topicB, p0, batchSize)
.expectEarliestLocalOffsetInLogDirectory(topicB, p0, 4L)
.expectSegmentToBeOffloaded(broker, topicB, p0, 0,
new KeyValueSpec("k1", "v1"), new KeyValueSpec("k2", "v2"))
.expectSegmentToBeOffloaded(broker, topicB, p0, 2,
new KeyValueSpec("k3", "v3"), new KeyValueSpec("k4", "v4"))
.produce(topicB, p0, new KeyValueSpec("k1", "v1"), new KeyValueSpec("k2", "v2"),
new KeyValueSpec("k3", "v3"), new KeyValueSpec("k4", "v4"), new KeyValueSpec("k5", "v5"))
/*
* (A.3) Stops and restarts the broker. The purpose of this test is to a) exercise consumption
* from a given offset and b) verify that upon broker start, existing remote log segments
* metadata are loaded by Kafka and these log segments available.
*
* Acceptance:
* -----------
* - For topic A, this offset is defined such that only the second segment is fetched from
* the tiered storage.
* - For topic B, only one segment is present in the tiered storage, as asserted by the
* previous sub-test-case.
*/
.bounce(broker)
.expectFetchFromTieredStorage(broker, topicA, p0, 1)
.expectFetchFromTieredStorage(broker, topicB, p0, 2)
.consume(topicA, p0, 1L, 2, 1)
.consume(topicB, p0, 1L, 4, 3);
}
}

70
storage/src/test/java/org/apache/kafka/tiered/storage/specs/ConsumableSpec.java

@ -0,0 +1,70 @@ @@ -0,0 +1,70 @@
/*
* 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 org.apache.kafka.tiered.storage.specs;
import java.util.Objects;
public final class ConsumableSpec {
private final Long fetchOffset;
private final Integer expectedTotalCount;
private final Integer expectedFromSecondTierCount;
public ConsumableSpec(Long fetchOffset,
Integer expectedTotalCount,
Integer expectedFromSecondTierCount) {
this.fetchOffset = fetchOffset;
this.expectedTotalCount = expectedTotalCount;
this.expectedFromSecondTierCount = expectedFromSecondTierCount;
}
public Long getFetchOffset() {
return fetchOffset;
}
public Integer getExpectedTotalCount() {
return expectedTotalCount;
}
public Integer getExpectedFromSecondTierCount() {
return expectedFromSecondTierCount;
}
@Override
public String toString() {
return "ConsumableSpec{" +
"fetchOffset=" + fetchOffset +
", expectedTotalCount=" + expectedTotalCount +
", expectedFromSecondTierCount=" + expectedFromSecondTierCount +
'}';
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
ConsumableSpec that = (ConsumableSpec) o;
return Objects.equals(fetchOffset, that.fetchOffset)
&& Objects.equals(expectedTotalCount, that.expectedTotalCount)
&& Objects.equals(expectedFromSecondTierCount, that.expectedFromSecondTierCount);
}
@Override
public int hashCode() {
return Objects.hash(fetchOffset, expectedTotalCount, expectedFromSecondTierCount);
}
}

72
storage/src/test/java/org/apache/kafka/tiered/storage/specs/DeletableSpec.java

@ -0,0 +1,72 @@ @@ -0,0 +1,72 @@
/*
* 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 org.apache.kafka.tiered.storage.specs;
import org.apache.kafka.server.log.remote.storage.LocalTieredStorageEvent;
import java.util.Objects;
public final class DeletableSpec {
private final Integer sourceBrokerId;
private final LocalTieredStorageEvent.EventType eventType;
private final Integer eventCount;
public DeletableSpec(Integer sourceBrokerId,
LocalTieredStorageEvent.EventType eventType,
Integer eventCount) {
this.sourceBrokerId = sourceBrokerId;
this.eventType = eventType;
this.eventCount = eventCount;
}
public Integer getSourceBrokerId() {
return sourceBrokerId;
}
public LocalTieredStorageEvent.EventType getEventType() {
return eventType;
}
public Integer getEventCount() {
return eventCount;
}
@Override
public String toString() {
return "DeletableSpec{" +
"sourceBrokerId=" + sourceBrokerId +
", eventType=" + eventType +
", eventCount=" + eventCount +
'}';
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
DeletableSpec that = (DeletableSpec) o;
return Objects.equals(sourceBrokerId, that.sourceBrokerId)
&& eventType == that.eventType
&& Objects.equals(eventCount, that.eventCount);
}
@Override
public int hashCode() {
return Objects.hash(sourceBrokerId, eventType, eventCount);
}
}

74
storage/src/test/java/org/apache/kafka/tiered/storage/specs/ExpandPartitionCountSpec.java

@ -0,0 +1,74 @@ @@ -0,0 +1,74 @@
/*
* 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 org.apache.kafka.tiered.storage.specs;
import java.util.List;
import java.util.Map;
import java.util.Objects;
public final class ExpandPartitionCountSpec {
private final String topicName;
private final int partitionCount;
private final Map<Integer, List<Integer>> assignment;
public ExpandPartitionCountSpec(String topicName,
int partitionCount) {
this(topicName, partitionCount, null);
}
public ExpandPartitionCountSpec(String topicName,
int partitionCount,
Map<Integer, List<Integer>> assignment) {
this.topicName = topicName;
this.partitionCount = partitionCount;
this.assignment = assignment;
}
public String getTopicName() {
return topicName;
}
public int getPartitionCount() {
return partitionCount;
}
public Map<Integer, List<Integer>> getAssignment() {
return assignment;
}
@Override
public String toString() {
return String.format("ExpandPartitionCountSpec[topicName=%s, partitionCount=%d, assignment=%s]",
topicName, partitionCount, assignment);
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
ExpandPartitionCountSpec that = (ExpandPartitionCountSpec) o;
return partitionCount == that.partitionCount
&& Objects.equals(topicName, that.topicName)
&& Objects.equals(assignment, that.assignment);
}
@Override
public int hashCode() {
return Objects.hash(topicName, partitionCount, assignment);
}
}

60
storage/src/test/java/org/apache/kafka/tiered/storage/specs/FetchableSpec.java

@ -0,0 +1,60 @@ @@ -0,0 +1,60 @@
/*
* 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 org.apache.kafka.tiered.storage.specs;
import java.util.Objects;
public final class FetchableSpec {
private final Integer sourceBrokerId;
private final Integer fetchCount;
public FetchableSpec(Integer sourceBrokerId,
Integer fetchCount) {
this.sourceBrokerId = sourceBrokerId;
this.fetchCount = fetchCount;
}
public Integer getSourceBrokerId() {
return sourceBrokerId;
}
public Integer getFetchCount() {
return fetchCount;
}
@Override
public String toString() {
return "FetchableSpec{" +
"sourceBrokerId=" + sourceBrokerId +
", fetchCount=" + fetchCount +
'}';
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
FetchableSpec that = (FetchableSpec) o;
return Objects.equals(sourceBrokerId, that.sourceBrokerId) && Objects.equals(fetchCount, that.fetchCount);
}
@Override
public int hashCode() {
return Objects.hash(sourceBrokerId, fetchCount);
}
}

75
storage/src/test/java/org/apache/kafka/tiered/storage/specs/KeyValueSpec.java

@ -0,0 +1,75 @@ @@ -0,0 +1,75 @@
/*
* 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 org.apache.kafka.tiered.storage.specs;
import java.util.Objects;
public final class KeyValueSpec {
private final String key;
private final String value;
private final Long timestamp;
public KeyValueSpec(String key,
String value) {
this(key, value, null);
}
public KeyValueSpec(String key,
String value,
Long timestamp) {
this.key = key;
this.value = value;
this.timestamp = timestamp;
}
public String getKey() {
return key;
}
public String getValue() {
return value;
}
public Long getTimestamp() {
return timestamp;
}
@Override
public String toString() {
return "KeyValueSpec{" +
"key='" + key + '\'' +
", value='" + value + '\'' +
", timestamp=" + timestamp +
'}';
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
KeyValueSpec that = (KeyValueSpec) o;
return Objects.equals(key, that.key)
&& Objects.equals(value, that.value)
&& Objects.equals(timestamp, that.timestamp);
}
@Override
public int hashCode() {
return Objects.hash(key, value, timestamp);
}
}

73
storage/src/test/java/org/apache/kafka/tiered/storage/specs/OffloadableSpec.java

@ -0,0 +1,73 @@ @@ -0,0 +1,73 @@
/*
* 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 org.apache.kafka.tiered.storage.specs;
import org.apache.kafka.clients.producer.ProducerRecord;
import java.util.List;
import java.util.Objects;
public final class OffloadableSpec {
private final Integer sourceBrokerId;
private final Integer baseOffset;
private final List<ProducerRecord<String, String>> records;
public OffloadableSpec(Integer sourceBrokerId,
Integer baseOffset,
List<ProducerRecord<String, String>> records) {
this.sourceBrokerId = sourceBrokerId;
this.baseOffset = baseOffset;
this.records = records;
}
public Integer getSourceBrokerId() {
return sourceBrokerId;
}
public Integer getBaseOffset() {
return baseOffset;
}
public List<ProducerRecord<String, String>> getRecords() {
return records;
}
@Override
public String toString() {
return "OffloadableSpec{" +
"sourceBrokerId=" + sourceBrokerId +
", baseOffset=" + baseOffset +
", records=" + records +
'}';
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
OffloadableSpec that = (OffloadableSpec) o;
return Objects.equals(sourceBrokerId, that.sourceBrokerId)
&& Objects.equals(baseOffset, that.baseOffset)
&& Objects.equals(records, that.records);
}
@Override
public int hashCode() {
return Objects.hash(sourceBrokerId, baseOffset, records);
}
}

87
storage/src/test/java/org/apache/kafka/tiered/storage/specs/OffloadedSegmentSpec.java

@ -0,0 +1,87 @@ @@ -0,0 +1,87 @@
/*
* 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 org.apache.kafka.tiered.storage.specs;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.TopicPartition;
import java.util.List;
import java.util.Objects;
public final class OffloadedSegmentSpec {
private final int sourceBrokerId;
private final TopicPartition topicPartition;
private final int baseOffset;
private final List<ProducerRecord<String, String>> records;
/**
* Specifies a remote log segment expected to be found in a second-tier storage.
*
* @param sourceBrokerId The broker which offloaded (uploaded) the segment to the second-tier storage.
* @param topicPartition The topic-partition which the remote log segment belongs to.
* @param baseOffset The base offset of the remote log segment.
* @param records The records *expected* in the remote log segment.
*/
public OffloadedSegmentSpec(int sourceBrokerId,
TopicPartition topicPartition,
int baseOffset,
List<ProducerRecord<String, String>> records) {
this.sourceBrokerId = sourceBrokerId;
this.topicPartition = topicPartition;
this.baseOffset = baseOffset;
this.records = records;
}
public int getSourceBrokerId() {
return sourceBrokerId;
}
public TopicPartition getTopicPartition() {
return topicPartition;
}
public int getBaseOffset() {
return baseOffset;
}
public List<ProducerRecord<String, String>> getRecords() {
return records;
}
@Override
public String toString() {
return String.format("Segment[partition=%s offloaded-by-broker-id=%d base-offset=%d record-count=%d]",
topicPartition, sourceBrokerId, baseOffset, records.size());
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
OffloadedSegmentSpec that = (OffloadedSegmentSpec) o;
return sourceBrokerId == that.sourceBrokerId
&& baseOffset == that.baseOffset
&& Objects.equals(topicPartition, that.topicPartition)
&& Objects.equals(records, that.records);
}
@Override
public int hashCode() {
return Objects.hash(sourceBrokerId, topicPartition, baseOffset, records);
}
}

81
storage/src/test/java/org/apache/kafka/tiered/storage/specs/ProducableSpec.java

@ -0,0 +1,81 @@ @@ -0,0 +1,81 @@
/*
* 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 org.apache.kafka.tiered.storage.specs;
import org.apache.kafka.clients.producer.ProducerRecord;
import java.util.List;
import java.util.Objects;
public final class ProducableSpec {
private final List<ProducerRecord<String, String>> records;
private Integer batchSize;
private Long earliestLocalLogOffset;
public ProducableSpec(List<ProducerRecord<String, String>> records,
Integer batchSize,
Long earliestLocalLogOffset) {
this.records = records;
this.batchSize = batchSize;
this.earliestLocalLogOffset = earliestLocalLogOffset;
}
public List<ProducerRecord<String, String>> getRecords() {
return records;
}
public Integer getBatchSize() {
return batchSize;
}
public void setBatchSize(Integer batchSize) {
this.batchSize = batchSize;
}
public Long getEarliestLocalLogOffset() {
return earliestLocalLogOffset;
}
public void setEarliestLocalLogOffset(Long earliestLocalLogOffset) {
this.earliestLocalLogOffset = earliestLocalLogOffset;
}
@Override
public String toString() {
return "ProducableSpec{" +
"records=" + records +
", batchSize=" + batchSize +
", earliestLocalLogOffset=" + earliestLocalLogOffset +
'}';
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
ProducableSpec that = (ProducableSpec) o;
return Objects.equals(records, that.records)
&& Objects.equals(batchSize, that.batchSize)
&& Objects.equals(earliestLocalLogOffset, that.earliestLocalLogOffset);
}
@Override
public int hashCode() {
return Objects.hash(records, batchSize, earliestLocalLogOffset);
}
}

89
storage/src/test/java/org/apache/kafka/tiered/storage/specs/RemoteDeleteSegmentSpec.java

@ -0,0 +1,89 @@ @@ -0,0 +1,89 @@
/*
* 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 org.apache.kafka.tiered.storage.specs;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.server.log.remote.storage.LocalTieredStorageEvent;
import java.util.Objects;
public final class RemoteDeleteSegmentSpec {
private final int sourceBrokerId;
private final TopicPartition topicPartition;
private final LocalTieredStorageEvent.EventType eventType;
private final int eventCount;
/**
* Specifies a delete segment/partition event from a second-tier storage. This is used to ensure the
* interactions between Kafka and the second-tier storage match expectations.
*
* @param sourceBrokerId The broker which deletes (a) remote log segments(s) (or) deletes the entire partition from
* the second-tier storage.
* @param topicPartition The topic-partition in which the deletion should happen.
* @param eventType Allowed event types are {@link LocalTieredStorageEvent.EventType#DELETE_PARTITION} and
* {@link LocalTieredStorageEvent.EventType#DELETE_SEGMENT}
* @param eventCount How many events are expected to interact with the second-tier storage.
*/
public RemoteDeleteSegmentSpec(int sourceBrokerId,
TopicPartition topicPartition,
LocalTieredStorageEvent.EventType eventType,
int eventCount) {
this.sourceBrokerId = sourceBrokerId;
this.topicPartition = topicPartition;
this.eventType = eventType;
this.eventCount = eventCount;
}
public int getSourceBrokerId() {
return sourceBrokerId;
}
public TopicPartition getTopicPartition() {
return topicPartition;
}
public LocalTieredStorageEvent.EventType getEventType() {
return eventType;
}
public int getEventCount() {
return eventCount;
}
@Override
public String toString() {
return String.format("RemoteDeleteSegment[source-broker-id=%d topic-partition=%s event-type=%s event-count=%d]",
sourceBrokerId, topicPartition, eventType, eventCount);
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
RemoteDeleteSegmentSpec that = (RemoteDeleteSegmentSpec) o;
return sourceBrokerId == that.sourceBrokerId
&& eventCount == that.eventCount
&& Objects.equals(topicPartition, that.topicPartition)
&& eventType == that.eventType;
}
@Override
public int hashCode() {
return Objects.hash(sourceBrokerId, topicPartition, eventType, eventCount);
}
}

77
storage/src/test/java/org/apache/kafka/tiered/storage/specs/RemoteFetchSpec.java

@ -0,0 +1,77 @@ @@ -0,0 +1,77 @@
/*
* 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 org.apache.kafka.tiered.storage.specs;
import org.apache.kafka.common.TopicPartition;
import java.util.Objects;
public final class RemoteFetchSpec {
private final int sourceBrokerId;
private final TopicPartition topicPartition;
private final int count;
/**
* Specifies a fetch (download) event from a second-tier storage. This is used to ensure the
* interactions between Kafka and the second-tier storage match expectations.
*
* @param sourceBrokerId The broker which fetched (a) remote log segment(s) from the second-tier storage.
* @param topicPartition The topic-partition which segment(s) were fetched.
* @param count The number of remote log segment(s) fetched.
*/
public RemoteFetchSpec(int sourceBrokerId,
TopicPartition topicPartition,
int count) {
this.sourceBrokerId = sourceBrokerId;
this.topicPartition = topicPartition;
this.count = count;
}
public int getSourceBrokerId() {
return sourceBrokerId;
}
public TopicPartition getTopicPartition() {
return topicPartition;
}
public int getCount() {
return count;
}
@Override
public String toString() {
return String.format("RemoteFetch[source-broker-id=%d topic-partition=%s count=%d]",
sourceBrokerId, topicPartition, count);
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
RemoteFetchSpec that = (RemoteFetchSpec) o;
return sourceBrokerId == that.sourceBrokerId
&& count == that.count
&& Objects.equals(topicPartition, that.topicPartition);
}
@Override
public int hashCode() {
return Objects.hash(sourceBrokerId, topicPartition, count);
}
}

106
storage/src/test/java/org/apache/kafka/tiered/storage/specs/TopicSpec.java

@ -0,0 +1,106 @@ @@ -0,0 +1,106 @@
/*
* 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 org.apache.kafka.tiered.storage.specs;
import java.util.List;
import java.util.Map;
import java.util.Objects;
public final class TopicSpec {
private final String topicName;
private final int partitionCount;
private final int replicationFactor;
private final int maxBatchCountPerSegment;
private final Map<Integer, List<Integer>> assignment;
private final Map<String, String> properties;
/**
* Specifies a topic-partition with attributes customized for the purpose of tiered-storage tests.
*
* @param topicName The name of the topic.
* @param partitionCount The number of partitions for the topic.
* @param replicationFactor The replication factor of the topic.
* @param maxBatchCountPerSegment The maximal number of batch in segments of the topic.
* This allows to obtain a fixed, pre-determined size for the segment, which ease
* reasoning on the expected states of local and tiered storages.
* @param properties Configuration of the topic customized for the purpose of tiered-storage tests.
*/
public TopicSpec(String topicName,
int partitionCount,
int replicationFactor,
int maxBatchCountPerSegment,
Map<Integer, List<Integer>> assignment,
Map<String, String> properties) {
this.topicName = topicName;
this.partitionCount = partitionCount;
this.replicationFactor = replicationFactor;
this.maxBatchCountPerSegment = maxBatchCountPerSegment;
this.assignment = assignment;
this.properties = properties;
}
public String getTopicName() {
return topicName;
}
public int getPartitionCount() {
return partitionCount;
}
public int getReplicationFactor() {
return replicationFactor;
}
public int getMaxBatchCountPerSegment() {
return maxBatchCountPerSegment;
}
public Map<Integer, List<Integer>> getAssignment() {
return assignment;
}
public Map<String, String> getProperties() {
return properties;
}
@Override
public String toString() {
return String.format(
"Topic[name=%s partition-count=%d replication-factor=%d segment-size=%d assignment=%s properties=%s]",
topicName, partitionCount, replicationFactor, maxBatchCountPerSegment, assignment, properties);
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
TopicSpec topicSpec = (TopicSpec) o;
return partitionCount == topicSpec.partitionCount
&& replicationFactor == topicSpec.replicationFactor
&& maxBatchCountPerSegment == topicSpec.maxBatchCountPerSegment
&& Objects.equals(topicName, topicSpec.topicName)
&& Objects.equals(assignment, topicSpec.assignment)
&& Objects.equals(properties, topicSpec.properties);
}
@Override
public int hashCode() {
return Objects.hash(topicName, partitionCount, replicationFactor, maxBatchCountPerSegment, assignment,
properties);
}
}

72
storage/src/test/java/org/apache/kafka/tiered/storage/utils/ActionUtils.java

@ -0,0 +1,72 @@ @@ -0,0 +1,72 @@
/*
* 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 org.apache.kafka.tiered.storage.utils;
import org.apache.kafka.clients.admin.TopicDescription;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.record.Record;
import org.apache.kafka.tiered.storage.TieredStorageTestContext;
import java.io.IOException;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ExecutionException;
import java.util.stream.Collectors;
public class ActionUtils {
public static TopicDescription describeTopic(TieredStorageTestContext context, String topic)
throws ExecutionException, InterruptedException {
return describeTopics(context, Collections.singletonList(topic)).get(topic);
}
public static Map<String, TopicDescription> describeTopics(TieredStorageTestContext context,
List<String> topics)
throws ExecutionException, InterruptedException {
return context.admin()
.describeTopics(topics)
.allTopicNames()
.get();
}
/**
* Get the records found in the local tiered storage.
* Snapshot does not sort the filesets by base offset.
* @param context The test context.
* @param topicPartition The topic-partition of the records.
* @return The records found in the local tiered storage.
*/
public static List<Record> tieredStorageRecords(TieredStorageTestContext context,
TopicPartition topicPartition) {
return context.takeTieredStorageSnapshot()
.getFilesets(topicPartition)
.stream()
.map(fileset -> {
try {
return fileset.getRecords();
} catch (IOException e) {
throw new RuntimeException(e);
}
})
.sorted(Comparator.comparingLong(records -> records.get(0).offset()))
.flatMap(Collection::stream)
.collect(Collectors.toList());
}
}

162
storage/src/test/java/org/apache/kafka/tiered/storage/utils/BrokerLocalStorage.java

@ -0,0 +1,162 @@ @@ -0,0 +1,162 @@
/*
* 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 org.apache.kafka.tiered.storage.utils;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.common.utils.Timer;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.storage.internals.log.LogFileUtils;
import java.io.File;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
import java.util.Optional;
import java.util.concurrent.TimeUnit;
import java.util.function.Function;
import java.util.stream.Collectors;
public final class BrokerLocalStorage {
private final Integer brokerId;
private final File brokerStorageDirectory;
private final Integer storageWaitTimeoutSec;
private final int storagePollPeriodSec = 1;
private final Time time = Time.SYSTEM;
public BrokerLocalStorage(Integer brokerId,
String storageDirname,
Integer storageWaitTimeoutSec) {
this.brokerId = brokerId;
this.brokerStorageDirectory = new File(storageDirname);
this.storageWaitTimeoutSec = storageWaitTimeoutSec;
}
public Integer getBrokerId() {
return brokerId;
}
/**
* Wait until the first segment offset in Apache Kafka storage for the given topic-partition is
* equal or greater to the provided offset.
* This ensures segments can be retrieved from the local tiered storage when expected.
*
* @param topicPartition The topic-partition to check.
* @param offset The offset to wait for.
* @throws AssertionError if the timeout is reached or the earliest-local offset is not equal to the provided
* offset.
*/
public void waitForEarliestLocalOffset(TopicPartition topicPartition,
Long offset) {
Function<OffsetHolder, Optional<String>> relativePosFunc = offsetHolder -> {
if (offsetHolder.firstLogFileBaseOffset < offset) {
return Optional.of("smaller than");
} else if (offsetHolder.firstLogFileBaseOffset > offset) {
return Optional.of("ahead of");
}
return Optional.empty();
};
waitForOffset(topicPartition, offset, relativePosFunc);
}
/**
* Wait until the first segment offset in Apache Kafka storage for the given topic-partition is
* equal or greater to the provided offset.
* This ensures segments can be retrieved from the local tiered storage when expected.
*
* @param topicPartition The topic-partition to check.
* @param offset The offset to wait for.
* @throws AssertionError if the timeout is reached or the earliest-local offset is lesser than to the provided
* offset.
*/
public void waitForAtLeastEarliestLocalOffset(TopicPartition topicPartition,
Long offset) {
Function<OffsetHolder, Optional<String>> relativePosFunc = offsetHolder -> {
if (offsetHolder.firstLogFileBaseOffset < offset) {
return Optional.of("smaller than");
}
return Optional.empty();
};
waitForOffset(topicPartition, offset, relativePosFunc);
}
private void waitForOffset(TopicPartition topicPartition,
Long offset,
Function<OffsetHolder, Optional<String>> relativePosFunc) {
Timer timer = time.timer(TimeUnit.SECONDS.toMillis(storageWaitTimeoutSec));
OffsetHolder offsetHolder = new OffsetHolder(0L, Collections.emptyList());
while (timer.notExpired() && offsetHolder.firstLogFileBaseOffset < offset) {
timer.sleep(TimeUnit.SECONDS.toMillis(storagePollPeriodSec));
offsetHolder = getEarliestLocalOffset(topicPartition);
}
Optional<String> relativePos = relativePosFunc.apply(offsetHolder);
if (relativePos.isPresent()) {
String pos = relativePos.get();
String message = String.format("[BrokerId=%d] The base offset of the first log segment of %s " +
"in the log directory is %d which is %s the expected offset %s. The directory of %s is " +
"made of the following files: %s", brokerId, topicPartition,
offsetHolder.firstLogFileBaseOffset, pos, offset, topicPartition,
Utils.join(offsetHolder.partitionFiles, System.lineSeparator()));
throw new AssertionError(message);
}
}
public void eraseStorage() throws IOException {
for (File file : Objects.requireNonNull(brokerStorageDirectory.listFiles())) {
Utils.delete(file);
}
}
private OffsetHolder getEarliestLocalOffset(TopicPartition topicPartition) {
List<String> partitionFiles = getTopicPartitionFiles(topicPartition);
Optional<String> firstLogFile = partitionFiles.stream()
.filter(filename -> filename.endsWith(LogFileUtils.LOG_FILE_SUFFIX))
.sorted()
.findFirst();
if (!firstLogFile.isPresent()) {
throw new IllegalArgumentException(String.format(
"[BrokerId=%d] No log file found for the topic-partition %s", brokerId, topicPartition));
}
return new OffsetHolder(LogFileUtils.offsetFromFileName(firstLogFile.get()), partitionFiles);
}
private List<String> getTopicPartitionFiles(TopicPartition topicPartition) {
File[] files = brokerStorageDirectory.listFiles((dir, name) -> name.equals(topicPartition.toString()));
if (files == null || files.length == 0) {
throw new IllegalArgumentException(String.format("[BrokerId=%d] Directory for the topic-partition %s " +
"was not found", brokerId, topicPartition));
}
File topicPartitionDir = files[0];
return Arrays.stream(Objects.requireNonNull(topicPartitionDir.listFiles()))
.map(File::getName)
.collect(Collectors.toList());
}
private static final class OffsetHolder {
private final long firstLogFileBaseOffset;
private final List<String> partitionFiles;
public OffsetHolder(long firstLogFileBaseOffset, List<String> partitionFiles) {
this.firstLogFileBaseOffset = firstLogFileBaseOffset;
this.partitionFiles = partitionFiles;
}
}
}

29
storage/src/test/java/org/apache/kafka/tiered/storage/utils/DumpLocalTieredStorage.java

@ -0,0 +1,29 @@ @@ -0,0 +1,29 @@
/*
* 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 org.apache.kafka.tiered.storage.utils;
import org.apache.kafka.common.serialization.Deserializer;
import org.apache.kafka.server.log.remote.storage.LocalTieredStorage;
public final class DumpLocalTieredStorage {
public static <K, V> String dump(LocalTieredStorage storage, Deserializer<K> keyDe, Deserializer<V> valueDe) {
LocalTieredStorageOutput<K, V> output = new LocalTieredStorageOutput<>(keyDe, valueDe);
storage.traverse(output);
return output.getOutput();
}
}

121
storage/src/test/java/org/apache/kafka/tiered/storage/utils/LocalTieredStorageOutput.java

@ -0,0 +1,121 @@ @@ -0,0 +1,121 @@
/*
* 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 org.apache.kafka.tiered.storage.utils;
import org.apache.kafka.common.TopicIdPartition;
import org.apache.kafka.common.record.Record;
import org.apache.kafka.common.serialization.Deserializer;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.server.log.remote.storage.LocalTieredStorageTraverser;
import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentFileset;
import java.nio.ByteBuffer;
import java.util.List;
import java.util.stream.Collectors;
import static org.apache.kafka.server.log.remote.storage.RemoteLogSegmentFileset.RemoteLogSegmentFileType.SEGMENT;
public final class LocalTieredStorageOutput<K, V> implements LocalTieredStorageTraverser {
private final Deserializer<K> keyDe;
private final Deserializer<V> valueDe;
private String output = row("File", "Offsets", "Records", "Broker ID");
private String currentTopic = "";
public LocalTieredStorageOutput(Deserializer<K> keyDe, Deserializer<V> valueDe) {
this.keyDe = keyDe;
this.valueDe = valueDe;
// Columns length + 5 column separators.
output += repeatString("-", 51 + 8 + 13 + 10 + (3 * 2)) + System.lineSeparator();
}
private String row(String file, Object offset, String record, String ident) {
return String.format("%-51s |%8s |%13s %n", ident + file, offset.toString(), record);
}
private String row(String file, Object offset, String record) {
return row(file, offset, record, " ");
}
private String row(String file, Object offset) {
return row(file, offset, "");
}
private String row(String file) {
return row(file, "", "");
}
private String row() {
return row("", "", "");
}
private String repeatString(String str, int times) {
StringBuilder builder = new StringBuilder();
for (int i = 0; i < times; i++) {
builder.append(str);
}
return builder.toString();
}
@Override
public void visitTopicIdPartition(TopicIdPartition topicIdPartition) {
currentTopic = topicIdPartition.topicPartition().topic();
output += row(topicIdPartition.topicPartition().toString(), "", "", "");
}
@Override
public void visitSegment(RemoteLogSegmentFileset fileset) {
try {
List<Record> records = fileset.getRecords();
String segFilename = fileset.getFile(SEGMENT).getName();
if (records.isEmpty()) {
output += row(segFilename, -1, "");
} else {
List<Tuple2<Long, String>> offsetKeyValues = records
.stream()
.map(record -> new Tuple2<>(record.offset(),
"(" + des(keyDe, record.key()) + ", " + des(valueDe, record.value()) + ")"))
.collect(Collectors.toList());
output += row(segFilename, offsetKeyValues.get(0).t1, offsetKeyValues.get(0).t2);
if (offsetKeyValues.size() > 1) {
offsetKeyValues.subList(1, records.size()).forEach(offsetKeyValue ->
output += row("", offsetKeyValue.t1, offsetKeyValue.t2));
}
}
output += row();
} catch (Exception ex) {
throw new RuntimeException(ex);
}
}
public String getOutput() {
return output;
}
private String des(Deserializer<?> de, ByteBuffer bytes) {
return de.deserialize(currentTopic, Utils.toNullableArray(bytes)).toString();
}
private static class Tuple2<T1, T2> {
private final T1 t1;
private final T2 t2;
Tuple2(T1 t1, T2 t2) {
this.t1 = t1;
this.t2 = t2;
}
}
}

180
storage/src/test/java/org/apache/kafka/tiered/storage/utils/RecordsKeyValueMatcher.java

@ -0,0 +1,180 @@ @@ -0,0 +1,180 @@
/*
* 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 org.apache.kafka.tiered.storage.utils;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.record.Record;
import org.apache.kafka.common.record.SimpleRecord;
import org.apache.kafka.common.serialization.Deserializer;
import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.common.utils.Utils;
import org.hamcrest.Description;
import org.hamcrest.TypeSafeDiagnosingMatcher;
import java.nio.ByteBuffer;
import java.util.Collection;
import java.util.Iterator;
public final class RecordsKeyValueMatcher<R1, R2, K, V> extends TypeSafeDiagnosingMatcher<Collection<R2>> {
private final Collection<R1> expectedRecords;
private final TopicPartition topicPartition;
private final Serde<K> keySerde;
private final Serde<V> valueSerde;
/**
* Heterogeneous matcher between alternative types of records:
* [[ProducerRecord]], [[ConsumerRecord]] or [[Record]].
*
* It is conceptually incorrect to try to match records of different natures.
* Only a committed [[Record]] is univoque, whereas a [[ProducerRecord]] or [[ConsumerRecord]] is
* a physical representation of a record-to-be or viewed record.
*
* This matcher breaches that semantic so that testers can avoid performing manual comparisons on
* targeted internal fields of these type of records. This implementation only compares key and
* value of the records.
*
* @param expectedRecords The records expected.
* @param topicPartition The topic-partition which the records belong to.
* @param keySerde The [[Serde]] for the keys of the records.
* @param valueSerde The [[Serde]] for the values of the records.
* @tparam R1 The type of records used to formulate the expectations.
* @tparam R2 The type of records compared against the expectations.
* @tparam K The type of the record keys.
* @tparam V The type of the record values.
*/
public RecordsKeyValueMatcher(Collection<R1> expectedRecords,
TopicPartition topicPartition,
Serde<K> keySerde,
Serde<V> valueSerde) {
this.expectedRecords = expectedRecords;
this.topicPartition = topicPartition;
this.keySerde = keySerde;
this.valueSerde = valueSerde;
}
@Override
public void describeTo(Description description) {
description.appendText("Records of ").appendValue(topicPartition).appendText(": ").appendValue(expectedRecords);
}
@Override
protected boolean matchesSafely(Collection<R2> actualRecords, Description mismatchDescription) {
if (expectedRecords.size() != actualRecords.size()) {
mismatchDescription.appendText("Number of records differ. Expected: ")
.appendValue(expectedRecords.size())
.appendText(", Actual: ")
.appendValue(actualRecords.size())
.appendText("; ");
return false;
}
Iterator<R1> expectedIterator = expectedRecords.iterator();
Iterator<R2> actualIterator = actualRecords.iterator();
while (expectedIterator.hasNext() && actualIterator.hasNext()) {
R1 expected = expectedIterator.next();
R2 actual = actualIterator.next();
if (!matches(expected, actual, mismatchDescription)) {
return false;
}
}
return true;
}
private boolean matches(R1 expected, R2 actual, Description mismatchDescription) {
SimpleRecord expectedRecord = convert(expected);
SimpleRecord actualRecord = convert(actual);
if (expectedRecord == null) {
mismatchDescription.appendText("Invalid expected record type: ")
.appendValue(expected.getClass().getSimpleName());
return false;
}
if (actualRecord == null) {
mismatchDescription.appendText("Invalid actual record type: ")
.appendValue(actual.getClass().getSimpleName());
return false;
}
if (!compare(expectedRecord.key(), actualRecord.key(), keySerde.deserializer(), "Record key",
mismatchDescription) ||
!compare(expectedRecord.value(), actualRecord.value(), valueSerde.deserializer(), "Record value",
mismatchDescription)) {
return false;
}
return true;
}
private boolean compare(ByteBuffer lhs,
ByteBuffer rhs,
Deserializer<?> deserializer,
String desc,
Description mismatchDescription) {
if ((lhs != null && !lhs.equals(rhs)) || (lhs == null && rhs != null)) {
mismatchDescription.appendText(desc).appendText(" mismatch. Expected: ")
.appendValue(deserializer.deserialize(topicPartition.topic(), Utils.toNullableArray(lhs)))
.appendText("; Actual: ")
.appendValue(deserializer.deserialize(topicPartition.topic(), Utils.toNullableArray(rhs)))
.appendText("; ");
return false;
}
return true;
}
@SuppressWarnings("unchecked")
private SimpleRecord convert(Object recordCandidate) {
if (recordCandidate instanceof ProducerRecord) {
ProducerRecord<?, ?> record = (ProducerRecord<?, ?>) recordCandidate;
ByteBuffer keyBytes =
Utils.wrapNullable(keySerde.serializer().serialize(topicPartition.topic(), (K) record.key()));
ByteBuffer valueBytes =
Utils.wrapNullable(valueSerde.serializer().serialize(topicPartition.topic(), (V) record.value()));
return new SimpleRecord(record.timestamp(), keyBytes, valueBytes, record.headers().toArray());
} else if (recordCandidate instanceof ConsumerRecord) {
ConsumerRecord<?, ?> record = (ConsumerRecord<?, ?>) recordCandidate;
ByteBuffer keyBytes =
Utils.wrapNullable(keySerde.serializer().serialize(topicPartition.topic(), (K) record.key()));
ByteBuffer valueBytes =
Utils.wrapNullable(valueSerde.serializer().serialize(topicPartition.topic(), (V) record.value()));
return new SimpleRecord(record.timestamp(), keyBytes, valueBytes, record.headers().toArray());
} else if (recordCandidate instanceof Record) {
Record record = (Record) recordCandidate;
return new SimpleRecord(record.timestamp(), record.key(), record.value(), record.headers());
} else {
return null;
}
}
/**
* Provides a matcher which compares the key and value of a sequence of records with those of
* the expectedRecords sequence, in order.
*
* @param expectedRecords The records expected.
* @param topicPartition The topic-partition which the records belong to.
* @param keySerde The [[Serde]] for the keys of the records.
* @param valueSerde The [[Serde]] for the values of the records.
* @tparam R1 The type of records used to formulate the expectations.
* @tparam R2 The type of records compared against the expectations.
* @tparam K The type of the record keys.
* @tparam V The type of the record values.
*/
public static <R1, R2, K, V> RecordsKeyValueMatcher<R1, R2, K, V> correspondTo(Collection<R1> expectedRecords,
TopicPartition topicPartition,
Serde<K> keySerde,
Serde<V> valueSerde) {
return new RecordsKeyValueMatcher<>(expectedRecords, topicPartition, keySerde, valueSerde);
}
}

6
storage/src/test/resources/log4j.properties

@ -18,5 +18,11 @@ log4j.appender.stdout=org.apache.log4j.ConsoleAppender @@ -18,5 +18,11 @@ log4j.appender.stdout=org.apache.log4j.ConsoleAppender
log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c:%L)%n
log4j.appender.fileAppender=org.apache.log4j.RollingFileAppender
log4j.appender.fileAppender.layout=org.apache.log4j.PatternLayout
log4j.appender.fileAppender.layout.ConversionPattern=%d [%t] %-5p %c %x - %m%n
log4j.appender.fileAppender.File=storage.log
log4j.logger.org.apache.kafka.server.log.remote.storage=INFO
log4j.logger.org.apache.kafka.server.log.remote.metadata.storage=INFO
log4j.logger.kafka.log.remote=INFO

Loading…
Cancel
Save