Browse Source
`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
51 changed files with 4238 additions and 32 deletions
@ -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> |
@ -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 |
@ -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); |
||||
} |
@ -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); |
||||
} |
||||
} |
||||
|
@ -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"); |
||||
} |
||||
} |
@ -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()); |
||||
} |
||||
} |
@ -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); |
||||
} |
||||
} |
@ -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); |
||||
} |
||||
} |
@ -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); |
||||
} |
||||
} |
@ -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); |
||||
} |
||||
} |
@ -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); |
||||
} |
||||
} |
@ -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); |
||||
} |
||||
} |
@ -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)); |
||||
} |
||||
} |
@ -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); |
||||
} |
||||
} |
@ -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); |
||||
} |
||||
} |
@ -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); |
||||
} |
||||
} |
@ -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); |
||||
} |
||||
} |
@ -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); |
||||
} |
||||
} |
@ -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); |
||||
} |
||||
} |
@ -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); |
||||
} |
||||
} |
@ -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); |
||||
} |
||||
} |
@ -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)); |
||||
} |
||||
} |
@ -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); |
||||
} |
||||
} |
@ -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); |
||||
} |
||||
} |
@ -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); |
||||
} |
||||
} |
@ -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); |
||||
} |
||||
} |
@ -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); |
||||
} |
||||
} |
@ -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); |
||||
} |
||||
} |
@ -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); |
||||
} |
||||
} |
@ -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); |
||||
} |
||||
} |
@ -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); |
||||
} |
||||
} |
@ -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); |
||||
} |
||||
} |
@ -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); |
||||
} |
||||
} |
@ -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); |
||||
} |
||||
} |
@ -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); |
||||
} |
||||
} |
@ -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); |
||||
} |
||||
} |
@ -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); |
||||
} |
||||
} |
@ -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); |
||||
} |
||||
} |
@ -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); |
||||
} |
||||
} |
@ -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); |
||||
} |
||||
} |
@ -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()); |
||||
} |
||||
} |
@ -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; |
||||
} |
||||
} |
||||
} |
@ -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(); |
||||
} |
||||
} |
@ -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; |
||||
} |
||||
} |
||||
} |
@ -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); |
||||
} |
||||
} |
Loading…
Reference in new issue