mirror of https://github.com/apache/kafka.git
Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread-without-addAndGet
This commit is contained in:
commit
524782cd79
|
@ -3350,6 +3350,7 @@ project(':jmh-benchmarks') {
|
|||
implementation project(':raft')
|
||||
implementation project(':clients')
|
||||
implementation project(':coordinator-common')
|
||||
implementation project(':coordinator-common').sourceSets.test.output
|
||||
implementation project(':group-coordinator')
|
||||
implementation project(':group-coordinator:group-coordinator-api')
|
||||
implementation project(':metadata')
|
||||
|
|
|
@ -52,9 +52,7 @@
|
|||
<allow pkg="org.apache.kafka.server"/>
|
||||
<allow pkg="org.apache.kafka.storage"/>
|
||||
<allow pkg="org.apache.kafka.clients"/>
|
||||
<allow class="org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage"/>
|
||||
<allow class="org.apache.kafka.coordinator.common.runtime.KRaftCoordinatorMetadataImage"/>
|
||||
<allow class="org.apache.kafka.coordinator.common.runtime.HdrHistogram"/>
|
||||
<allow pkg="org.apache.kafka.coordinator.common.runtime"/>
|
||||
<allow pkg="org.apache.kafka.coordinator.group"/>
|
||||
<allow pkg="org.apache.kafka.image"/>
|
||||
<allow pkg="org.apache.kafka.metadata"/>
|
||||
|
|
|
@ -107,6 +107,7 @@
|
|||
<allow pkg="org.apache.kafka.common.config" />
|
||||
<allow pkg="org.apache.kafka.common.message" />
|
||||
<allow pkg="org.apache.kafka.common.metadata" />
|
||||
<allow pkg="org.apache.kafka.common.network" />
|
||||
<allow pkg="org.apache.kafka.common.protocol" />
|
||||
<allow pkg="org.apache.kafka.common.quota" />
|
||||
<allow pkg="org.apache.kafka.common.record" />
|
||||
|
|
|
@ -27,7 +27,9 @@ import java.util.Collection;
|
|||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.function.BiConsumer;
|
||||
|
||||
import static org.apache.kafka.clients.ClientsTestUtils.consumeAndVerifyRecords;
|
||||
|
@ -166,6 +168,80 @@ public class PlaintextConsumerCallbackTest {
|
|||
testGetPositionOfNewlyAssignedPartitionOnPartitionsAssignedCallback(CLASSIC);
|
||||
}
|
||||
|
||||
@ClusterTest
|
||||
public void testOnPartitionsAssignedCalledWithNewPartitionsOnlyForClassicCooperative() throws InterruptedException {
|
||||
try (var consumer = createClassicConsumerCooperativeProtocol()) {
|
||||
testOnPartitionsAssignedCalledWithExpectedPartitions(consumer, true);
|
||||
}
|
||||
}
|
||||
|
||||
@ClusterTest
|
||||
public void testOnPartitionsAssignedCalledWithNewPartitionsOnlyForAsyncConsumer() throws InterruptedException {
|
||||
try (var consumer = createConsumer(CONSUMER)) {
|
||||
testOnPartitionsAssignedCalledWithExpectedPartitions(consumer, true);
|
||||
}
|
||||
}
|
||||
|
||||
@ClusterTest
|
||||
public void testOnPartitionsAssignedCalledWithNewPartitionsOnlyForClassicEager() throws InterruptedException {
|
||||
try (var consumer = createConsumer(CLASSIC)) {
|
||||
testOnPartitionsAssignedCalledWithExpectedPartitions(consumer, false);
|
||||
}
|
||||
}
|
||||
|
||||
private void testOnPartitionsAssignedCalledWithExpectedPartitions(
|
||||
Consumer<byte[], byte[]> consumer,
|
||||
boolean expectNewPartitionsOnlyInCallback) throws InterruptedException {
|
||||
subscribeAndExpectOnPartitionsAssigned(consumer, List.of(topic), List.of(tp));
|
||||
assertEquals(Set.of(tp), consumer.assignment());
|
||||
|
||||
// Add a new partition assignment while keeping the previous one
|
||||
String newTopic = "newTopic";
|
||||
TopicPartition addedPartition = new TopicPartition(newTopic, 0);
|
||||
List<TopicPartition> expectedPartitionsInCallback;
|
||||
if (expectNewPartitionsOnlyInCallback) {
|
||||
expectedPartitionsInCallback = List.of(addedPartition);
|
||||
} else {
|
||||
expectedPartitionsInCallback = List.of(tp, addedPartition);
|
||||
}
|
||||
|
||||
// Change subscription to keep the previous one and add a new topic. Assignment should be updated
|
||||
// to contain partitions from both topics, but the onPartitionsAssigned parameters may containing
|
||||
// the full new assignment or just the newly added partitions depending on the case.
|
||||
subscribeAndExpectOnPartitionsAssigned(
|
||||
consumer,
|
||||
List.of(topic, newTopic),
|
||||
expectedPartitionsInCallback);
|
||||
assertEquals(Set.of(tp, addedPartition), consumer.assignment());
|
||||
}
|
||||
|
||||
private void subscribeAndExpectOnPartitionsAssigned(Consumer<byte[], byte[]> consumer, List<String> topics, Collection<TopicPartition> expectedPartitionsInCallback) throws InterruptedException {
|
||||
var partitionsAssigned = new AtomicBoolean(false);
|
||||
AtomicReference<Collection<TopicPartition>> partitionsFromCallback = new AtomicReference<>();
|
||||
consumer.subscribe(topics, new ConsumerRebalanceListener() {
|
||||
@Override
|
||||
public void onPartitionsAssigned(Collection<TopicPartition> partitions) {
|
||||
if (partitions.containsAll(expectedPartitionsInCallback)) {
|
||||
partitionsFromCallback.set(partitions);
|
||||
partitionsAssigned.set(true);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onPartitionsRevoked(Collection<TopicPartition> partitions) {
|
||||
// noop
|
||||
}
|
||||
});
|
||||
ClientsTestUtils.pollUntilTrue(
|
||||
consumer,
|
||||
partitionsAssigned::get,
|
||||
"Timed out before expected rebalance completed"
|
||||
);
|
||||
// These are different types, so comparing values instead
|
||||
assertTrue(expectedPartitionsInCallback.containsAll(partitionsFromCallback.get()) && partitionsFromCallback.get().containsAll(expectedPartitionsInCallback),
|
||||
"Expected partitions " + expectedPartitionsInCallback + " as parameter for onPartitionsAssigned, but got " + partitionsFromCallback.get());
|
||||
}
|
||||
|
||||
@ClusterTest
|
||||
public void testAsyncConsumerGetPositionOfNewlyAssignedPartitionOnPartitionsAssignedCallback() throws InterruptedException {
|
||||
testGetPositionOfNewlyAssignedPartitionOnPartitionsAssignedCallback(CONSUMER);
|
||||
|
@ -284,4 +360,12 @@ public class PlaintextConsumerCallbackTest {
|
|||
ENABLE_AUTO_COMMIT_CONFIG, "false"
|
||||
));
|
||||
}
|
||||
|
||||
private Consumer<byte[], byte[]> createClassicConsumerCooperativeProtocol() {
|
||||
return cluster.consumer(Map.of(
|
||||
GROUP_PROTOCOL_CONFIG, CLASSIC.name.toLowerCase(Locale.ROOT),
|
||||
ENABLE_AUTO_COMMIT_CONFIG, "false",
|
||||
ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, "org.apache.kafka.clients.consumer.CooperativeStickyAssignor"
|
||||
));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,7 +22,7 @@ import org.apache.kafka.common.annotation.InterfaceStability;
|
|||
import java.util.Collection;
|
||||
|
||||
/**
|
||||
* Options for the {@link Admin#deleteShareGroups(Collection <String>, DeleteShareGroupsOptions)} call.
|
||||
* Options for the {@link Admin#deleteShareGroups(Collection, DeleteShareGroupsOptions)} call.
|
||||
* <p>
|
||||
* The API of this class is evolving, see {@link Admin} for details.
|
||||
*/
|
||||
|
|
|
@ -25,7 +25,7 @@ import java.util.HashMap;
|
|||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* The result of the {@link Admin#deleteShareGroups(Collection <String>, DeleteShareGroupsOptions)} call.
|
||||
* The result of the {@link Admin#deleteShareGroups(Collection, DeleteShareGroupsOptions)} call.
|
||||
* <p></p>
|
||||
* The API of this class is evolving, see {@link Admin} for details.
|
||||
*/
|
||||
|
|
|
@ -21,7 +21,7 @@ import org.apache.kafka.common.annotation.InterfaceStability;
|
|||
import java.util.Collection;
|
||||
|
||||
/**
|
||||
* Options for the {@link Admin#deleteStreamsGroups(Collection<String>, DeleteStreamsGroupsOptions)} call.
|
||||
* Options for the {@link Admin#deleteStreamsGroups(Collection, DeleteStreamsGroupsOptions)} call.
|
||||
* <p>
|
||||
* The API of this class is evolving, see {@link Admin} for details.
|
||||
*/
|
||||
|
|
|
@ -21,6 +21,7 @@ import org.apache.kafka.common.TopicPartition;
|
|||
import org.apache.kafka.common.annotation.InterfaceStability;
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Specification of streams group offsets to list using {@link Admin#listStreamsGroupOffsets(Map, ListStreamsGroupOffsetsOptions)}.
|
||||
|
|
|
@ -33,14 +33,16 @@ public class RecordsToDelete {
|
|||
/**
|
||||
* Delete all the records before the given {@code offset}
|
||||
*
|
||||
* @param offset the offset before which all records will be deleted
|
||||
* @param offset The offset before which all records will be deleted.
|
||||
* Use {@code -1} to truncate to the high watermark.
|
||||
*/
|
||||
public static RecordsToDelete beforeOffset(long offset) {
|
||||
return new RecordsToDelete(offset);
|
||||
}
|
||||
|
||||
/**
|
||||
* The offset before which all records will be deleted
|
||||
* The offset before which all records will be deleted.
|
||||
* Use {@code -1} to truncate to the high watermark.
|
||||
*/
|
||||
public long beforeOffset() {
|
||||
return offset;
|
||||
|
|
|
@ -122,8 +122,8 @@ public class DeleteShareGroupOffsetsHandler extends AdminApiHandler.Batched<Coor
|
|||
if (topic.errorCode() != Errors.NONE.code()) {
|
||||
final Errors topicError = Errors.forCode(topic.errorCode());
|
||||
final String topicErrorMessage = topic.errorMessage();
|
||||
log.debug("DeleteShareGroupOffsets request for group id {} and topic {} failed and returned error {}." + topicErrorMessage,
|
||||
groupId.idValue, topic.topicName(), topicError);
|
||||
log.debug("DeleteShareGroupOffsets request for group id {} and topic {} failed and returned error {}. {}",
|
||||
groupId.idValue, topic.topicName(), topicError, topicErrorMessage);
|
||||
}
|
||||
topicResults.put(
|
||||
topic.topicName(),
|
||||
|
@ -147,14 +147,14 @@ public class DeleteShareGroupOffsetsHandler extends AdminApiHandler.Batched<Coor
|
|||
case REBALANCE_IN_PROGRESS:
|
||||
// If the coordinator is in the middle of loading, then we just need to retry
|
||||
log.debug("DeleteShareGroupOffsets request for group id {} failed because the coordinator" +
|
||||
" is still in the process of loading state. Will retry. " + errorMessage, groupId.idValue);
|
||||
" is still in the process of loading state. Will retry. {}", groupId.idValue, errorMessage);
|
||||
break;
|
||||
case COORDINATOR_NOT_AVAILABLE:
|
||||
case NOT_COORDINATOR:
|
||||
// If the coordinator is unavailable or there was a coordinator change, then we unmap
|
||||
// the key so that we retry the `FindCoordinator` request
|
||||
log.debug("DeleteShareGroupOffsets request for group id {} returned error {}. Will rediscover the coordinator and retry. " + errorMessage,
|
||||
groupId.idValue, error);
|
||||
log.debug("DeleteShareGroupOffsets request for group id {} returned error {}. Will rediscover the coordinator and retry. {}",
|
||||
groupId.idValue, error, errorMessage);
|
||||
groupsToUnmap.add(groupId);
|
||||
break;
|
||||
case INVALID_GROUP_ID:
|
||||
|
@ -164,11 +164,11 @@ public class DeleteShareGroupOffsetsHandler extends AdminApiHandler.Batched<Coor
|
|||
case UNKNOWN_SERVER_ERROR:
|
||||
case KAFKA_STORAGE_ERROR:
|
||||
case GROUP_AUTHORIZATION_FAILED:
|
||||
log.debug("DeleteShareGroupOffsets request for group id {} failed due to error {}. " + errorMessage, groupId.idValue, error);
|
||||
log.debug("DeleteShareGroupOffsets request for group id {} failed due to error {}. {}", groupId.idValue, error, errorMessage);
|
||||
failed.put(groupId, error.exception(errorMessage));
|
||||
break;
|
||||
default:
|
||||
log.error("DeleteShareGroupOffsets request for group id {} failed due to unexpected error {}. " + errorMessage, groupId.idValue, error);
|
||||
log.error("DeleteShareGroupOffsets request for group id {} failed due to unexpected error {}. {}", groupId.idValue, error, errorMessage);
|
||||
failed.put(groupId, error.exception(errorMessage));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -202,7 +202,7 @@ public final class ListOffsetsHandler extends Batched<TopicPartition, ListOffset
|
|||
public Map<TopicPartition, Throwable> handleUnsupportedVersionException(
|
||||
int brokerId, UnsupportedVersionException exception, Set<TopicPartition> keys
|
||||
) {
|
||||
log.warn("Broker " + brokerId + " does not support MAX_TIMESTAMP offset specs");
|
||||
log.warn("Broker {} does not support MAX_TIMESTAMP offset specs", brokerId);
|
||||
Map<TopicPartition, Throwable> maxTimestampPartitions = new HashMap<>();
|
||||
for (TopicPartition topicPartition : keys) {
|
||||
Long offsetTimestamp = offsetTimestampsByPartition.get(topicPartition);
|
||||
|
|
|
@ -50,7 +50,7 @@ import java.util.Collection;
|
|||
* Under normal conditions, if a partition is reassigned from one consumer to another, then the old consumer will
|
||||
* always invoke {@link #onPartitionsRevoked(Collection) onPartitionsRevoked} for that partition prior to the new consumer
|
||||
* invoking {@link #onPartitionsAssigned(Collection) onPartitionsAssigned} for the same partition. So if offsets or other state is saved in the
|
||||
* {@link #onPartitionsRevoked(Collection) onPartitionsRevoked} call by one consumer member, it will be always accessible by the time the
|
||||
* {@link #onPartitionsRevoked(Collection) onPartitionsRevoked} call by one consumer member, it will always be accessible by the time the
|
||||
* other consumer member taking over that partition and triggering its {@link #onPartitionsAssigned(Collection) onPartitionsAssigned} callback to load the state.
|
||||
* <p>
|
||||
* You can think of revocation as a graceful way to give up ownership of a partition. In some cases, the consumer may not have an opportunity to do so.
|
||||
|
@ -120,13 +120,31 @@ public interface ConsumerRebalanceListener {
|
|||
/**
|
||||
* A callback method the user can implement to provide handling of offset commits to a customized store.
|
||||
* This method will be called during a rebalance operation when the consumer has to give up some partitions.
|
||||
* It can also be called when consumer is being closed ({@link KafkaConsumer#close(CloseOptions option)})
|
||||
* or is unsubscribing ({@link KafkaConsumer#unsubscribe()}).
|
||||
* The consumer may need to give up some partitions (thus this callback executed) under the following scenarios:
|
||||
* <ul>
|
||||
* <li>If the consumer assignment changes</li>
|
||||
* <li>If the consumer is being closed ({@link KafkaConsumer#close(CloseOptions option)})</li>
|
||||
* <li>If the consumer is unsubscribing ({@link KafkaConsumer#unsubscribe()})</li>
|
||||
* </ul>
|
||||
* It is recommended that offsets should be committed in this callback to either Kafka or a
|
||||
* custom offset store to prevent duplicate data.
|
||||
* <p>
|
||||
* In eager rebalancing, it will always be called at the start of a rebalance and after the consumer stops fetching data.
|
||||
* In cooperative rebalancing, it will be called at the end of a rebalance on the set of partitions being revoked iff the set is non-empty.
|
||||
* This callback is always called before re-assigning the partitions.
|
||||
* If the consumer is using the {@link GroupProtocol#CLASSIC} rebalance protocol:
|
||||
* <ul>
|
||||
* <li>
|
||||
* In eager rebalancing, onPartitionsRevoked will be called with the full set of assigned partitions as a parameter (all partitions are revoked).
|
||||
* It will be called even if there are no partitions to revoke.
|
||||
* </li>
|
||||
* <li>
|
||||
* In cooperative rebalancing, onPartitionsRevoked will be called with the set of partitions to revoke,
|
||||
* iff the set is non-empty.
|
||||
* </li>
|
||||
* </ul>
|
||||
* If the consumer is using the {@link GroupProtocol#CONSUMER} rebalance protocol, this callback will be called
|
||||
* with the set of partitions to revoke iff the set is non-empty
|
||||
* (same behavior as the {@link GroupProtocol#CLASSIC} rebalance protocol with Cooperative mode).
|
||||
* <p>
|
||||
* For examples on usage of this API, see Usage Examples section of {@link KafkaConsumer KafkaConsumer}.
|
||||
* <p>
|
||||
* It is common for the revocation callback to use the consumer instance in order to commit offsets. It is possible
|
||||
|
@ -135,8 +153,9 @@ public interface ConsumerRebalanceListener {
|
|||
* invocation of {@link KafkaConsumer#poll(java.time.Duration)} in which this callback is being executed. This means it is not
|
||||
* necessary to catch these exceptions and re-attempt to wakeup or interrupt the consumer thread.
|
||||
*
|
||||
* @param partitions The list of partitions that were assigned to the consumer and now need to be revoked (may not
|
||||
* include all currently assigned partitions, i.e. there may still be some partitions left)
|
||||
* @param partitions The list of partitions that were assigned to the consumer and now need to be revoked. This will
|
||||
* include the full assignment under the Classic/Eager protocol, given that it revokes all partitions.
|
||||
* It will only include the subset to revoke under the Classic/Cooperative and Consumer protocols.
|
||||
* @throws org.apache.kafka.common.errors.WakeupException If raised from a nested call to {@link KafkaConsumer}
|
||||
* @throws org.apache.kafka.common.errors.InterruptException If raised from a nested call to {@link KafkaConsumer}
|
||||
*/
|
||||
|
@ -144,12 +163,13 @@ public interface ConsumerRebalanceListener {
|
|||
|
||||
/**
|
||||
* A callback method the user can implement to provide handling of customized offsets on completion of a successful
|
||||
* partition re-assignment. This method will be called after the partition re-assignment completes and before the
|
||||
* consumer starts fetching data, and only as the result of a {@link Consumer#poll(java.time.Duration) poll(long)} call.
|
||||
* partition re-assignment. This method will be called after the partition re-assignment completes (even if no new
|
||||
* partitions were assigned to the consumer), and before the consumer starts fetching data,
|
||||
* and only as the result of a {@link Consumer#poll(java.time.Duration) poll(long)} call.
|
||||
* <p>
|
||||
* It is guaranteed that under normal conditions all the processes in a consumer group will execute their
|
||||
* {@link #onPartitionsRevoked(Collection)} callback before any instance executes its
|
||||
* {@link #onPartitionsAssigned(Collection)} callback. During exceptional scenarios, partitions may be migrated
|
||||
* {@link #onPartitionsRevoked(Collection)} callback before any instance executes this onPartitionsAssigned callback.
|
||||
* During exceptional scenarios, partitions may be migrated
|
||||
* without the old owner being notified (i.e. their {@link #onPartitionsRevoked(Collection)} callback not triggered),
|
||||
* and later when the old owner consumer realized this event, the {@link #onPartitionsLost(Collection)} callback
|
||||
* will be triggered by the consumer then.
|
||||
|
@ -160,8 +180,10 @@ public interface ConsumerRebalanceListener {
|
|||
* invocation of {@link KafkaConsumer#poll(java.time.Duration)} in which this callback is being executed. This means it is not
|
||||
* necessary to catch these exceptions and re-attempt to wakeup or interrupt the consumer thread.
|
||||
*
|
||||
* @param partitions The list of partitions that are now assigned to the consumer (previously owned partitions will
|
||||
* NOT be included, i.e. this list will only include newly added partitions)
|
||||
* @param partitions Partitions that have been added to the assignment as a result of the rebalance.
|
||||
* Note that partitions that were already owned by this consumer and remain assigned are not
|
||||
* included in this list under the Classic/Cooperative or Consumer protocols. THe full assignment
|
||||
* will be received under the Classic/Eager protocol.
|
||||
* @throws org.apache.kafka.common.errors.WakeupException If raised from a nested call to {@link KafkaConsumer}
|
||||
* @throws org.apache.kafka.common.errors.InterruptException If raised from a nested call to {@link KafkaConsumer}
|
||||
*/
|
||||
|
@ -187,9 +209,8 @@ public interface ConsumerRebalanceListener {
|
|||
* necessary to catch these exceptions and re-attempt to wakeup or interrupt the consumer thread.
|
||||
*
|
||||
* @param partitions The list of partitions that were assigned to the consumer and now have been reassigned
|
||||
* to other consumers. With the current protocol this will always include all of the consumer's
|
||||
* previously assigned partitions, but this may change in future protocols (ie there would still
|
||||
* be some partitions left)
|
||||
* to other consumers. With both, the Classic and Consumer protocols, this will always include
|
||||
* all partitions that were previously assigned to the consumer.
|
||||
* @throws org.apache.kafka.common.errors.WakeupException If raised from a nested call to {@link KafkaConsumer}
|
||||
* @throws org.apache.kafka.common.errors.InterruptException If raised from a nested call to {@link KafkaConsumer}
|
||||
*/
|
||||
|
|
|
@ -661,7 +661,7 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
|
|||
* If the given list of topics is empty, it is treated the same as {@link #unsubscribe()}.
|
||||
*
|
||||
* <p>
|
||||
* As part of group management, the consumer will keep track of the list of consumers that belong to a particular
|
||||
* As part of group management, the group coordinator will keep track of the list of consumers that belong to a particular
|
||||
* group and will trigger a rebalance operation if any one of the following events are triggered:
|
||||
* <ul>
|
||||
* <li>Number of partitions change for any of the subscribed topics
|
||||
|
@ -670,8 +670,11 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
|
|||
* <li>A new member is added to the consumer group
|
||||
* </ul>
|
||||
* <p>
|
||||
* When any of these events are triggered, the provided listener will be invoked first to indicate that
|
||||
* the consumer's assignment has been revoked, and then again when the new assignment has been received.
|
||||
* When any of these events are triggered, the provided listener will be invoked in this way:
|
||||
* <ul>
|
||||
* <li>{@link ConsumerRebalanceListener#onPartitionsRevoked(Collection)} will be invoked with the partitions to revoke, before re-assigning those partitions to another consumer.</li>
|
||||
* <li>{@link ConsumerRebalanceListener#onPartitionsAssigned(Collection)} will be invoked when the rebalance completes (even if no new partitions are assigned to the consumer)</li>
|
||||
* </ul>
|
||||
* Note that rebalances will only occur during an active call to {@link #poll(Duration)}, so callbacks will
|
||||
* also only be invoked during that time.
|
||||
*
|
||||
|
|
|
@ -988,7 +988,7 @@ public abstract class AbstractMembershipManager<R extends AbstractResponse> impl
|
|||
String reason = rejoinedWhileReconciliationInProgress ?
|
||||
"the member has re-joined the group" :
|
||||
"the member already transitioned out of the reconciling state into " + state;
|
||||
log.info("Interrupting reconciliation that is not relevant anymore because " + reason);
|
||||
log.info("Interrupting reconciliation that is not relevant anymore because {}", reason);
|
||||
markReconciliationCompleted();
|
||||
}
|
||||
return shouldAbort;
|
||||
|
|
|
@ -965,8 +965,8 @@ public abstract class AbstractStickyAssignor extends AbstractPartitionAssignor {
|
|||
super(partitionsPerTopic, rackInfo, currentAssignment);
|
||||
this.subscriptions = subscriptions;
|
||||
|
||||
topic2AllPotentialConsumers = new HashMap<>(partitionsPerTopic.keySet().size());
|
||||
consumer2AllPotentialTopics = new HashMap<>(subscriptions.keySet().size());
|
||||
topic2AllPotentialConsumers = new HashMap<>(partitionsPerTopic.size());
|
||||
consumer2AllPotentialTopics = new HashMap<>(subscriptions.size());
|
||||
|
||||
// initialize topic2AllPotentialConsumers and consumer2AllPotentialTopics
|
||||
partitionsPerTopic.keySet().forEach(
|
||||
|
|
|
@ -187,25 +187,6 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
|
|||
*/
|
||||
private class BackgroundEventProcessor implements EventProcessor<BackgroundEvent> {
|
||||
|
||||
private Optional<StreamsRebalanceListener> streamsRebalanceListener = Optional.empty();
|
||||
private final Optional<StreamsRebalanceData> streamsRebalanceData;
|
||||
|
||||
public BackgroundEventProcessor() {
|
||||
this.streamsRebalanceData = Optional.empty();
|
||||
}
|
||||
|
||||
public BackgroundEventProcessor(final Optional<StreamsRebalanceData> streamsRebalanceData) {
|
||||
this.streamsRebalanceData = streamsRebalanceData;
|
||||
}
|
||||
|
||||
private void setStreamsRebalanceListener(final StreamsRebalanceListener streamsRebalanceListener) {
|
||||
if (streamsRebalanceData.isEmpty()) {
|
||||
throw new IllegalStateException("Background event processor was not created to be used with Streams " +
|
||||
"rebalance protocol events");
|
||||
}
|
||||
this.streamsRebalanceListener = Optional.of(streamsRebalanceListener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void process(final BackgroundEvent event) {
|
||||
switch (event.type()) {
|
||||
|
@ -278,44 +259,26 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
|
|||
|
||||
private StreamsOnTasksRevokedCallbackCompletedEvent invokeOnTasksRevokedCallback(final Set<StreamsRebalanceData.TaskId> activeTasksToRevoke,
|
||||
final CompletableFuture<Void> future) {
|
||||
final Optional<Exception> exceptionFromCallback = streamsRebalanceListener().onTasksRevoked(activeTasksToRevoke);
|
||||
final Optional<Exception> exceptionFromCallback = Optional.ofNullable(streamsRebalanceListenerInvoker().invokeTasksRevoked(activeTasksToRevoke));
|
||||
final Optional<KafkaException> error = exceptionFromCallback.map(e -> ConsumerUtils.maybeWrapAsKafkaException(e, "Task revocation callback throws an error"));
|
||||
return new StreamsOnTasksRevokedCallbackCompletedEvent(future, error);
|
||||
}
|
||||
|
||||
private StreamsOnTasksAssignedCallbackCompletedEvent invokeOnTasksAssignedCallback(final StreamsRebalanceData.Assignment assignment,
|
||||
final CompletableFuture<Void> future) {
|
||||
final Optional<KafkaException> error;
|
||||
final Optional<Exception> exceptionFromCallback = streamsRebalanceListener().onTasksAssigned(assignment);
|
||||
if (exceptionFromCallback.isPresent()) {
|
||||
error = Optional.of(ConsumerUtils.maybeWrapAsKafkaException(exceptionFromCallback.get(), "Task assignment callback throws an error"));
|
||||
} else {
|
||||
error = Optional.empty();
|
||||
streamsRebalanceData().setReconciledAssignment(assignment);
|
||||
}
|
||||
final Optional<Exception> exceptionFromCallback = Optional.ofNullable(streamsRebalanceListenerInvoker().invokeTasksAssigned(assignment));
|
||||
final Optional<KafkaException> error = exceptionFromCallback.map(e -> ConsumerUtils.maybeWrapAsKafkaException(e, "Task assignment callback throws an error"));
|
||||
return new StreamsOnTasksAssignedCallbackCompletedEvent(future, error);
|
||||
}
|
||||
|
||||
private StreamsOnAllTasksLostCallbackCompletedEvent invokeOnAllTasksLostCallback(final CompletableFuture<Void> future) {
|
||||
final Optional<KafkaException> error;
|
||||
final Optional<Exception> exceptionFromCallback = streamsRebalanceListener().onAllTasksLost();
|
||||
if (exceptionFromCallback.isPresent()) {
|
||||
error = Optional.of(ConsumerUtils.maybeWrapAsKafkaException(exceptionFromCallback.get(), "All tasks lost callback throws an error"));
|
||||
} else {
|
||||
error = Optional.empty();
|
||||
streamsRebalanceData().setReconciledAssignment(StreamsRebalanceData.Assignment.EMPTY);
|
||||
}
|
||||
final Optional<Exception> exceptionFromCallback = Optional.ofNullable(streamsRebalanceListenerInvoker().invokeAllTasksLost());
|
||||
final Optional<KafkaException> error = exceptionFromCallback.map(e -> ConsumerUtils.maybeWrapAsKafkaException(e, "All tasks lost callback throws an error"));
|
||||
return new StreamsOnAllTasksLostCallbackCompletedEvent(future, error);
|
||||
}
|
||||
|
||||
private StreamsRebalanceData streamsRebalanceData() {
|
||||
return streamsRebalanceData.orElseThrow(
|
||||
() -> new IllegalStateException("Background event processor was not created to be used with Streams " +
|
||||
"rebalance protocol events"));
|
||||
}
|
||||
|
||||
private StreamsRebalanceListener streamsRebalanceListener() {
|
||||
return streamsRebalanceListener.orElseThrow(
|
||||
private StreamsRebalanceListenerInvoker streamsRebalanceListenerInvoker() {
|
||||
return streamsRebalanceListenerInvoker.orElseThrow(
|
||||
() -> new IllegalStateException("Background event processor was not created to be used with Streams " +
|
||||
"rebalance protocol events"));
|
||||
}
|
||||
|
@ -365,6 +328,7 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
|
|||
private final WakeupTrigger wakeupTrigger = new WakeupTrigger();
|
||||
private final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker;
|
||||
private final ConsumerRebalanceListenerInvoker rebalanceListenerInvoker;
|
||||
private final Optional<StreamsRebalanceListenerInvoker> streamsRebalanceListenerInvoker;
|
||||
// Last triggered async commit future. Used to wait until all previous async commits are completed.
|
||||
// We only need to keep track of the last one, since they are guaranteed to complete in order.
|
||||
private CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> lastPendingAsyncCommit = null;
|
||||
|
@ -520,7 +484,9 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
|
|||
time,
|
||||
new RebalanceCallbackMetricsManager(metrics)
|
||||
);
|
||||
this.backgroundEventProcessor = new BackgroundEventProcessor(streamsRebalanceData);
|
||||
this.streamsRebalanceListenerInvoker = streamsRebalanceData.map(s ->
|
||||
new StreamsRebalanceListenerInvoker(logContext, s));
|
||||
this.backgroundEventProcessor = new BackgroundEventProcessor();
|
||||
this.backgroundEventReaper = backgroundEventReaperFactory.build(logContext);
|
||||
|
||||
// The FetchCollector is only used on the application thread.
|
||||
|
@ -580,6 +546,7 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
|
|||
this.time = time;
|
||||
this.backgroundEventQueue = backgroundEventQueue;
|
||||
this.rebalanceListenerInvoker = rebalanceListenerInvoker;
|
||||
this.streamsRebalanceListenerInvoker = Optional.empty();
|
||||
this.backgroundEventProcessor = new BackgroundEventProcessor();
|
||||
this.backgroundEventReaper = backgroundEventReaper;
|
||||
this.metrics = metrics;
|
||||
|
@ -706,6 +673,7 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
|
|||
networkClientDelegateSupplier,
|
||||
requestManagersSupplier,
|
||||
asyncConsumerMetrics);
|
||||
this.streamsRebalanceListenerInvoker = Optional.empty();
|
||||
this.backgroundEventProcessor = new BackgroundEventProcessor();
|
||||
this.backgroundEventReaper = new CompletableEventReaper(logContext);
|
||||
}
|
||||
|
@ -1493,7 +1461,7 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
|
|||
() -> autoCommitOnClose(closeTimer), firstException);
|
||||
swallow(log, Level.ERROR, "Failed to stop finding coordinator",
|
||||
this::stopFindCoordinatorOnClose, firstException);
|
||||
swallow(log, Level.ERROR, "Failed to release group assignment",
|
||||
swallow(log, Level.ERROR, "Failed to run rebalance callbacks",
|
||||
this::runRebalanceCallbacksOnClose, firstException);
|
||||
swallow(log, Level.ERROR, "Failed to leave group while closing consumer",
|
||||
() -> leaveGroupOnClose(closeTimer, membershipOperation), firstException);
|
||||
|
@ -1543,11 +1511,23 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
|
|||
}
|
||||
|
||||
private void runRebalanceCallbacksOnClose() {
|
||||
if (groupMetadata.get().isEmpty() || rebalanceListenerInvoker == null)
|
||||
if (groupMetadata.get().isEmpty())
|
||||
return;
|
||||
|
||||
int memberEpoch = groupMetadata.get().get().generationId();
|
||||
|
||||
Exception error = null;
|
||||
|
||||
if (streamsRebalanceListenerInvoker != null && streamsRebalanceListenerInvoker.isPresent()) {
|
||||
|
||||
if (memberEpoch > 0) {
|
||||
error = streamsRebalanceListenerInvoker.get().invokeAllTasksRevoked();
|
||||
} else {
|
||||
error = streamsRebalanceListenerInvoker.get().invokeAllTasksLost();
|
||||
}
|
||||
|
||||
} else if (rebalanceListenerInvoker != null) {
|
||||
|
||||
Set<TopicPartition> assignedPartitions = groupAssignmentSnapshot.get();
|
||||
|
||||
if (assignedPartitions.isEmpty())
|
||||
|
@ -1557,12 +1537,13 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
|
|||
SortedSet<TopicPartition> droppedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR);
|
||||
droppedPartitions.addAll(assignedPartitions);
|
||||
|
||||
final Exception error;
|
||||
|
||||
if (memberEpoch > 0)
|
||||
if (memberEpoch > 0) {
|
||||
error = rebalanceListenerInvoker.invokePartitionsRevoked(droppedPartitions);
|
||||
else
|
||||
} else {
|
||||
error = rebalanceListenerInvoker.invokePartitionsLost(droppedPartitions);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
if (error != null)
|
||||
throw ConsumerUtils.maybeWrapAsKafkaException(error);
|
||||
|
@ -1926,8 +1907,12 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
|
|||
}
|
||||
|
||||
public void subscribe(Collection<String> topics, StreamsRebalanceListener streamsRebalanceListener) {
|
||||
|
||||
streamsRebalanceListenerInvoker
|
||||
.orElseThrow(() -> new IllegalStateException("Consumer was not created to be used with Streams rebalance protocol events"))
|
||||
.setRebalanceListener(streamsRebalanceListener);
|
||||
|
||||
subscribeInternal(topics, Optional.empty());
|
||||
backgroundEventProcessor.setStreamsRebalanceListener(streamsRebalanceListener);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -541,7 +541,7 @@ public class CommitRequestManager implements RequestManager, MemberStateListener
|
|||
boolean inflightRemoved = pendingRequests.inflightOffsetFetches.remove(fetchRequest);
|
||||
if (!inflightRemoved) {
|
||||
log.warn("A duplicated, inflight, request was identified, but unable to find it in the " +
|
||||
"outbound buffer:" + fetchRequest);
|
||||
"outbound buffer: {}", fetchRequest);
|
||||
}
|
||||
if (error == null) {
|
||||
maybeUpdateLastSeenEpochIfNewer(res);
|
||||
|
|
|
@ -163,7 +163,7 @@ public class ConsumerMembershipManager extends AbstractMembershipManager<Consume
|
|||
logContext,
|
||||
backgroundEventHandler,
|
||||
time,
|
||||
new ConsumerRebalanceMetricsManager(metrics),
|
||||
new ConsumerRebalanceMetricsManager(metrics, subscriptions),
|
||||
autoCommitEnabled);
|
||||
}
|
||||
|
||||
|
|
|
@ -287,7 +287,7 @@ public class StreamsGroupHeartbeatRequestManager implements RequestManager {
|
|||
|
||||
private final HeartbeatMetricsManager metricsManager;
|
||||
|
||||
private StreamsRebalanceData streamsRebalanceData;
|
||||
private final StreamsRebalanceData streamsRebalanceData;
|
||||
|
||||
/**
|
||||
* Timer for tracking the time since the last consumer poll. If the timer expires, the consumer will stop
|
||||
|
|
|
@ -296,7 +296,7 @@ public class StreamsMembershipManager implements RequestManager {
|
|||
this.backgroundEventHandler = backgroundEventHandler;
|
||||
this.streamsRebalanceData = streamsRebalanceData;
|
||||
this.subscriptionState = subscriptionState;
|
||||
metricsManager = new ConsumerRebalanceMetricsManager(metrics);
|
||||
metricsManager = new ConsumerRebalanceMetricsManager(metrics, subscriptionState);
|
||||
this.time = time;
|
||||
}
|
||||
|
||||
|
@ -1010,8 +1010,8 @@ public class StreamsMembershipManager implements RequestManager {
|
|||
return;
|
||||
}
|
||||
if (reconciliationInProgress) {
|
||||
log.trace("Ignoring reconciliation attempt. Another reconciliation is already in progress. Assignment " +
|
||||
targetAssignment + " will be handled in the next reconciliation loop.");
|
||||
log.trace("Ignoring reconciliation attempt. Another reconciliation is already in progress. Assignment {}" +
|
||||
" will be handled in the next reconciliation loop.", targetAssignment);
|
||||
return;
|
||||
}
|
||||
|
||||
|
@ -1131,12 +1131,12 @@ public class StreamsMembershipManager implements RequestManager {
|
|||
);
|
||||
|
||||
final SortedSet<TopicPartition> partitionsToAssign = topicPartitionsForActiveTasks(activeTasksToAssign);
|
||||
final SortedSet<TopicPartition> partitionsToAssigneNotPreviouslyOwned =
|
||||
final SortedSet<TopicPartition> partitionsToAssignNotPreviouslyOwned =
|
||||
partitionsToAssignNotPreviouslyOwned(partitionsToAssign, topicPartitionsForActiveTasks(ownedActiveTasks));
|
||||
|
||||
subscriptionState.assignFromSubscribedAwaitingCallback(
|
||||
partitionsToAssign,
|
||||
partitionsToAssigneNotPreviouslyOwned
|
||||
partitionsToAssignNotPreviouslyOwned
|
||||
);
|
||||
notifyAssignmentChange(partitionsToAssign);
|
||||
|
||||
|
@ -1152,10 +1152,10 @@ public class StreamsMembershipManager implements RequestManager {
|
|||
if (callbackError == null) {
|
||||
subscriptionState.enablePartitionsAwaitingCallback(partitionsToAssign);
|
||||
} else {
|
||||
if (!partitionsToAssigneNotPreviouslyOwned.isEmpty()) {
|
||||
if (!partitionsToAssignNotPreviouslyOwned.isEmpty()) {
|
||||
log.warn("Leaving newly assigned partitions {} marked as non-fetchable and not " +
|
||||
"requiring initializing positions after onTasksAssigned callback failed.",
|
||||
partitionsToAssigneNotPreviouslyOwned, callbackError);
|
||||
partitionsToAssignNotPreviouslyOwned, callbackError);
|
||||
}
|
||||
}
|
||||
});
|
||||
|
@ -1205,9 +1205,9 @@ public class StreamsMembershipManager implements RequestManager {
|
|||
Stream.concat(
|
||||
streamsRebalanceData.subtopologies().get(task.subtopologyId()).sourceTopics().stream(),
|
||||
streamsRebalanceData.subtopologies().get(task.subtopologyId()).repartitionSourceTopics().keySet().stream()
|
||||
).forEach(topic -> {
|
||||
topicPartitions.add(new TopicPartition(topic, task.partitionId()));
|
||||
})
|
||||
).forEach(topic ->
|
||||
topicPartitions.add(new TopicPartition(topic, task.partitionId()))
|
||||
)
|
||||
);
|
||||
return topicPartitions;
|
||||
}
|
||||
|
@ -1223,7 +1223,7 @@ public class StreamsMembershipManager implements RequestManager {
|
|||
String reason = rejoinedWhileReconciliationInProgress ?
|
||||
"the member has re-joined the group" :
|
||||
"the member already transitioned out of the reconciling state into " + state;
|
||||
log.info("Interrupting reconciliation that is not relevant anymore because " + reason);
|
||||
log.info("Interrupting reconciliation that is not relevant anymore because {}", reason);
|
||||
markReconciliationCompleted();
|
||||
}
|
||||
return shouldAbort;
|
||||
|
|
|
@ -0,0 +1,117 @@
|
|||
/*
|
||||
* 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.clients.consumer.internals;
|
||||
|
||||
import org.apache.kafka.common.errors.InterruptException;
|
||||
import org.apache.kafka.common.errors.WakeupException;
|
||||
import org.apache.kafka.common.utils.LogContext;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
|
||||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* This class encapsulates the invocation of the callback methods defined in the {@link StreamsRebalanceListener}
|
||||
* interface. When streams group task assignment changes, these methods are invoked. This class wraps those
|
||||
* callback calls with some logging and error handling.
|
||||
*/
|
||||
public class StreamsRebalanceListenerInvoker {
|
||||
|
||||
private final Logger log;
|
||||
|
||||
private final StreamsRebalanceData streamsRebalanceData;
|
||||
private Optional<StreamsRebalanceListener> listener;
|
||||
|
||||
StreamsRebalanceListenerInvoker(LogContext logContext, StreamsRebalanceData streamsRebalanceData) {
|
||||
this.log = logContext.logger(getClass());
|
||||
this.listener = Optional.empty();
|
||||
this.streamsRebalanceData = streamsRebalanceData;
|
||||
}
|
||||
|
||||
public void setRebalanceListener(StreamsRebalanceListener streamsRebalanceListener) {
|
||||
Objects.requireNonNull(streamsRebalanceListener, "StreamsRebalanceListener cannot be null");
|
||||
this.listener = Optional.of(streamsRebalanceListener);
|
||||
}
|
||||
|
||||
public Exception invokeAllTasksRevoked() {
|
||||
if (listener.isEmpty()) {
|
||||
throw new IllegalStateException("StreamsRebalanceListener is not defined");
|
||||
}
|
||||
return invokeTasksRevoked(streamsRebalanceData.reconciledAssignment().activeTasks());
|
||||
}
|
||||
|
||||
public Exception invokeTasksAssigned(final StreamsRebalanceData.Assignment assignment) {
|
||||
if (listener.isEmpty()) {
|
||||
throw new IllegalStateException("StreamsRebalanceListener is not defined");
|
||||
}
|
||||
log.info("Invoking tasks assigned callback for new assignment: {}", assignment);
|
||||
try {
|
||||
listener.get().onTasksAssigned(assignment);
|
||||
} catch (WakeupException | InterruptException e) {
|
||||
throw e;
|
||||
} catch (Exception e) {
|
||||
log.error(
|
||||
"Streams rebalance listener failed on invocation of onTasksAssigned for tasks {}",
|
||||
assignment,
|
||||
e
|
||||
);
|
||||
return e;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
public Exception invokeTasksRevoked(final Set<StreamsRebalanceData.TaskId> tasks) {
|
||||
if (listener.isEmpty()) {
|
||||
throw new IllegalStateException("StreamsRebalanceListener is not defined");
|
||||
}
|
||||
log.info("Invoking task revoked callback for revoked active tasks {}", tasks);
|
||||
try {
|
||||
listener.get().onTasksRevoked(tasks);
|
||||
} catch (WakeupException | InterruptException e) {
|
||||
throw e;
|
||||
} catch (Exception e) {
|
||||
log.error(
|
||||
"Streams rebalance listener failed on invocation of onTasksRevoked for tasks {}",
|
||||
tasks,
|
||||
e
|
||||
);
|
||||
return e;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
public Exception invokeAllTasksLost() {
|
||||
if (listener.isEmpty()) {
|
||||
throw new IllegalStateException("StreamsRebalanceListener is not defined");
|
||||
}
|
||||
log.info("Invoking tasks lost callback for all tasks");
|
||||
try {
|
||||
listener.get().onAllTasksLost();
|
||||
} catch (WakeupException | InterruptException e) {
|
||||
throw e;
|
||||
} catch (Exception e) {
|
||||
log.error(
|
||||
"Streams rebalance listener failed on invocation of onTasksLost.",
|
||||
e
|
||||
);
|
||||
return e;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
}
|
|
@ -477,7 +477,7 @@ public class SubscriptionState {
|
|||
* Provides the number of assigned partitions in a thread safe manner.
|
||||
* @return the number of assigned partitions.
|
||||
*/
|
||||
synchronized int numAssignedPartitions() {
|
||||
public synchronized int numAssignedPartitions() {
|
||||
return this.assignment.size();
|
||||
}
|
||||
|
||||
|
|
|
@ -16,6 +16,8 @@
|
|||
*/
|
||||
package org.apache.kafka.clients.consumer.internals.metrics;
|
||||
|
||||
import org.apache.kafka.clients.consumer.KafkaConsumer;
|
||||
import org.apache.kafka.clients.consumer.internals.SubscriptionState;
|
||||
import org.apache.kafka.common.MetricName;
|
||||
import org.apache.kafka.common.metrics.Measurable;
|
||||
import org.apache.kafka.common.metrics.Metrics;
|
||||
|
@ -27,7 +29,9 @@ import org.apache.kafka.common.metrics.stats.Max;
|
|||
import org.apache.kafka.common.metrics.stats.Rate;
|
||||
import org.apache.kafka.common.metrics.stats.WindowedCount;
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_METRIC_GROUP_PREFIX;
|
||||
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.COORDINATOR_METRICS_SUFFIX;
|
||||
|
@ -44,11 +48,14 @@ public final class ConsumerRebalanceMetricsManager extends RebalanceMetricsManag
|
|||
public final MetricName lastRebalanceSecondsAgo;
|
||||
public final MetricName failedRebalanceTotal;
|
||||
public final MetricName failedRebalanceRate;
|
||||
public final MetricName assignedPartitionsCount;
|
||||
private long lastRebalanceEndMs = -1L;
|
||||
private long lastRebalanceStartMs = -1L;
|
||||
private final Metrics metrics;
|
||||
|
||||
public ConsumerRebalanceMetricsManager(Metrics metrics) {
|
||||
public ConsumerRebalanceMetricsManager(Metrics metrics, SubscriptionState subscriptions) {
|
||||
super(CONSUMER_METRIC_GROUP_PREFIX + COORDINATOR_METRICS_SUFFIX);
|
||||
this.metrics = metrics;
|
||||
|
||||
rebalanceLatencyAvg = createMetric(metrics, "rebalance-latency-avg",
|
||||
"The average time in ms taken for a group to complete a rebalance");
|
||||
|
@ -64,6 +71,9 @@ public final class ConsumerRebalanceMetricsManager extends RebalanceMetricsManag
|
|||
"The total number of failed rebalance events");
|
||||
failedRebalanceRate = createMetric(metrics, "failed-rebalance-rate-per-hour",
|
||||
"The number of failed rebalance events per hour");
|
||||
assignedPartitionsCount = createMetric(metrics, "assigned-partitions",
|
||||
"The number of partitions currently assigned to this consumer");
|
||||
registerAssignedPartitionCount(subscriptions);
|
||||
|
||||
successfulRebalanceSensor = metrics.sensor("rebalance-latency");
|
||||
successfulRebalanceSensor.add(rebalanceLatencyAvg, new Avg());
|
||||
|
@ -106,4 +116,15 @@ public final class ConsumerRebalanceMetricsManager extends RebalanceMetricsManag
|
|||
public boolean rebalanceStarted() {
|
||||
return lastRebalanceStartMs > lastRebalanceEndMs;
|
||||
}
|
||||
|
||||
/**
|
||||
* Register metric to track the number of assigned partitions.
|
||||
* It will consider partitions assigned to the consumer
|
||||
* regardless of whether they were assigned via {@link KafkaConsumer#subscribe(Pattern)} or
|
||||
* {@link KafkaConsumer#assign(Collection)}
|
||||
*/
|
||||
private void registerAssignedPartitionCount(SubscriptionState subscriptions) {
|
||||
Measurable numParts = (config, now) -> subscriptions.numAssignedPartitions();
|
||||
metrics.addMetric(assignedPartitionsCount, numParts);
|
||||
}
|
||||
}
|
|
@ -553,7 +553,7 @@ public final class Metrics implements Closeable {
|
|||
try {
|
||||
reporter.metricRemoval(metric);
|
||||
} catch (Exception e) {
|
||||
log.error("Error when removing metric from " + reporter.getClass().getName(), e);
|
||||
log.error("Error when removing metric from {}", reporter.getClass().getName(), e);
|
||||
}
|
||||
}
|
||||
log.trace("Removed metric named {}", metricName);
|
||||
|
@ -596,7 +596,7 @@ public final class Metrics implements Closeable {
|
|||
try {
|
||||
reporter.metricChange(metric);
|
||||
} catch (Exception e) {
|
||||
log.error("Error when registering metric on " + reporter.getClass().getName(), e);
|
||||
log.error("Error when registering metric on {}", reporter.getClass().getName(), e);
|
||||
}
|
||||
}
|
||||
log.trace("Registered metric named {}", metricName);
|
||||
|
@ -688,7 +688,7 @@ public final class Metrics implements Closeable {
|
|||
log.info("Closing reporter {}", reporter.getClass().getName());
|
||||
reporter.close();
|
||||
} catch (Exception e) {
|
||||
log.error("Error when closing " + reporter.getClass().getName(), e);
|
||||
log.error("Error when closing {}", reporter.getClass().getName(), e);
|
||||
}
|
||||
}
|
||||
log.info("Metrics reporters closed");
|
||||
|
|
|
@ -40,7 +40,7 @@ public final class ByteBufferUnmapper {
|
|||
private static final RuntimeException UNMAP_NOT_SUPPORTED_EXCEPTION;
|
||||
|
||||
static {
|
||||
Object unmap = null;
|
||||
MethodHandle unmap = null;
|
||||
RuntimeException exception = null;
|
||||
try {
|
||||
unmap = lookupUnmapMethodHandle();
|
||||
|
@ -48,7 +48,7 @@ public final class ByteBufferUnmapper {
|
|||
exception = e;
|
||||
}
|
||||
if (unmap != null) {
|
||||
UNMAP = (MethodHandle) unmap;
|
||||
UNMAP = unmap;
|
||||
UNMAP_NOT_SUPPORTED_EXCEPTION = null;
|
||||
} else {
|
||||
UNMAP = null;
|
||||
|
|
|
@ -75,7 +75,8 @@ public class ConfigUtils {
|
|||
} else if (value instanceof String) {
|
||||
return Boolean.parseBoolean((String) value);
|
||||
} else {
|
||||
log.error("Invalid value (" + value + ") on configuration '" + key + "'. The default value '" + defaultValue + "' will be used instead. Please specify a true/false value.");
|
||||
log.error("Invalid value ({}) on configuration '{}'. The default value '{}' will be used instead. Please specify a true/false value.",
|
||||
value, key, defaultValue);
|
||||
return defaultValue;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -69,7 +69,7 @@ public class LoggingSignalHandler {
|
|||
for (String signal : SIGNALS) {
|
||||
register(signal, jvmSignalHandlers);
|
||||
}
|
||||
log.info("Registered signal handlers for " + String.join(", ", SIGNALS));
|
||||
log.info("Registered signal handlers for {}", String.join(", ", SIGNALS));
|
||||
}
|
||||
|
||||
private Object createSignalHandler(final Map<String, Object> jvmSignalHandlers) {
|
||||
|
|
|
@ -33,7 +33,7 @@ import java.util.Map;
|
|||
|
||||
public class SecurityUtils {
|
||||
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger(SecurityConfig.class);
|
||||
private static final Logger LOGGER = LoggerFactory.getLogger(SecurityUtils.class);
|
||||
|
||||
private static final Map<String, ResourceType> NAME_TO_RESOURCE_TYPES;
|
||||
private static final Map<String, AclOperation> NAME_TO_OPERATIONS;
|
||||
|
|
|
@ -857,7 +857,7 @@ public final class Utils {
|
|||
public static void delete(final File rootFile) throws IOException {
|
||||
if (rootFile == null)
|
||||
return;
|
||||
Files.walkFileTree(rootFile.toPath(), new SimpleFileVisitor<Path>() {
|
||||
Files.walkFileTree(rootFile.toPath(), new SimpleFileVisitor<>() {
|
||||
@Override
|
||||
public FileVisitResult visitFileFailed(Path path, IOException exc) throws IOException {
|
||||
if (exc instanceof NoSuchFileException) {
|
||||
|
@ -1403,7 +1403,7 @@ public final class Utils {
|
|||
* @return new Collector<Map.Entry<K, V>, M, M>
|
||||
*/
|
||||
public static <K, V, M extends Map<K, V>> Collector<Map.Entry<K, V>, M, M> entriesToMap(final Supplier<M> mapSupplier) {
|
||||
return new Collector<Map.Entry<K, V>, M, M>() {
|
||||
return new Collector<>() {
|
||||
@Override
|
||||
public Supplier<M> supplier() {
|
||||
return mapSupplier;
|
||||
|
|
|
@ -33,7 +33,7 @@
|
|||
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
|
||||
"about": "The partition index." },
|
||||
{ "name": "Offset", "type": "int64", "versions": "0+",
|
||||
"about": "The deletion offset." }
|
||||
"about": "The deletion offset. -1 means that records should be truncated to the high watermark." }
|
||||
]}
|
||||
]},
|
||||
{ "name": "TimeoutMs", "type": "int32", "versions": "0+",
|
||||
|
|
|
@ -155,6 +155,8 @@ import javax.management.ObjectName;
|
|||
|
||||
import static java.util.Collections.singletonList;
|
||||
import static org.apache.kafka.clients.consumer.internals.ClassicKafkaConsumer.DEFAULT_REASON;
|
||||
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_METRIC_GROUP_PREFIX;
|
||||
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.COORDINATOR_METRICS_SUFFIX;
|
||||
import static org.apache.kafka.common.requests.FetchMetadata.INVALID_SESSION_ID;
|
||||
import static org.apache.kafka.common.utils.Utils.propsToMap;
|
||||
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
|
||||
|
@ -271,6 +273,35 @@ public class KafkaConsumerTest {
|
|||
}
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@EnumSource(GroupProtocol.class)
|
||||
public void testAssignedPartitionsMetrics(GroupProtocol groupProtocol) throws InterruptedException {
|
||||
consumer = newConsumer(groupProtocol, time, mock(KafkaClient.class), subscription,
|
||||
mock(ConsumerMetadata.class), assignor, false, groupInstanceId);
|
||||
Metrics metrics = consumer.metricsRegistry();
|
||||
|
||||
// This metric is added in the background thread for the AsyncConsumer, so waiting on it to avoid flakiness.
|
||||
TestUtils.waitForCondition(() -> getMetric(metrics, "assigned-partitions") != null,
|
||||
"Consumer should register the assigned-partitions metric");
|
||||
assertNotNull(getMetric(metrics, "assigned-partitions"));
|
||||
assertEquals(0.0d, getMetric(metrics, "assigned-partitions").metricValue());
|
||||
|
||||
subscription.assignFromUser(Set.of(tp0));
|
||||
assertEquals(1.0d, getMetric(metrics, "assigned-partitions").metricValue());
|
||||
|
||||
subscription.assignFromUser(Set.of(tp0, tp1));
|
||||
assertEquals(2.0d, getMetric(metrics, "assigned-partitions").metricValue());
|
||||
|
||||
subscription.unsubscribe();
|
||||
subscription.subscribe(Set.of(topic), Optional.empty());
|
||||
subscription.assignFromSubscribed(Set.of(tp0));
|
||||
assertEquals(1.0d, getMetric(metrics, "assigned-partitions").metricValue());
|
||||
}
|
||||
|
||||
private KafkaMetric getMetric(Metrics metrics, String name) {
|
||||
return metrics.metrics().get(metrics.metricName(name, CONSUMER_METRIC_GROUP_PREFIX + COORDINATOR_METRICS_SUFFIX));
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@EnumSource(GroupProtocol.class)
|
||||
public void testUnsubscribingCustomMetricsWithSameNameDoesntAffectConsumerMetrics(GroupProtocol groupProtocol) {
|
||||
|
|
|
@ -2221,6 +2221,73 @@ public class AsyncKafkaConsumerTest {
|
|||
}).when(applicationEventHandler).add(ArgumentMatchers.isA(CommitEvent.class));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCloseInvokesStreamsRebalanceListenerOnTasksRevokedWhenMemberEpochPositive() {
|
||||
final String groupId = "streamsGroup";
|
||||
final StreamsRebalanceData streamsRebalanceData = new StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of());
|
||||
|
||||
try (final MockedStatic<RequestManagers> requestManagers = mockStatic(RequestManagers.class)) {
|
||||
consumer = newConsumerWithStreamRebalanceData(requiredConsumerConfigAndGroupId(groupId), streamsRebalanceData);
|
||||
StreamsRebalanceListener mockStreamsListener = mock(StreamsRebalanceListener.class);
|
||||
when(mockStreamsListener.onTasksRevoked(any())).thenReturn(Optional.empty());
|
||||
consumer.subscribe(singletonList("topic"), mockStreamsListener);
|
||||
final MemberStateListener groupMetadataUpdateListener = captureGroupMetadataUpdateListener(requestManagers);
|
||||
final int memberEpoch = 42;
|
||||
final String memberId = "memberId";
|
||||
groupMetadataUpdateListener.onMemberEpochUpdated(Optional.of(memberEpoch), memberId);
|
||||
|
||||
consumer.close(CloseOptions.timeout(Duration.ZERO));
|
||||
|
||||
verify(mockStreamsListener).onTasksRevoked(any());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCloseInvokesStreamsRebalanceListenerOnAllTasksLostWhenMemberEpochZeroOrNegative() {
|
||||
final String groupId = "streamsGroup";
|
||||
final StreamsRebalanceData streamsRebalanceData = new StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of());
|
||||
|
||||
try (final MockedStatic<RequestManagers> requestManagers = mockStatic(RequestManagers.class)) {
|
||||
consumer = newConsumerWithStreamRebalanceData(requiredConsumerConfigAndGroupId(groupId), streamsRebalanceData);
|
||||
StreamsRebalanceListener mockStreamsListener = mock(StreamsRebalanceListener.class);
|
||||
when(mockStreamsListener.onAllTasksLost()).thenReturn(Optional.empty());
|
||||
consumer.subscribe(singletonList("topic"), mockStreamsListener);
|
||||
final MemberStateListener groupMetadataUpdateListener = captureGroupMetadataUpdateListener(requestManagers);
|
||||
final int memberEpoch = 0;
|
||||
final String memberId = "memberId";
|
||||
groupMetadataUpdateListener.onMemberEpochUpdated(Optional.of(memberEpoch), memberId);
|
||||
|
||||
consumer.close(CloseOptions.timeout(Duration.ZERO));
|
||||
|
||||
verify(mockStreamsListener).onAllTasksLost();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCloseWrapsStreamsRebalanceListenerException() {
|
||||
final String groupId = "streamsGroup";
|
||||
final StreamsRebalanceData streamsRebalanceData = new StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of());
|
||||
|
||||
try (final MockedStatic<RequestManagers> requestManagers = mockStatic(RequestManagers.class)) {
|
||||
consumer = newConsumerWithStreamRebalanceData(requiredConsumerConfigAndGroupId(groupId), streamsRebalanceData);
|
||||
StreamsRebalanceListener mockStreamsListener = mock(StreamsRebalanceListener.class);
|
||||
RuntimeException testException = new RuntimeException("Test streams listener exception");
|
||||
doThrow(testException).when(mockStreamsListener).onTasksRevoked(any());
|
||||
consumer.subscribe(singletonList("topic"), mockStreamsListener);
|
||||
final MemberStateListener groupMetadataUpdateListener = captureGroupMetadataUpdateListener(requestManagers);
|
||||
final int memberEpoch = 1;
|
||||
final String memberId = "memberId";
|
||||
groupMetadataUpdateListener.onMemberEpochUpdated(Optional.of(memberEpoch), memberId);
|
||||
|
||||
KafkaException thrownException = assertThrows(KafkaException.class,
|
||||
() -> consumer.close(CloseOptions.timeout(Duration.ZERO)));
|
||||
|
||||
assertInstanceOf(RuntimeException.class, thrownException.getCause());
|
||||
assertTrue(thrownException.getCause().getMessage().contains("Test streams listener exception"));
|
||||
verify(mockStreamsListener).onTasksRevoked(any());
|
||||
}
|
||||
}
|
||||
|
||||
private void markReconcileAndAutoCommitCompleteForPollEvent() {
|
||||
doAnswer(invocation -> {
|
||||
PollEvent event = invocation.getArgument(0);
|
||||
|
|
|
@ -71,6 +71,8 @@ import java.util.stream.Stream;
|
|||
|
||||
import static org.apache.kafka.clients.consumer.internals.AbstractMembershipManager.TOPIC_PARTITION_COMPARATOR;
|
||||
import static org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer.invokeRebalanceCallbacks;
|
||||
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_METRIC_GROUP_PREFIX;
|
||||
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.COORDINATOR_METRICS_SUFFIX;
|
||||
import static org.apache.kafka.common.requests.ConsumerGroupHeartbeatRequest.LEAVE_GROUP_MEMBER_EPOCH;
|
||||
import static org.apache.kafka.common.utils.Utils.mkEntry;
|
||||
import static org.apache.kafka.common.utils.Utils.mkMap;
|
||||
|
@ -125,7 +127,7 @@ public class ConsumerMembershipManagerTest {
|
|||
time = new MockTime(0);
|
||||
backgroundEventHandler = new BackgroundEventHandler(backgroundEventQueue, time, mock(AsyncConsumerMetrics.class));
|
||||
metrics = new Metrics(time);
|
||||
rebalanceMetricsManager = new ConsumerRebalanceMetricsManager(metrics);
|
||||
rebalanceMetricsManager = new ConsumerRebalanceMetricsManager(metrics, subscriptionState);
|
||||
|
||||
when(commitRequestManager.maybeAutoCommitSyncBeforeRebalance(anyLong())).thenReturn(CompletableFuture.completedFuture(null));
|
||||
}
|
||||
|
@ -181,6 +183,15 @@ public class ConsumerMembershipManagerTest {
|
|||
assertEquals(Optional.of("rack1"), membershipManager.rackId());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAssignedPartitionCountMetricRegistered() {
|
||||
MetricName metricName = metrics.metricName(
|
||||
"assigned-partitions",
|
||||
CONSUMER_METRIC_GROUP_PREFIX + COORDINATOR_METRICS_SUFFIX
|
||||
);
|
||||
assertNotNull(metrics.metric(metricName), "Metric assigned-partitions should have been registered");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMembershipManagerInitSupportsEmptyGroupInstanceId() {
|
||||
createMembershipManagerJoiningGroup();
|
||||
|
|
|
@ -67,6 +67,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
|
|||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.mockito.ArgumentMatchers.any;
|
||||
|
@ -131,6 +132,15 @@ public class StreamsMembershipManagerTest {
|
|||
verifyInStateUnsubscribed(membershipManager);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAssignedPartitionCountMetricRegistered() {
|
||||
MetricName metricName = metrics.metricName(
|
||||
"assigned-partitions",
|
||||
CONSUMER_METRIC_GROUP_PREFIX + COORDINATOR_METRICS_SUFFIX
|
||||
);
|
||||
assertNotNull(metrics.metric(metricName), "Metric assigned-partitions should have been registered");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnexpectedErrorInHeartbeatResponse() {
|
||||
final String errorMessage = "Nobody expects the Spanish Inquisition!";
|
||||
|
|
|
@ -0,0 +1,293 @@
|
|||
/*
|
||||
* 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.clients.consumer.internals;
|
||||
|
||||
import org.apache.kafka.common.errors.InterruptException;
|
||||
import org.apache.kafka.common.errors.WakeupException;
|
||||
import org.apache.kafka.common.utils.LogContext;
|
||||
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
import org.mockito.Mock;
|
||||
import org.mockito.junit.jupiter.MockitoExtension;
|
||||
import org.mockito.junit.jupiter.MockitoSettings;
|
||||
import org.mockito.quality.Strictness;
|
||||
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.mockito.ArgumentMatchers.any;
|
||||
import static org.mockito.ArgumentMatchers.eq;
|
||||
import static org.mockito.Mockito.doThrow;
|
||||
import static org.mockito.Mockito.never;
|
||||
import static org.mockito.Mockito.verify;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
@ExtendWith(MockitoExtension.class)
|
||||
@MockitoSettings(strictness = Strictness.STRICT_STUBS)
|
||||
public class StreamsRebalanceListenerInvokerTest {
|
||||
|
||||
@Mock
|
||||
private StreamsRebalanceListener mockListener;
|
||||
|
||||
@Mock
|
||||
private StreamsRebalanceData streamsRebalanceData;
|
||||
|
||||
private StreamsRebalanceListenerInvoker invoker;
|
||||
private final LogContext logContext = new LogContext();
|
||||
|
||||
@BeforeEach
|
||||
public void setup() {
|
||||
invoker = new StreamsRebalanceListenerInvoker(logContext, streamsRebalanceData);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSetRebalanceListenerWithNull() {
|
||||
NullPointerException exception = assertThrows(NullPointerException.class,
|
||||
() -> invoker.setRebalanceListener(null));
|
||||
assertEquals("StreamsRebalanceListener cannot be null", exception.getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSetRebalanceListenerOverwritesExisting() {
|
||||
StreamsRebalanceListener firstListener = org.mockito.Mockito.mock(StreamsRebalanceListener.class);
|
||||
StreamsRebalanceListener secondListener = org.mockito.Mockito.mock(StreamsRebalanceListener.class);
|
||||
|
||||
StreamsRebalanceData.Assignment mockAssignment = createMockAssignment();
|
||||
when(streamsRebalanceData.reconciledAssignment()).thenReturn(mockAssignment);
|
||||
when(secondListener.onTasksRevoked(any())).thenReturn(Optional.empty());
|
||||
|
||||
// Set first listener
|
||||
invoker.setRebalanceListener(firstListener);
|
||||
|
||||
// Overwrite with second listener
|
||||
invoker.setRebalanceListener(secondListener);
|
||||
|
||||
// Should use second listener
|
||||
invoker.invokeAllTasksRevoked();
|
||||
verify(firstListener, never()).onTasksRevoked(any());
|
||||
verify(secondListener).onTasksRevoked(eq(mockAssignment.activeTasks()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvokeMethodsWithNoListener() {
|
||||
IllegalStateException exception1 = assertThrows(IllegalStateException.class,
|
||||
() -> invoker.invokeAllTasksRevoked());
|
||||
assertEquals("StreamsRebalanceListener is not defined", exception1.getMessage());
|
||||
|
||||
IllegalStateException exception2 = assertThrows(IllegalStateException.class,
|
||||
() -> invoker.invokeTasksAssigned(createMockAssignment()));
|
||||
assertEquals("StreamsRebalanceListener is not defined", exception2.getMessage());
|
||||
|
||||
IllegalStateException exception3 = assertThrows(IllegalStateException.class,
|
||||
() -> invoker.invokeTasksRevoked(createMockTasks()));
|
||||
assertEquals("StreamsRebalanceListener is not defined", exception3.getMessage());
|
||||
|
||||
IllegalStateException exception4 = assertThrows(IllegalStateException.class,
|
||||
() -> invoker.invokeAllTasksLost());
|
||||
assertEquals("StreamsRebalanceListener is not defined", exception4.getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvokeAllTasksRevokedWithListener() {
|
||||
invoker.setRebalanceListener(mockListener);
|
||||
|
||||
StreamsRebalanceData.Assignment mockAssignment = createMockAssignment();
|
||||
when(streamsRebalanceData.reconciledAssignment()).thenReturn(mockAssignment);
|
||||
when(mockListener.onTasksRevoked(any())).thenReturn(Optional.empty());
|
||||
|
||||
Exception result = invoker.invokeAllTasksRevoked();
|
||||
|
||||
assertNull(result);
|
||||
verify(mockListener).onTasksRevoked(eq(mockAssignment.activeTasks()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvokeTasksAssignedWithListener() {
|
||||
invoker.setRebalanceListener(mockListener);
|
||||
StreamsRebalanceData.Assignment assignment = createMockAssignment();
|
||||
when(mockListener.onTasksAssigned(assignment)).thenReturn(Optional.empty());
|
||||
|
||||
Exception result = invoker.invokeTasksAssigned(assignment);
|
||||
|
||||
assertNull(result);
|
||||
verify(mockListener).onTasksAssigned(eq(assignment));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvokeTasksAssignedWithWakeupException() {
|
||||
invoker.setRebalanceListener(mockListener);
|
||||
StreamsRebalanceData.Assignment assignment = createMockAssignment();
|
||||
WakeupException wakeupException = new WakeupException();
|
||||
doThrow(wakeupException).when(mockListener).onTasksAssigned(assignment);
|
||||
|
||||
WakeupException thrownException = assertThrows(WakeupException.class,
|
||||
() -> invoker.invokeTasksAssigned(assignment));
|
||||
|
||||
assertEquals(wakeupException, thrownException);
|
||||
verify(mockListener).onTasksAssigned(eq(assignment));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvokeTasksAssignedWithInterruptException() {
|
||||
invoker.setRebalanceListener(mockListener);
|
||||
StreamsRebalanceData.Assignment assignment = createMockAssignment();
|
||||
InterruptException interruptException = new InterruptException("Test interrupt");
|
||||
doThrow(interruptException).when(mockListener).onTasksAssigned(assignment);
|
||||
|
||||
InterruptException thrownException = assertThrows(InterruptException.class,
|
||||
() -> invoker.invokeTasksAssigned(assignment));
|
||||
|
||||
assertEquals(interruptException, thrownException);
|
||||
verify(mockListener).onTasksAssigned(eq(assignment));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvokeTasksAssignedWithOtherException() {
|
||||
invoker.setRebalanceListener(mockListener);
|
||||
StreamsRebalanceData.Assignment assignment = createMockAssignment();
|
||||
RuntimeException runtimeException = new RuntimeException("Test exception");
|
||||
doThrow(runtimeException).when(mockListener).onTasksAssigned(assignment);
|
||||
|
||||
Exception result = invoker.invokeTasksAssigned(assignment);
|
||||
|
||||
assertEquals(runtimeException, result);
|
||||
verify(mockListener).onTasksAssigned(eq(assignment));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvokeTasksRevokedWithListener() {
|
||||
invoker.setRebalanceListener(mockListener);
|
||||
Set<StreamsRebalanceData.TaskId> tasks = createMockTasks();
|
||||
when(mockListener.onTasksRevoked(tasks)).thenReturn(Optional.empty());
|
||||
|
||||
Exception result = invoker.invokeTasksRevoked(tasks);
|
||||
|
||||
assertNull(result);
|
||||
verify(mockListener).onTasksRevoked(eq(tasks));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvokeTasksRevokedWithWakeupException() {
|
||||
invoker.setRebalanceListener(mockListener);
|
||||
Set<StreamsRebalanceData.TaskId> tasks = createMockTasks();
|
||||
WakeupException wakeupException = new WakeupException();
|
||||
doThrow(wakeupException).when(mockListener).onTasksRevoked(tasks);
|
||||
|
||||
WakeupException thrownException = assertThrows(WakeupException.class,
|
||||
() -> invoker.invokeTasksRevoked(tasks));
|
||||
|
||||
assertEquals(wakeupException, thrownException);
|
||||
verify(mockListener).onTasksRevoked(eq(tasks));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvokeTasksRevokedWithInterruptException() {
|
||||
invoker.setRebalanceListener(mockListener);
|
||||
Set<StreamsRebalanceData.TaskId> tasks = createMockTasks();
|
||||
InterruptException interruptException = new InterruptException("Test interrupt");
|
||||
doThrow(interruptException).when(mockListener).onTasksRevoked(tasks);
|
||||
|
||||
InterruptException thrownException = assertThrows(InterruptException.class,
|
||||
() -> invoker.invokeTasksRevoked(tasks));
|
||||
|
||||
assertEquals(interruptException, thrownException);
|
||||
verify(mockListener).onTasksRevoked(eq(tasks));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvokeTasksRevokedWithOtherException() {
|
||||
invoker.setRebalanceListener(mockListener);
|
||||
Set<StreamsRebalanceData.TaskId> tasks = createMockTasks();
|
||||
RuntimeException runtimeException = new RuntimeException("Test exception");
|
||||
doThrow(runtimeException).when(mockListener).onTasksRevoked(tasks);
|
||||
|
||||
Exception result = invoker.invokeTasksRevoked(tasks);
|
||||
|
||||
assertEquals(runtimeException, result);
|
||||
verify(mockListener).onTasksRevoked(eq(tasks));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvokeAllTasksLostWithListener() {
|
||||
invoker.setRebalanceListener(mockListener);
|
||||
when(mockListener.onAllTasksLost()).thenReturn(Optional.empty());
|
||||
|
||||
Exception result = invoker.invokeAllTasksLost();
|
||||
|
||||
assertNull(result);
|
||||
verify(mockListener).onAllTasksLost();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvokeAllTasksLostWithWakeupException() {
|
||||
invoker.setRebalanceListener(mockListener);
|
||||
WakeupException wakeupException = new WakeupException();
|
||||
doThrow(wakeupException).when(mockListener).onAllTasksLost();
|
||||
|
||||
WakeupException thrownException = assertThrows(WakeupException.class,
|
||||
() -> invoker.invokeAllTasksLost());
|
||||
|
||||
assertEquals(wakeupException, thrownException);
|
||||
verify(mockListener).onAllTasksLost();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvokeAllTasksLostWithInterruptException() {
|
||||
invoker.setRebalanceListener(mockListener);
|
||||
InterruptException interruptException = new InterruptException("Test interrupt");
|
||||
doThrow(interruptException).when(mockListener).onAllTasksLost();
|
||||
|
||||
InterruptException thrownException = assertThrows(InterruptException.class,
|
||||
() -> invoker.invokeAllTasksLost());
|
||||
|
||||
assertEquals(interruptException, thrownException);
|
||||
verify(mockListener).onAllTasksLost();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvokeAllTasksLostWithOtherException() {
|
||||
invoker.setRebalanceListener(mockListener);
|
||||
RuntimeException runtimeException = new RuntimeException("Test exception");
|
||||
doThrow(runtimeException).when(mockListener).onAllTasksLost();
|
||||
|
||||
Exception result = invoker.invokeAllTasksLost();
|
||||
|
||||
assertEquals(runtimeException, result);
|
||||
verify(mockListener).onAllTasksLost();
|
||||
}
|
||||
|
||||
private StreamsRebalanceData.Assignment createMockAssignment() {
|
||||
Set<StreamsRebalanceData.TaskId> activeTasks = createMockTasks();
|
||||
Set<StreamsRebalanceData.TaskId> standbyTasks = Set.of();
|
||||
Set<StreamsRebalanceData.TaskId> warmupTasks = Set.of();
|
||||
|
||||
return new StreamsRebalanceData.Assignment(activeTasks, standbyTasks, warmupTasks);
|
||||
}
|
||||
|
||||
private Set<StreamsRebalanceData.TaskId> createMockTasks() {
|
||||
return Set.of(
|
||||
new StreamsRebalanceData.TaskId("subtopology1", 0),
|
||||
new StreamsRebalanceData.TaskId("subtopology1", 1)
|
||||
);
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,63 @@
|
|||
/*
|
||||
* 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.clients.consumer.internals.metrics;
|
||||
|
||||
import org.apache.kafka.clients.consumer.internals.AutoOffsetResetStrategy;
|
||||
import org.apache.kafka.clients.consumer.internals.SubscriptionState;
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.apache.kafka.common.metrics.Metrics;
|
||||
import org.apache.kafka.common.utils.LogContext;
|
||||
import org.apache.kafka.common.utils.MockTime;
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||
import static org.mockito.Mockito.mock;
|
||||
|
||||
|
||||
class ConsumerRebalanceMetricsManagerTest {
|
||||
|
||||
private final Time time = new MockTime();
|
||||
private final Metrics metrics = new Metrics(time);
|
||||
|
||||
@Test
|
||||
public void testAssignedPartitionCountMetric() {
|
||||
SubscriptionState subscriptionState = new SubscriptionState(mock(LogContext.class), AutoOffsetResetStrategy.EARLIEST);
|
||||
ConsumerRebalanceMetricsManager consumerRebalanceMetricsManager = new ConsumerRebalanceMetricsManager(metrics, subscriptionState);
|
||||
|
||||
assertNotNull(metrics.metric(consumerRebalanceMetricsManager.assignedPartitionsCount), "Metric assigned-partitions has not been registered as expected");
|
||||
|
||||
// Check for manually assigned partitions
|
||||
subscriptionState.assignFromUser(Set.of(new TopicPartition("topic", 0), new TopicPartition("topic", 1)));
|
||||
assertEquals(2.0d, metrics.metric(consumerRebalanceMetricsManager.assignedPartitionsCount).metricValue());
|
||||
subscriptionState.assignFromUser(Set.of());
|
||||
assertEquals(0.0d, metrics.metric(consumerRebalanceMetricsManager.assignedPartitionsCount).metricValue());
|
||||
|
||||
subscriptionState.unsubscribe();
|
||||
assertEquals(0.0d, metrics.metric(consumerRebalanceMetricsManager.assignedPartitionsCount).metricValue());
|
||||
|
||||
// Check for automatically assigned partitions
|
||||
subscriptionState.subscribe(Set.of("topic"), Optional.empty());
|
||||
subscriptionState.assignFromSubscribed(Set.of(new TopicPartition("topic", 0)));
|
||||
assertEquals(1.0d, metrics.metric(consumerRebalanceMetricsManager.assignedPartitionsCount).metricValue());
|
||||
}
|
||||
}
|
|
@ -318,7 +318,7 @@ public class KafkaFutureTest {
|
|||
awaitAndAssertResult(future, 21, null);
|
||||
Throwable cause = awaitAndAssertFailure(dependantFuture, CompletionException.class, "java.lang.RuntimeException: We require more vespene gas");
|
||||
assertInstanceOf(RuntimeException.class, cause.getCause());
|
||||
assertEquals(cause.getCause().getMessage(), "We require more vespene gas");
|
||||
assertEquals("We require more vespene gas", cause.getCause().getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -35,8 +35,8 @@ public class UuidTest {
|
|||
public void testSignificantBits() {
|
||||
Uuid id = new Uuid(34L, 98L);
|
||||
|
||||
assertEquals(id.getMostSignificantBits(), 34L);
|
||||
assertEquals(id.getLeastSignificantBits(), 98L);
|
||||
assertEquals(34L, id.getMostSignificantBits());
|
||||
assertEquals(98L, id.getLeastSignificantBits());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -74,15 +74,15 @@ public class UuidTest {
|
|||
|
||||
String zeroIdString = Uuid.ZERO_UUID.toString();
|
||||
|
||||
assertEquals(Uuid.fromString(zeroIdString), Uuid.ZERO_UUID);
|
||||
assertEquals(Uuid.ZERO_UUID, Uuid.fromString(zeroIdString));
|
||||
}
|
||||
|
||||
@RepeatedTest(value = 100, name = RepeatedTest.LONG_DISPLAY_NAME)
|
||||
public void testRandomUuid() {
|
||||
Uuid randomID = Uuid.randomUuid();
|
||||
|
||||
assertNotEquals(randomID, Uuid.ZERO_UUID);
|
||||
assertNotEquals(randomID, Uuid.METADATA_TOPIC_ID);
|
||||
assertNotEquals(Uuid.ZERO_UUID, randomID);
|
||||
assertNotEquals(Uuid.METADATA_TOPIC_ID, randomID);
|
||||
assertFalse(randomID.toString().startsWith("-"));
|
||||
}
|
||||
|
||||
|
|
|
@ -379,8 +379,8 @@ public class AbstractConfigTest {
|
|||
Properties props = new Properties();
|
||||
props.put("config.providers", "file");
|
||||
TestIndirectConfigResolution config = new TestIndirectConfigResolution(props);
|
||||
assertEquals(config.originals().get("config.providers"), "file");
|
||||
assertEquals(config.originals(Collections.singletonMap("config.providers", "file2")).get("config.providers"), "file2");
|
||||
assertEquals("file", config.originals().get("config.providers"));
|
||||
assertEquals("file2", config.originals(Collections.singletonMap("config.providers", "file2")).get("config.providers"));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -62,10 +62,10 @@ public class ChannelBuildersTest {
|
|||
assertNull(configs.get("listener.name.listener1.gssapi.sasl.kerberos.service.name"));
|
||||
assertFalse(securityConfig.unused().contains("listener.name.listener1.gssapi.sasl.kerberos.service.name"));
|
||||
|
||||
assertEquals(configs.get("gssapi.sasl.kerberos.service.name"), "testkafka");
|
||||
assertEquals("testkafka", configs.get("gssapi.sasl.kerberos.service.name"));
|
||||
assertFalse(securityConfig.unused().contains("gssapi.sasl.kerberos.service.name"));
|
||||
|
||||
assertEquals(configs.get("sasl.kerberos.service.name"), "testkafkaglobal");
|
||||
assertEquals("testkafkaglobal", configs.get("sasl.kerberos.service.name"));
|
||||
assertFalse(securityConfig.unused().contains("sasl.kerberos.service.name"));
|
||||
|
||||
assertNull(configs.get("listener.name.listener1.sasl.kerberos.service.name"));
|
||||
|
@ -74,35 +74,35 @@ public class ChannelBuildersTest {
|
|||
assertNull(configs.get("plain.sasl.server.callback.handler.class"));
|
||||
assertFalse(securityConfig.unused().contains("plain.sasl.server.callback.handler.class"));
|
||||
|
||||
assertEquals(configs.get("listener.name.listener1.gssapi.config1.key"), "custom.config1");
|
||||
assertEquals("custom.config1", configs.get("listener.name.listener1.gssapi.config1.key"));
|
||||
assertFalse(securityConfig.unused().contains("listener.name.listener1.gssapi.config1.key"));
|
||||
|
||||
assertEquals(configs.get("custom.config2.key"), "custom.config2");
|
||||
assertEquals("custom.config2", configs.get("custom.config2.key"));
|
||||
assertFalse(securityConfig.unused().contains("custom.config2.key"));
|
||||
|
||||
// test configs without listener prefix
|
||||
securityConfig = new TestSecurityConfig(props);
|
||||
configs = ChannelBuilders.channelBuilderConfigs(securityConfig, null);
|
||||
|
||||
assertEquals(configs.get("listener.name.listener1.gssapi.sasl.kerberos.service.name"), "testkafka");
|
||||
assertEquals("testkafka", configs.get("listener.name.listener1.gssapi.sasl.kerberos.service.name"));
|
||||
assertFalse(securityConfig.unused().contains("listener.name.listener1.gssapi.sasl.kerberos.service.name"));
|
||||
|
||||
assertNull(configs.get("gssapi.sasl.kerberos.service.name"));
|
||||
assertFalse(securityConfig.unused().contains("gssapi.sasl.kerberos.service.name"));
|
||||
|
||||
assertEquals(configs.get("listener.name.listener1.sasl.kerberos.service.name"), "testkafkaglobal");
|
||||
assertEquals("testkafkaglobal", configs.get("listener.name.listener1.sasl.kerberos.service.name"));
|
||||
assertFalse(securityConfig.unused().contains("listener.name.listener1.sasl.kerberos.service.name"));
|
||||
|
||||
assertNull(configs.get("sasl.kerberos.service.name"));
|
||||
assertFalse(securityConfig.unused().contains("sasl.kerberos.service.name"));
|
||||
|
||||
assertEquals(configs.get("plain.sasl.server.callback.handler.class"), "callback");
|
||||
assertEquals("callback", configs.get("plain.sasl.server.callback.handler.class"));
|
||||
assertFalse(securityConfig.unused().contains("plain.sasl.server.callback.handler.class"));
|
||||
|
||||
assertEquals(configs.get("listener.name.listener1.gssapi.config1.key"), "custom.config1");
|
||||
assertEquals("custom.config1", configs.get("listener.name.listener1.gssapi.config1.key"));
|
||||
assertFalse(securityConfig.unused().contains("listener.name.listener1.gssapi.config1.key"));
|
||||
|
||||
assertEquals(configs.get("custom.config2.key"), "custom.config2");
|
||||
assertEquals("custom.config2", configs.get("custom.config2.key"));
|
||||
assertFalse(securityConfig.unused().contains("custom.config2.key"));
|
||||
}
|
||||
|
||||
|
|
|
@ -53,7 +53,7 @@ public class ErrorsTest {
|
|||
public void testExceptionsAreNotGeneric() {
|
||||
for (Errors error : Errors.values()) {
|
||||
if (error != Errors.NONE)
|
||||
assertNotEquals(error.exception().getClass(), ApiException.class, "Generic ApiException should not be used");
|
||||
assertNotEquals(ApiException.class, error.exception().getClass(), "Generic ApiException should not be used");
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -197,7 +197,7 @@ public class FileRecordsTest {
|
|||
* Test a simple append and read.
|
||||
*/
|
||||
@Test
|
||||
public void testRead() throws IOException {
|
||||
public void testRead() {
|
||||
FileRecords read = fileRecords.slice(0, fileRecords.sizeInBytes());
|
||||
assertEquals(fileRecords.sizeInBytes(), read.sizeInBytes());
|
||||
TestUtils.checkEquals(fileRecords.batches(), read.batches());
|
||||
|
@ -279,7 +279,7 @@ public class FileRecordsTest {
|
|||
* Test that the message set iterator obeys start and end slicing
|
||||
*/
|
||||
@Test
|
||||
public void testIteratorWithLimits() throws IOException {
|
||||
public void testIteratorWithLimits() {
|
||||
RecordBatch batch = batches(fileRecords).get(1);
|
||||
int start = fileRecords.searchForOffsetFromPosition(1, 0).position;
|
||||
int size = batch.sizeInBytes();
|
||||
|
|
|
@ -46,8 +46,8 @@ public class ReplicaSelectorTest {
|
|||
ReplicaSelector selector = new RackAwareReplicaSelector();
|
||||
Optional<ReplicaView> selected = selector.select(tp, metadata("rack-b"), partitionView);
|
||||
assertOptional(selected, replicaInfo -> {
|
||||
assertEquals(replicaInfo.endpoint().rack(), "rack-b", "Expect replica to be in rack-b");
|
||||
assertEquals(replicaInfo.endpoint().id(), 3, "Expected replica 3 since it is more caught-up");
|
||||
assertEquals("rack-b", replicaInfo.endpoint().rack(), "Expect replica to be in rack-b");
|
||||
assertEquals(3, replicaInfo.endpoint().id(), "Expected replica 3 since it is more caught-up");
|
||||
});
|
||||
|
||||
selected = selector.select(tp, metadata("not-a-rack"), partitionView);
|
||||
|
@ -57,7 +57,7 @@ public class ReplicaSelectorTest {
|
|||
|
||||
selected = selector.select(tp, metadata("rack-a"), partitionView);
|
||||
assertOptional(selected, replicaInfo -> {
|
||||
assertEquals(replicaInfo.endpoint().rack(), "rack-a", "Expect replica to be in rack-a");
|
||||
assertEquals("rack-a", replicaInfo.endpoint().rack(), "Expect replica to be in rack-a");
|
||||
assertEquals(replicaInfo, leader, "Expect the leader since it's in rack-a");
|
||||
});
|
||||
|
||||
|
|
|
@ -50,6 +50,20 @@ import java.util.function.Function;
|
|||
*/
|
||||
public class CoordinatorLoaderImpl<T> implements CoordinatorLoader<T> {
|
||||
|
||||
/**
|
||||
* The interval between updating the last committed offset during loading, in offsets. Smaller
|
||||
* values commit more often at the expense of loading times when the workload is simple and does
|
||||
* not create collections that need to participate in {@link CoordinatorPlayback} snapshotting.
|
||||
* Larger values commit less often and allow more temporary data to accumulate before the next
|
||||
* commit when the workload creates many temporary collections that need to be snapshotted.
|
||||
*
|
||||
* The value of 16,384 was chosen as a trade-off between the performance of these two workloads.
|
||||
*
|
||||
* When changing this value, please run the GroupCoordinatorShardLoadingBenchmark to evaluate
|
||||
* the relative change in performance.
|
||||
*/
|
||||
public static final long DEFAULT_COMMIT_INTERVAL_OFFSETS = 16384;
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(CoordinatorLoaderImpl.class);
|
||||
|
||||
private final Time time;
|
||||
|
@ -57,6 +71,7 @@ public class CoordinatorLoaderImpl<T> implements CoordinatorLoader<T> {
|
|||
private final Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier;
|
||||
private final Deserializer<T> deserializer;
|
||||
private final int loadBufferSize;
|
||||
private final long commitIntervalOffsets;
|
||||
|
||||
private final AtomicBoolean isRunning = new AtomicBoolean(true);
|
||||
private final KafkaScheduler scheduler = new KafkaScheduler(1);
|
||||
|
@ -66,13 +81,15 @@ public class CoordinatorLoaderImpl<T> implements CoordinatorLoader<T> {
|
|||
Function<TopicPartition, Optional<UnifiedLog>> partitionLogSupplier,
|
||||
Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier,
|
||||
Deserializer<T> deserializer,
|
||||
int loadBufferSize
|
||||
int loadBufferSize,
|
||||
long commitIntervalOffsets
|
||||
) {
|
||||
this.time = time;
|
||||
this.partitionLogSupplier = partitionLogSupplier;
|
||||
this.partitionLogEndOffsetSupplier = partitionLogEndOffsetSupplier;
|
||||
this.deserializer = deserializer;
|
||||
this.loadBufferSize = loadBufferSize;
|
||||
this.commitIntervalOffsets = commitIntervalOffsets;
|
||||
this.scheduler.startup();
|
||||
}
|
||||
|
||||
|
@ -121,7 +138,7 @@ public class CoordinatorLoaderImpl<T> implements CoordinatorLoader<T> {
|
|||
long currentOffset = log.logStartOffset();
|
||||
LoadStats stats = new LoadStats();
|
||||
|
||||
long previousHighWatermark = -1L;
|
||||
long lastCommittedOffset = -1L;
|
||||
while (shouldFetchNextBatch(currentOffset, logEndOffset(tp), stats.readAtLeastOneRecord)) {
|
||||
FetchDataInfo fetchDataInfo = log.read(currentOffset, loadBufferSize, FetchIsolation.LOG_END, true);
|
||||
|
||||
|
@ -133,17 +150,16 @@ public class CoordinatorLoaderImpl<T> implements CoordinatorLoader<T> {
|
|||
buffer = memoryRecords.buffer();
|
||||
}
|
||||
|
||||
ReplayResult replayResult = processMemoryRecords(tp, log, memoryRecords, coordinator, stats, currentOffset, previousHighWatermark);
|
||||
ReplayResult replayResult = processMemoryRecords(tp, log, memoryRecords, coordinator, stats, currentOffset, lastCommittedOffset);
|
||||
currentOffset = replayResult.nextOffset;
|
||||
previousHighWatermark = replayResult.highWatermark;
|
||||
lastCommittedOffset = replayResult.lastCommittedOffset;
|
||||
}
|
||||
|
||||
long endTimeMs = time.milliseconds();
|
||||
|
||||
if (logEndOffset(tp) == -1L) {
|
||||
future.completeExceptionally(new NotLeaderOrFollowerException(
|
||||
String.format("Stopped loading records from %s because the partition is not online or is no longer the leader.", tp)
|
||||
));
|
||||
String.format("Stopped loading records from %s because the partition is not online or is no longer the leader.", tp)));
|
||||
} else if (isRunning.get()) {
|
||||
future.complete(new LoadSummary(startTimeMs, endTimeMs, schedulerQueueTimeMs, stats.numRecords, stats.numBytes));
|
||||
} else {
|
||||
|
@ -208,9 +224,8 @@ public class CoordinatorLoaderImpl<T> implements CoordinatorLoader<T> {
|
|||
CoordinatorPlayback<T> coordinator,
|
||||
LoadStats loadStats,
|
||||
long currentOffset,
|
||||
long previousHighWatermark
|
||||
long lastCommittedOffset
|
||||
) {
|
||||
|
||||
for (MutableRecordBatch batch : memoryRecords.batches()) {
|
||||
if (batch.isControlBatch()) {
|
||||
for (Record record : batch) {
|
||||
|
@ -288,14 +303,18 @@ public class CoordinatorLoaderImpl<T> implements CoordinatorLoader<T> {
|
|||
if (currentOffset >= currentHighWatermark) {
|
||||
coordinator.updateLastWrittenOffset(currentOffset);
|
||||
|
||||
if (currentHighWatermark > previousHighWatermark) {
|
||||
if (currentHighWatermark > lastCommittedOffset) {
|
||||
coordinator.updateLastCommittedOffset(currentHighWatermark);
|
||||
previousHighWatermark = currentHighWatermark;
|
||||
lastCommittedOffset = currentHighWatermark;
|
||||
}
|
||||
} else if (currentOffset - lastCommittedOffset >= commitIntervalOffsets) {
|
||||
coordinator.updateLastWrittenOffset(currentOffset);
|
||||
coordinator.updateLastCommittedOffset(currentOffset);
|
||||
lastCommittedOffset = currentOffset;
|
||||
}
|
||||
}
|
||||
loadStats.numBytes += memoryRecords.sizeInBytes();
|
||||
return new ReplayResult(currentOffset, previousHighWatermark);
|
||||
return new ReplayResult(currentOffset, lastCommittedOffset);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -320,14 +339,13 @@ public class CoordinatorLoaderImpl<T> implements CoordinatorLoader<T> {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "LoadStats{" +
|
||||
return "LoadStats(" +
|
||||
"numRecords=" + numRecords +
|
||||
", numBytes=" + numBytes +
|
||||
", readAtLeastOneRecord=" + readAtLeastOneRecord +
|
||||
'}';
|
||||
')';
|
||||
}
|
||||
}
|
||||
|
||||
private record ReplayResult(long nextOffset, long highWatermark) {
|
||||
}
|
||||
private record ReplayResult(long nextOffset, long lastCommittedOffset) { }
|
||||
}
|
||||
|
|
|
@ -61,7 +61,7 @@ public class SnapshottableCoordinator<S extends CoordinatorShard<U>, U> implemen
|
|||
*/
|
||||
private long lastCommittedOffset;
|
||||
|
||||
SnapshottableCoordinator(
|
||||
public SnapshottableCoordinator(
|
||||
LogContext logContext,
|
||||
SnapshotRegistry snapshotRegistry,
|
||||
S coordinator,
|
||||
|
|
|
@ -91,7 +91,8 @@ class CoordinatorLoaderImplTest {
|
|||
partitionLogSupplier,
|
||||
partitionLogEndOffsetSupplier,
|
||||
serde,
|
||||
1000
|
||||
1000,
|
||||
CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS
|
||||
)) {
|
||||
assertFutureThrows(NotLeaderOrFollowerException.class, loader.load(tp, coordinator));
|
||||
}
|
||||
|
@ -110,7 +111,8 @@ class CoordinatorLoaderImplTest {
|
|||
partitionLogSupplier,
|
||||
partitionLogEndOffsetSupplier,
|
||||
serde,
|
||||
1000
|
||||
1000,
|
||||
CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS
|
||||
)) {
|
||||
loader.close();
|
||||
assertFutureThrows(RuntimeException.class, loader.load(tp, coordinator));
|
||||
|
@ -131,7 +133,8 @@ class CoordinatorLoaderImplTest {
|
|||
partitionLogSupplier,
|
||||
partitionLogEndOffsetSupplier,
|
||||
serde,
|
||||
1000
|
||||
1000,
|
||||
CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS
|
||||
)) {
|
||||
when(log.logStartOffset()).thenReturn(0L);
|
||||
when(log.highWatermark()).thenReturn(0L);
|
||||
|
@ -217,7 +220,8 @@ class CoordinatorLoaderImplTest {
|
|||
partitionLogSupplier,
|
||||
partitionLogEndOffsetSupplier,
|
||||
serde,
|
||||
1000
|
||||
1000,
|
||||
CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS
|
||||
)) {
|
||||
when(log.logStartOffset()).thenReturn(0L);
|
||||
|
||||
|
@ -262,7 +266,8 @@ class CoordinatorLoaderImplTest {
|
|||
partitionLogSupplier,
|
||||
partitionLogEndOffsetSupplier,
|
||||
serde,
|
||||
1000
|
||||
1000,
|
||||
CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS
|
||||
)) {
|
||||
when(log.logStartOffset()).thenReturn(0L);
|
||||
|
||||
|
@ -298,7 +303,8 @@ class CoordinatorLoaderImplTest {
|
|||
partitionLogSupplier,
|
||||
partitionLogEndOffsetSupplier,
|
||||
serde,
|
||||
1000
|
||||
1000,
|
||||
CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS
|
||||
)) {
|
||||
when(log.logStartOffset()).thenReturn(0L);
|
||||
|
||||
|
@ -337,7 +343,8 @@ class CoordinatorLoaderImplTest {
|
|||
partitionLogSupplier,
|
||||
partitionLogEndOffsetSupplier,
|
||||
serde,
|
||||
1000
|
||||
1000,
|
||||
CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS
|
||||
)) {
|
||||
when(log.logStartOffset()).thenReturn(0L);
|
||||
|
||||
|
@ -365,7 +372,8 @@ class CoordinatorLoaderImplTest {
|
|||
partitionLogSupplier,
|
||||
partitionLogEndOffsetSupplier,
|
||||
serde,
|
||||
1000
|
||||
1000,
|
||||
CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS
|
||||
)) {
|
||||
long startTimeMs = time.milliseconds();
|
||||
when(log.logStartOffset()).thenReturn(0L);
|
||||
|
@ -412,7 +420,8 @@ class CoordinatorLoaderImplTest {
|
|||
partitionLogSupplier,
|
||||
partitionLogEndOffsetSupplier,
|
||||
serde,
|
||||
1000
|
||||
1000,
|
||||
CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS
|
||||
)) {
|
||||
when(log.logStartOffset()).thenReturn(0L);
|
||||
when(log.highWatermark()).thenReturn(0L, 0L, 2L);
|
||||
|
@ -475,7 +484,8 @@ class CoordinatorLoaderImplTest {
|
|||
partitionLogSupplier,
|
||||
partitionLogEndOffsetSupplier,
|
||||
serde,
|
||||
1000
|
||||
1000,
|
||||
CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS
|
||||
)) {
|
||||
when(log.logStartOffset()).thenReturn(0L);
|
||||
when(log.highWatermark()).thenReturn(0L);
|
||||
|
@ -501,7 +511,8 @@ class CoordinatorLoaderImplTest {
|
|||
partitionLogSupplier,
|
||||
partitionLogEndOffsetSupplier,
|
||||
serde,
|
||||
1000
|
||||
1000,
|
||||
CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS
|
||||
)) {
|
||||
when(log.logStartOffset()).thenReturn(0L);
|
||||
when(log.highWatermark()).thenReturn(5L, 7L, 7L);
|
||||
|
@ -551,6 +562,79 @@ class CoordinatorLoaderImplTest {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testUpdateLastWrittenOffsetCommitInterval() throws Exception {
|
||||
TopicPartition tp = new TopicPartition("foo", 0);
|
||||
UnifiedLog log = mock(UnifiedLog.class);
|
||||
Function<TopicPartition, Optional<UnifiedLog>> partitionLogSupplier = partition -> Optional.of(log);
|
||||
Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier = partition -> Optional.of(7L);
|
||||
StringKeyValueDeserializer serde = new StringKeyValueDeserializer();
|
||||
CoordinatorPlayback<Map.Entry<String, String>> coordinator = mock(CoordinatorPlayback.class);
|
||||
|
||||
try (CoordinatorLoaderImpl<Map.Entry<String, String>> loader = new CoordinatorLoaderImpl<>(
|
||||
Time.SYSTEM,
|
||||
partitionLogSupplier,
|
||||
partitionLogEndOffsetSupplier,
|
||||
serde,
|
||||
1000,
|
||||
2L
|
||||
)) {
|
||||
when(log.logStartOffset()).thenReturn(0L);
|
||||
when(log.highWatermark()).thenReturn(7L);
|
||||
|
||||
FetchDataInfo readResult1 = logReadResult(0, Arrays.asList(
|
||||
new SimpleRecord("k1".getBytes(), "v1".getBytes()),
|
||||
new SimpleRecord("k2".getBytes(), "v2".getBytes())
|
||||
));
|
||||
|
||||
when(log.read(0L, 1000, FetchIsolation.LOG_END, true))
|
||||
.thenReturn(readResult1);
|
||||
|
||||
FetchDataInfo readResult2 = logReadResult(2, Arrays.asList(
|
||||
new SimpleRecord("k3".getBytes(), "v3".getBytes()),
|
||||
new SimpleRecord("k4".getBytes(), "v4".getBytes()),
|
||||
new SimpleRecord("k5".getBytes(), "v5".getBytes())
|
||||
));
|
||||
|
||||
when(log.read(2L, 1000, FetchIsolation.LOG_END, true))
|
||||
.thenReturn(readResult2);
|
||||
|
||||
FetchDataInfo readResult3 = logReadResult(5, Arrays.asList(
|
||||
new SimpleRecord("k6".getBytes(), "v6".getBytes())
|
||||
));
|
||||
|
||||
when(log.read(5L, 1000, FetchIsolation.LOG_END, true))
|
||||
.thenReturn(readResult3);
|
||||
|
||||
FetchDataInfo readResult4 = logReadResult(6, Arrays.asList(
|
||||
new SimpleRecord("k7".getBytes(), "v7".getBytes())
|
||||
));
|
||||
|
||||
when(log.read(6L, 1000, FetchIsolation.LOG_END, true))
|
||||
.thenReturn(readResult4);
|
||||
|
||||
assertNotNull(loader.load(tp, coordinator).get(10, TimeUnit.SECONDS));
|
||||
|
||||
verify(coordinator).replay(0L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, Map.entry("k1", "v1"));
|
||||
verify(coordinator).replay(1L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, Map.entry("k2", "v2"));
|
||||
verify(coordinator).replay(2L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, Map.entry("k3", "v3"));
|
||||
verify(coordinator).replay(3L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, Map.entry("k4", "v4"));
|
||||
verify(coordinator).replay(4L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, Map.entry("k5", "v5"));
|
||||
verify(coordinator).replay(5L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, Map.entry("k6", "v6"));
|
||||
verify(coordinator).replay(6L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, Map.entry("k7", "v7"));
|
||||
verify(coordinator, times(0)).updateLastWrittenOffset(0L);
|
||||
verify(coordinator, times(1)).updateLastWrittenOffset(2L);
|
||||
verify(coordinator, times(1)).updateLastWrittenOffset(5L);
|
||||
verify(coordinator, times(0)).updateLastWrittenOffset(6L);
|
||||
verify(coordinator, times(1)).updateLastWrittenOffset(7L);
|
||||
verify(coordinator, times(0)).updateLastCommittedOffset(0L);
|
||||
verify(coordinator, times(1)).updateLastCommittedOffset(2L);
|
||||
verify(coordinator, times(1)).updateLastCommittedOffset(5L);
|
||||
verify(coordinator, times(0)).updateLastCommittedOffset(6L);
|
||||
verify(coordinator, times(1)).updateLastCommittedOffset(7L);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
void testPartitionGoesOfflineDuringLoad() throws Exception {
|
||||
TopicPartition tp = new TopicPartition("foo", 0);
|
||||
|
@ -565,7 +649,8 @@ class CoordinatorLoaderImplTest {
|
|||
partitionLogSupplier,
|
||||
partitionLogEndOffsetSupplier,
|
||||
serde,
|
||||
1000
|
||||
1000,
|
||||
CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS
|
||||
)) {
|
||||
when(log.logStartOffset()).thenReturn(0L);
|
||||
when(log.highWatermark()).thenReturn(0L);
|
||||
|
|
|
@ -20,16 +20,13 @@ import java.io.File
|
|||
import java.net.InetSocketAddress
|
||||
import java.nio.file.Files
|
||||
import java.nio.file.Paths
|
||||
import java.util.OptionalInt
|
||||
import java.util.{OptionalInt, Collection => JCollection, Map => JMap}
|
||||
import java.util.concurrent.CompletableFuture
|
||||
import java.util.{Map => JMap}
|
||||
import java.util.{Collection => JCollection}
|
||||
import kafka.server.KafkaConfig
|
||||
import kafka.utils.CoreUtils
|
||||
import kafka.utils.Logging
|
||||
import org.apache.kafka.clients.{ApiVersions, ManualMetadataUpdater, MetadataRecoveryStrategy, NetworkClient}
|
||||
import org.apache.kafka.common.KafkaException
|
||||
import org.apache.kafka.common.Node
|
||||
import org.apache.kafka.common.TopicPartition
|
||||
import org.apache.kafka.common.Uuid
|
||||
import org.apache.kafka.common.metrics.Metrics
|
||||
|
@ -40,7 +37,7 @@ import org.apache.kafka.common.requests.RequestHeader
|
|||
import org.apache.kafka.common.security.JaasContext
|
||||
import org.apache.kafka.common.security.auth.SecurityProtocol
|
||||
import org.apache.kafka.common.utils.{LogContext, Time, Utils}
|
||||
import org.apache.kafka.raft.{Endpoints, ExternalKRaftMetrics, FileQuorumStateStore, KafkaNetworkChannel, KafkaRaftClient, KafkaRaftClientDriver, LeaderAndEpoch, MetadataLogConfig, QuorumConfig, RaftClient, ReplicatedLog, TimingWheelExpirationService}
|
||||
import org.apache.kafka.raft.{Endpoints, ExternalKRaftMetrics, FileQuorumStateStore, KafkaNetworkChannel, KafkaRaftClient, KafkaRaftClientDriver, MetadataLogConfig, QuorumConfig, RaftManager, ReplicatedLog, TimingWheelExpirationService}
|
||||
import org.apache.kafka.server.ProcessRole
|
||||
import org.apache.kafka.server.common.Feature
|
||||
import org.apache.kafka.server.common.serialization.RecordSerde
|
||||
|
@ -50,7 +47,6 @@ import org.apache.kafka.server.util.timer.SystemTimer
|
|||
import org.apache.kafka.storage.internals.log.{LogManager, UnifiedLog}
|
||||
|
||||
import scala.jdk.CollectionConverters._
|
||||
import scala.jdk.OptionConverters._
|
||||
|
||||
object KafkaRaftManager {
|
||||
private def createLogDirectory(logDir: File, logDirName: String): File = {
|
||||
|
@ -85,29 +81,6 @@ object KafkaRaftManager {
|
|||
}
|
||||
}
|
||||
|
||||
trait RaftManager[T] {
|
||||
def handleRequest(
|
||||
context: RequestContext,
|
||||
header: RequestHeader,
|
||||
request: ApiMessage,
|
||||
createdTimeMs: Long
|
||||
): CompletableFuture[ApiMessage]
|
||||
|
||||
def register(
|
||||
listener: RaftClient.Listener[T]
|
||||
): Unit
|
||||
|
||||
def leaderAndEpoch: LeaderAndEpoch
|
||||
|
||||
def client: RaftClient[T]
|
||||
|
||||
def replicatedLog: ReplicatedLog
|
||||
|
||||
def voterNode(id: Int, listener: ListenerName): Option[Node]
|
||||
|
||||
def recordSerde: RecordSerde[T]
|
||||
}
|
||||
|
||||
class KafkaRaftManager[T](
|
||||
clusterId: String,
|
||||
config: KafkaConfig,
|
||||
|
@ -178,12 +151,6 @@ class KafkaRaftManager[T](
|
|||
CoreUtils.swallow(dataDirLock.foreach(_.destroy()), this)
|
||||
}
|
||||
|
||||
override def register(
|
||||
listener: RaftClient.Listener[T]
|
||||
): Unit = {
|
||||
client.register(listener)
|
||||
}
|
||||
|
||||
override def handleRequest(
|
||||
context: RequestContext,
|
||||
header: RequestHeader,
|
||||
|
@ -292,13 +259,5 @@ class KafkaRaftManager[T](
|
|||
(controllerListenerName, networkClient)
|
||||
}
|
||||
|
||||
override def leaderAndEpoch: LeaderAndEpoch = {
|
||||
client.leaderAndEpoch
|
||||
}
|
||||
|
||||
override def voterNode(id: Int, listener: ListenerName): Option[Node] = {
|
||||
client.voterNode(id, listener).toScala
|
||||
}
|
||||
|
||||
override def recordSerde: RecordSerde[T] = serde
|
||||
}
|
|
@ -18,6 +18,7 @@
|
|||
package kafka.server
|
||||
|
||||
import java.util.concurrent.ConcurrentHashMap
|
||||
import java.util.concurrent.locks.ReentrantLock
|
||||
import java.util.{Collections, Properties}
|
||||
import kafka.coordinator.transaction.TransactionCoordinator
|
||||
import kafka.utils.Logging
|
||||
|
@ -35,6 +36,7 @@ import org.apache.kafka.coordinator.share.ShareCoordinator
|
|||
import org.apache.kafka.coordinator.transaction.TransactionLogConfig
|
||||
import org.apache.kafka.server.common.{ControllerRequestCompletionHandler, NodeToControllerChannelManager}
|
||||
import org.apache.kafka.server.quota.ControllerMutationQuota
|
||||
import org.apache.kafka.common.utils.Time
|
||||
|
||||
import scala.collection.{Map, Seq, Set, mutable}
|
||||
import scala.jdk.CollectionConverters._
|
||||
|
@ -50,21 +52,96 @@ trait AutoTopicCreationManager {
|
|||
|
||||
def createStreamsInternalTopics(
|
||||
topics: Map[String, CreatableTopic],
|
||||
requestContext: RequestContext
|
||||
requestContext: RequestContext,
|
||||
timeoutMs: Long
|
||||
): Unit
|
||||
|
||||
def getStreamsInternalTopicCreationErrors(
|
||||
topicNames: Set[String],
|
||||
currentTimeMs: Long
|
||||
): Map[String, String]
|
||||
|
||||
def close(): Unit = {}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Thread-safe cache that stores topic creation errors with per-entry expiration.
|
||||
* - Expiration: maintained by a min-heap (priority queue) on expiration time
|
||||
* - Capacity: enforced by evicting entries with earliest expiration time (not LRU)
|
||||
* - Updates: old entries remain in queue but are ignored via reference equality check
|
||||
*/
|
||||
private[server] class ExpiringErrorCache(maxSize: Int, time: Time) {
|
||||
|
||||
private case class Entry(topicName: String, errorMessage: String, expirationTimeMs: Long)
|
||||
|
||||
private val byTopic = new ConcurrentHashMap[String, Entry]()
|
||||
private val expiryQueue = new java.util.PriorityQueue[Entry](11, new java.util.Comparator[Entry] {
|
||||
override def compare(a: Entry, b: Entry): Int = java.lang.Long.compare(a.expirationTimeMs, b.expirationTimeMs)
|
||||
})
|
||||
private val lock = new ReentrantLock()
|
||||
|
||||
def put(topicName: String, errorMessage: String, ttlMs: Long): Unit = {
|
||||
lock.lock()
|
||||
try {
|
||||
val currentTimeMs = time.milliseconds()
|
||||
val expirationTimeMs = currentTimeMs + ttlMs
|
||||
val entry = Entry(topicName, errorMessage, expirationTimeMs)
|
||||
byTopic.put(topicName, entry)
|
||||
expiryQueue.add(entry)
|
||||
|
||||
// Clean up expired entries and enforce capacity
|
||||
while (!expiryQueue.isEmpty &&
|
||||
(expiryQueue.peek().expirationTimeMs <= currentTimeMs || byTopic.size() > maxSize)) {
|
||||
val evicted = expiryQueue.poll()
|
||||
val current = byTopic.get(evicted.topicName)
|
||||
if (current != null && (current eq evicted)) {
|
||||
byTopic.remove(evicted.topicName)
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
lock.unlock()
|
||||
}
|
||||
}
|
||||
|
||||
def getErrorsForTopics(topicNames: Set[String], currentTimeMs: Long): Map[String, String] = {
|
||||
val result = mutable.Map.empty[String, String]
|
||||
topicNames.foreach { topicName =>
|
||||
val entry = byTopic.get(topicName)
|
||||
if (entry != null && entry.expirationTimeMs > currentTimeMs) {
|
||||
result.put(topicName, entry.errorMessage)
|
||||
}
|
||||
}
|
||||
result.toMap
|
||||
}
|
||||
|
||||
private[server] def clear(): Unit = {
|
||||
lock.lock()
|
||||
try {
|
||||
byTopic.clear()
|
||||
expiryQueue.clear()
|
||||
} finally {
|
||||
lock.unlock()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
class DefaultAutoTopicCreationManager(
|
||||
config: KafkaConfig,
|
||||
channelManager: NodeToControllerChannelManager,
|
||||
groupCoordinator: GroupCoordinator,
|
||||
txnCoordinator: TransactionCoordinator,
|
||||
shareCoordinator: ShareCoordinator
|
||||
shareCoordinator: ShareCoordinator,
|
||||
time: Time,
|
||||
topicErrorCacheCapacity: Int = 1000
|
||||
) extends AutoTopicCreationManager with Logging {
|
||||
|
||||
private val inflightTopics = Collections.newSetFromMap(new ConcurrentHashMap[String, java.lang.Boolean]())
|
||||
|
||||
// Hardcoded default capacity; can be overridden in tests via constructor param
|
||||
private val topicCreationErrorCache = new ExpiringErrorCache(topicErrorCacheCapacity, time)
|
||||
|
||||
/**
|
||||
* Initiate auto topic creation for the given topics.
|
||||
*
|
||||
|
@ -93,13 +170,21 @@ class DefaultAutoTopicCreationManager(
|
|||
|
||||
override def createStreamsInternalTopics(
|
||||
topics: Map[String, CreatableTopic],
|
||||
requestContext: RequestContext
|
||||
requestContext: RequestContext,
|
||||
timeoutMs: Long
|
||||
): Unit = {
|
||||
if (topics.nonEmpty) {
|
||||
sendCreateTopicRequest(topics, Some(requestContext))
|
||||
sendCreateTopicRequestWithErrorCaching(topics, Some(requestContext), timeoutMs)
|
||||
}
|
||||
}
|
||||
|
||||
override def getStreamsInternalTopicCreationErrors(
|
||||
topicNames: Set[String],
|
||||
currentTimeMs: Long
|
||||
): Map[String, String] = {
|
||||
topicCreationErrorCache.getErrorsForTopics(topicNames, currentTimeMs)
|
||||
}
|
||||
|
||||
private def sendCreateTopicRequest(
|
||||
creatableTopics: Map[String, CreatableTopic],
|
||||
requestContext: Option[RequestContext]
|
||||
|
@ -264,4 +349,101 @@ class DefaultAutoTopicCreationManager(
|
|||
|
||||
(creatableTopics, uncreatableTopics)
|
||||
}
|
||||
|
||||
private def sendCreateTopicRequestWithErrorCaching(
|
||||
creatableTopics: Map[String, CreatableTopic],
|
||||
requestContext: Option[RequestContext],
|
||||
timeoutMs: Long
|
||||
): Seq[MetadataResponseTopic] = {
|
||||
val topicsToCreate = new CreateTopicsRequestData.CreatableTopicCollection(creatableTopics.size)
|
||||
topicsToCreate.addAll(creatableTopics.values.asJavaCollection)
|
||||
|
||||
val createTopicsRequest = new CreateTopicsRequest.Builder(
|
||||
new CreateTopicsRequestData()
|
||||
.setTimeoutMs(config.requestTimeoutMs)
|
||||
.setTopics(topicsToCreate)
|
||||
)
|
||||
|
||||
val requestCompletionHandler = new ControllerRequestCompletionHandler {
|
||||
override def onTimeout(): Unit = {
|
||||
clearInflightRequests(creatableTopics)
|
||||
debug(s"Auto topic creation timed out for ${creatableTopics.keys}.")
|
||||
cacheTopicCreationErrors(creatableTopics.keys.toSet, "Auto topic creation timed out.", timeoutMs)
|
||||
}
|
||||
|
||||
override def onComplete(response: ClientResponse): Unit = {
|
||||
clearInflightRequests(creatableTopics)
|
||||
if (response.authenticationException() != null) {
|
||||
val authException = response.authenticationException()
|
||||
warn(s"Auto topic creation failed for ${creatableTopics.keys} with authentication exception: ${authException.getMessage}")
|
||||
cacheTopicCreationErrors(creatableTopics.keys.toSet, authException.getMessage, timeoutMs)
|
||||
} else if (response.versionMismatch() != null) {
|
||||
val versionException = response.versionMismatch()
|
||||
warn(s"Auto topic creation failed for ${creatableTopics.keys} with version mismatch exception: ${versionException.getMessage}")
|
||||
cacheTopicCreationErrors(creatableTopics.keys.toSet, versionException.getMessage, timeoutMs)
|
||||
} else {
|
||||
response.responseBody() match {
|
||||
case createTopicsResponse: CreateTopicsResponse =>
|
||||
cacheTopicCreationErrorsFromResponse(createTopicsResponse, timeoutMs)
|
||||
case _ =>
|
||||
debug(s"Auto topic creation completed for ${creatableTopics.keys} with response ${response.responseBody}.")
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
val request = requestContext.map { context =>
|
||||
val requestVersion =
|
||||
channelManager.controllerApiVersions.toScala match {
|
||||
case None =>
|
||||
// We will rely on the Metadata request to be retried in the case
|
||||
// that the latest version is not usable by the controller.
|
||||
ApiKeys.CREATE_TOPICS.latestVersion()
|
||||
case Some(nodeApiVersions) =>
|
||||
nodeApiVersions.latestUsableVersion(ApiKeys.CREATE_TOPICS)
|
||||
}
|
||||
|
||||
// Borrow client information such as client id and correlation id from the original request,
|
||||
// in order to correlate the create request with the original metadata request.
|
||||
val requestHeader = new RequestHeader(ApiKeys.CREATE_TOPICS,
|
||||
requestVersion,
|
||||
context.clientId,
|
||||
context.correlationId)
|
||||
ForwardingManager.buildEnvelopeRequest(context,
|
||||
createTopicsRequest.build(requestVersion).serializeWithHeader(requestHeader))
|
||||
}.getOrElse(createTopicsRequest)
|
||||
|
||||
channelManager.sendRequest(request, requestCompletionHandler)
|
||||
|
||||
val creatableTopicResponses = creatableTopics.keySet.toSeq.map { topic =>
|
||||
new MetadataResponseTopic()
|
||||
.setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code)
|
||||
.setName(topic)
|
||||
.setIsInternal(Topic.isInternal(topic))
|
||||
}
|
||||
|
||||
creatableTopicResponses
|
||||
}
|
||||
|
||||
private def cacheTopicCreationErrors(topicNames: Set[String], errorMessage: String, ttlMs: Long): Unit = {
|
||||
topicNames.foreach { topicName =>
|
||||
topicCreationErrorCache.put(topicName, errorMessage, ttlMs)
|
||||
}
|
||||
}
|
||||
|
||||
private def cacheTopicCreationErrorsFromResponse(response: CreateTopicsResponse, ttlMs: Long): Unit = {
|
||||
response.data().topics().forEach { topicResult =>
|
||||
if (topicResult.errorCode() != Errors.NONE.code()) {
|
||||
val errorMessage = Option(topicResult.errorMessage())
|
||||
.filter(_.nonEmpty)
|
||||
.getOrElse(Errors.forCode(topicResult.errorCode()).message())
|
||||
topicCreationErrorCache.put(topicResult.name(), errorMessage, ttlMs)
|
||||
debug(s"Cached topic creation error for ${topicResult.name()}: $errorMessage")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
override def close(): Unit = {
|
||||
topicCreationErrorCache.clear()
|
||||
}
|
||||
}
|
||||
|
|
|
@ -387,7 +387,7 @@ class BrokerServer(
|
|||
|
||||
autoTopicCreationManager = new DefaultAutoTopicCreationManager(
|
||||
config, clientToControllerChannelManager, groupCoordinator,
|
||||
transactionCoordinator, shareCoordinator)
|
||||
transactionCoordinator, shareCoordinator, time)
|
||||
|
||||
dynamicConfigHandlers = Map[ConfigType, ConfigHandler](
|
||||
ConfigType.TOPIC -> new TopicConfigHandler(replicaManager, config, quotaManagers),
|
||||
|
@ -613,7 +613,8 @@ class BrokerServer(
|
|||
tp => replicaManager.getLog(tp).toJava,
|
||||
tp => replicaManager.getLogEndOffset(tp).map(Long.box).toJava,
|
||||
serde,
|
||||
config.groupCoordinatorConfig.offsetsLoadBufferSize
|
||||
config.groupCoordinatorConfig.offsetsLoadBufferSize,
|
||||
CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS
|
||||
)
|
||||
val writer = new CoordinatorPartitionWriter(
|
||||
replicaManager
|
||||
|
@ -644,7 +645,8 @@ class BrokerServer(
|
|||
tp => replicaManager.getLog(tp).toJava,
|
||||
tp => replicaManager.getLogEndOffset(tp).map(Long.box).toJava,
|
||||
serde,
|
||||
config.shareCoordinatorConfig.shareCoordinatorLoadBufferSize()
|
||||
config.shareCoordinatorConfig.shareCoordinatorLoadBufferSize(),
|
||||
CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS
|
||||
)
|
||||
val writer = new CoordinatorPartitionWriter(
|
||||
replicaManager
|
||||
|
@ -780,6 +782,9 @@ class BrokerServer(
|
|||
if (shareCoordinator != null)
|
||||
CoreUtils.swallow(shareCoordinator.shutdown(), this)
|
||||
|
||||
if (autoTopicCreationManager != null)
|
||||
CoreUtils.swallow(autoTopicCreationManager.close(), this)
|
||||
|
||||
if (assignmentsManager != null)
|
||||
CoreUtils.swallow(assignmentsManager.close(), this)
|
||||
|
||||
|
|
|
@ -24,7 +24,6 @@ import java.util.Map.Entry
|
|||
import java.util.concurrent.CompletableFuture
|
||||
import java.util.function.Consumer
|
||||
import kafka.network.RequestChannel
|
||||
import kafka.raft.RaftManager
|
||||
import kafka.server.QuotaFactory.QuotaManagers
|
||||
import kafka.server.logger.RuntimeLoggerManager
|
||||
import kafka.server.metadata.KRaftMetadataCache
|
||||
|
@ -55,6 +54,7 @@ import org.apache.kafka.image.publisher.ControllerRegistrationsPublisher
|
|||
import org.apache.kafka.metadata.{BrokerHeartbeatReply, BrokerRegistrationReply}
|
||||
import org.apache.kafka.common.security.auth.KafkaPrincipal
|
||||
import org.apache.kafka.common.security.auth.SecurityProtocol
|
||||
import org.apache.kafka.raft.RaftManager
|
||||
import org.apache.kafka.server.{ApiVersionManager, DelegationTokenManager, ProcessRole}
|
||||
import org.apache.kafka.server.authorizer.Authorizer
|
||||
import org.apache.kafka.server.common.{ApiMessageAndVersion, RequestLocal}
|
||||
|
@ -1070,7 +1070,7 @@ class ControllerApis(
|
|||
EndpointType.CONTROLLER,
|
||||
clusterId,
|
||||
() => registrationsPublisher.describeClusterControllers(request.context.listenerName()),
|
||||
() => raftManager.leaderAndEpoch.leaderId().orElse(-1)
|
||||
() => raftManager.client.leaderAndEpoch.leaderId().orElse(-1)
|
||||
)
|
||||
requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
|
||||
new DescribeClusterResponse(response.setThrottleTimeMs(requestThrottleMs)))
|
||||
|
|
|
@ -2812,10 +2812,35 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
)
|
||||
}
|
||||
} else {
|
||||
autoTopicCreationManager.createStreamsInternalTopics(topicsToCreate, requestContext);
|
||||
}
|
||||
}
|
||||
// Compute group-specific timeout for caching errors (2 * heartbeat interval)
|
||||
val heartbeatIntervalMs = Option(groupConfigManager.groupConfig(streamsGroupHeartbeatRequest.data.groupId).orElse(null))
|
||||
.map(_.streamsHeartbeatIntervalMs().toLong)
|
||||
.getOrElse(config.groupCoordinatorConfig.streamsGroupHeartbeatIntervalMs().toLong)
|
||||
val timeoutMs = heartbeatIntervalMs * 2
|
||||
|
||||
autoTopicCreationManager.createStreamsInternalTopics(topicsToCreate, requestContext, timeoutMs)
|
||||
|
||||
// Check for cached topic creation errors only if there's already a MISSING_INTERNAL_TOPICS status
|
||||
val hasMissingInternalTopicsStatus = responseData.status() != null &&
|
||||
responseData.status().stream().anyMatch(s => s.statusCode() == StreamsGroupHeartbeatResponse.Status.MISSING_INTERNAL_TOPICS.code())
|
||||
|
||||
if (hasMissingInternalTopicsStatus) {
|
||||
val currentTimeMs = time.milliseconds()
|
||||
val cachedErrors = autoTopicCreationManager.getStreamsInternalTopicCreationErrors(topicsToCreate.keys.toSet, currentTimeMs)
|
||||
if (cachedErrors.nonEmpty) {
|
||||
val missingInternalTopicStatus =
|
||||
responseData.status().stream().filter(x => x.statusCode() == StreamsGroupHeartbeatResponse.Status.MISSING_INTERNAL_TOPICS.code()).findFirst()
|
||||
val creationErrorDetails = cachedErrors.map { case (topic, error) => s"$topic ($error)" }.mkString(", ")
|
||||
if (missingInternalTopicStatus.isPresent) {
|
||||
val existingDetail = Option(missingInternalTopicStatus.get().statusDetail()).getOrElse("")
|
||||
missingInternalTopicStatus.get().setStatusDetail(
|
||||
existingDetail + s"; Creation failed: $creationErrorDetails."
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
requestHelper.sendMaybeThrottle(request, new StreamsGroupHeartbeatResponse(responseData))
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,7 +17,6 @@
|
|||
|
||||
package kafka.server
|
||||
|
||||
import kafka.raft.RaftManager
|
||||
import kafka.utils.Logging
|
||||
import org.apache.kafka.clients._
|
||||
import org.apache.kafka.common.metrics.Metrics
|
||||
|
@ -28,6 +27,7 @@ import org.apache.kafka.common.security.JaasContext
|
|||
import org.apache.kafka.common.security.auth.SecurityProtocol
|
||||
import org.apache.kafka.common.utils.{LogContext, Time}
|
||||
import org.apache.kafka.common.{Node, Reconfigurable}
|
||||
import org.apache.kafka.raft.RaftManager
|
||||
import org.apache.kafka.server.common.{ApiMessageAndVersion, ControllerRequestCompletionHandler, NodeToControllerChannelManager}
|
||||
import org.apache.kafka.server.util.{InterBrokerSendThread, RequestAndCompletionHandler}
|
||||
|
||||
|
@ -37,7 +37,7 @@ import java.util.concurrent.LinkedBlockingDeque
|
|||
import java.util.concurrent.atomic.AtomicReference
|
||||
import scala.collection.Seq
|
||||
import scala.jdk.CollectionConverters._
|
||||
import scala.jdk.OptionConverters.{RichOption, RichOptionalInt}
|
||||
import scala.jdk.OptionConverters.{RichOption, RichOptional, RichOptionalInt}
|
||||
|
||||
case class ControllerInformation(
|
||||
node: Option[Node],
|
||||
|
@ -79,10 +79,10 @@ class RaftControllerNodeProvider(
|
|||
val saslMechanism: String
|
||||
) extends ControllerNodeProvider with Logging {
|
||||
|
||||
private def idToNode(id: Int): Option[Node] = raftManager.voterNode(id, listenerName)
|
||||
private def idToNode(id: Int): Option[Node] = raftManager.client.voterNode(id, listenerName).toScala
|
||||
|
||||
override def getControllerInfo(): ControllerInformation =
|
||||
ControllerInformation(raftManager.leaderAndEpoch.leaderId.toScala.flatMap(idToNode),
|
||||
ControllerInformation(raftManager.client.leaderAndEpoch.leaderId.toScala.flatMap(idToNode),
|
||||
listenerName, securityProtocol, saslMechanism)
|
||||
}
|
||||
|
||||
|
|
|
@ -342,7 +342,7 @@ class SharedServer(
|
|||
throw new RuntimeException("Unable to install metadata publishers.", t)
|
||||
}
|
||||
}
|
||||
_raftManager.register(loader)
|
||||
_raftManager.client.register(loader)
|
||||
debug("Completed SharedServer startup.")
|
||||
started = true
|
||||
} catch {
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
package kafka.tools
|
||||
|
||||
import kafka.network.RequestChannel
|
||||
import kafka.raft.RaftManager
|
||||
import kafka.server.ApiRequestHandler
|
||||
import kafka.utils.Logging
|
||||
import org.apache.kafka.common.internals.FatalExitError
|
||||
|
@ -26,6 +25,7 @@ import org.apache.kafka.common.message.{BeginQuorumEpochResponseData, EndQuorumE
|
|||
import org.apache.kafka.common.protocol.{ApiKeys, ApiMessage}
|
||||
import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, BeginQuorumEpochResponse, EndQuorumEpochResponse, FetchResponse, FetchSnapshotResponse, VoteResponse}
|
||||
import org.apache.kafka.common.utils.Time
|
||||
import org.apache.kafka.raft.RaftManager
|
||||
import org.apache.kafka.server.ApiVersionManager
|
||||
import org.apache.kafka.server.common.RequestLocal
|
||||
|
||||
|
|
|
@ -22,7 +22,7 @@ import java.util.concurrent.atomic.{AtomicInteger, AtomicLong}
|
|||
import java.util.concurrent.{CompletableFuture, CountDownLatch, LinkedBlockingDeque, TimeUnit}
|
||||
import joptsimple.{OptionException, OptionSpec}
|
||||
import kafka.network.SocketServer
|
||||
import kafka.raft.{DefaultExternalKRaftMetrics, KafkaRaftManager, RaftManager}
|
||||
import kafka.raft.{DefaultExternalKRaftMetrics, KafkaRaftManager}
|
||||
import kafka.server.{KafkaConfig, KafkaRequestHandlerPool}
|
||||
import kafka.utils.{CoreUtils, Logging}
|
||||
import org.apache.kafka.common.message.ApiMessageType.ListenerType
|
||||
|
@ -36,7 +36,7 @@ import org.apache.kafka.common.security.token.delegation.internals.DelegationTok
|
|||
import org.apache.kafka.common.utils.{Exit, Time, Utils}
|
||||
import org.apache.kafka.common.{TopicPartition, Uuid, protocol}
|
||||
import org.apache.kafka.raft.errors.NotLeaderException
|
||||
import org.apache.kafka.raft.{Batch, BatchReader, Endpoints, LeaderAndEpoch, QuorumConfig, RaftClient}
|
||||
import org.apache.kafka.raft.{Batch, BatchReader, Endpoints, LeaderAndEpoch, QuorumConfig, RaftClient, RaftManager}
|
||||
import org.apache.kafka.security.CredentialProvider
|
||||
import org.apache.kafka.server.SimpleApiVersionManager
|
||||
import org.apache.kafka.server.common.{FinalizedFeatures, MetadataVersion}
|
||||
|
@ -180,7 +180,7 @@ class TestRaftServer(
|
|||
|
||||
private var claimedEpoch: Option[Int] = None
|
||||
|
||||
raftManager.register(this)
|
||||
raftManager.client.register(this)
|
||||
|
||||
override def handleLeaderChange(newLeaderAndEpoch: LeaderAndEpoch): Unit = {
|
||||
if (newLeaderAndEpoch.isLeader(config.nodeId)) {
|
||||
|
|
|
@ -30,6 +30,7 @@ import org.apache.kafka.common.Uuid
|
|||
import org.apache.kafka.common.metrics.Metrics
|
||||
import org.apache.kafka.common.network.ListenerName
|
||||
import org.apache.kafka.common.utils.Time
|
||||
import org.apache.kafka.common.utils.Utils
|
||||
import org.apache.kafka.network.SocketServerConfigs
|
||||
import org.apache.kafka.raft.{Endpoints, MetadataLogConfig, QuorumConfig}
|
||||
import org.apache.kafka.server.ProcessRole
|
||||
|
@ -126,6 +127,7 @@ class RaftManagerTest {
|
|||
|
||||
val logDir = TestUtils.tempDir()
|
||||
val nodeId = 1
|
||||
try {
|
||||
val raftManager = createRaftManager(
|
||||
new TopicPartition("__raft_id_test", 0),
|
||||
createConfig(
|
||||
|
@ -137,6 +139,9 @@ class RaftManagerTest {
|
|||
)
|
||||
assertEquals(nodeId, raftManager.client.nodeId.getAsInt)
|
||||
raftManager.shutdown()
|
||||
} finally {
|
||||
Utils.delete(logDir)
|
||||
}
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
|
@ -155,6 +160,7 @@ class RaftManagerTest {
|
|||
}
|
||||
|
||||
val nodeId = 1
|
||||
try {
|
||||
val raftManager = createRaftManager(
|
||||
new TopicPartition("__raft_id_test", 0),
|
||||
createConfig(
|
||||
|
@ -171,6 +177,10 @@ class RaftManagerTest {
|
|||
raftManager.shutdown()
|
||||
|
||||
assertFalse(fileLocked(lockPath))
|
||||
} finally {
|
||||
logDir.foreach(p => Utils.delete(p.toFile))
|
||||
metadataDir.foreach(p => Utils.delete(p.toFile))
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -179,6 +189,7 @@ class RaftManagerTest {
|
|||
val metadataDir = Some(TestUtils.tempDir().toPath)
|
||||
|
||||
val nodeId = 1
|
||||
try {
|
||||
val raftManager = createRaftManager(
|
||||
new TopicPartition("__raft_id_test", 0),
|
||||
createConfig(
|
||||
|
@ -195,6 +206,10 @@ class RaftManagerTest {
|
|||
raftManager.shutdown()
|
||||
|
||||
assertFalse(fileLocked(lockPath))
|
||||
} finally {
|
||||
logDir.foreach(p => Utils.delete(p.toFile))
|
||||
metadataDir.foreach(p => Utils.delete(p.toFile))
|
||||
}
|
||||
}
|
||||
|
||||
def createMetadataLog(config: KafkaConfig): Unit = {
|
||||
|
|
|
@ -34,7 +34,7 @@ class AllocateProducerIdsRequestTest(cluster: ClusterInstance) {
|
|||
def testAllocateProducersIdSentToController(): Unit = {
|
||||
val sourceBroker = cluster.brokers.values().stream().findFirst().get().asInstanceOf[BrokerServer]
|
||||
|
||||
val controllerId = sourceBroker.raftManager.leaderAndEpoch.leaderId().getAsInt
|
||||
val controllerId = sourceBroker.raftManager.client.leaderAndEpoch.leaderId().getAsInt
|
||||
val controllerServer = cluster.controllers.values().stream()
|
||||
.filter(_.config.nodeId == controllerId)
|
||||
.findFirst()
|
||||
|
@ -50,7 +50,7 @@ class AllocateProducerIdsRequestTest(cluster: ClusterInstance) {
|
|||
def testAllocateProducersIdSentToNonController(): Unit = {
|
||||
val sourceBroker = cluster.brokers.values().stream().findFirst().get().asInstanceOf[BrokerServer]
|
||||
|
||||
val controllerId = sourceBroker.raftManager.leaderAndEpoch.leaderId().getAsInt
|
||||
val controllerId = sourceBroker.raftManager.client.leaderAndEpoch.leaderId().getAsInt
|
||||
val controllerServer = cluster.controllers().values().stream()
|
||||
.filter(_.config.nodeId != controllerId)
|
||||
.findFirst()
|
||||
|
|
|
@ -35,6 +35,7 @@ import org.apache.kafka.common.protocol.{ApiKeys, ByteBufferAccessor, Errors}
|
|||
import org.apache.kafka.common.requests._
|
||||
import org.apache.kafka.common.security.auth.{KafkaPrincipal, KafkaPrincipalSerde, SecurityProtocol}
|
||||
import org.apache.kafka.common.utils.{SecurityUtils, Utils}
|
||||
import org.apache.kafka.server.util.MockTime
|
||||
import org.apache.kafka.coordinator.group.{GroupCoordinator, GroupCoordinatorConfig}
|
||||
import org.apache.kafka.coordinator.share.{ShareCoordinator, ShareCoordinatorConfig}
|
||||
import org.apache.kafka.metadata.MetadataCache
|
||||
|
@ -45,14 +46,15 @@ import org.apache.kafka.server.quota.ControllerMutationQuota
|
|||
import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows, assertTrue}
|
||||
import org.junit.jupiter.api.{BeforeEach, Test}
|
||||
import org.mockito.ArgumentMatchers.any
|
||||
import org.mockito.Mockito.never
|
||||
import org.mockito.{ArgumentCaptor, ArgumentMatchers, Mockito}
|
||||
import org.mockito.Mockito.never
|
||||
|
||||
import scala.collection.{Map, Seq}
|
||||
|
||||
class AutoTopicCreationManagerTest {
|
||||
|
||||
private val requestTimeout = 100
|
||||
private val testCacheCapacity = 3
|
||||
private var config: KafkaConfig = _
|
||||
private val metadataCache = Mockito.mock(classOf[MetadataCache])
|
||||
private val brokerToController = Mockito.mock(classOf[NodeToControllerChannelManager])
|
||||
|
@ -60,6 +62,7 @@ class AutoTopicCreationManagerTest {
|
|||
private val transactionCoordinator = Mockito.mock(classOf[TransactionCoordinator])
|
||||
private val shareCoordinator = Mockito.mock(classOf[ShareCoordinator])
|
||||
private var autoTopicCreationManager: AutoTopicCreationManager = _
|
||||
private val mockTime = new MockTime(0L, 0L)
|
||||
|
||||
private val internalTopicPartitions = 2
|
||||
private val internalTopicReplicationFactor: Short = 2
|
||||
|
@ -76,6 +79,8 @@ class AutoTopicCreationManagerTest {
|
|||
props.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, internalTopicReplicationFactor.toString)
|
||||
props.setProperty(TransactionLogConfig.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, internalTopicReplicationFactor.toString)
|
||||
props.setProperty(ShareCoordinatorConfig.STATE_TOPIC_NUM_PARTITIONS_CONFIG, internalTopicReplicationFactor.toString)
|
||||
// Set a short group max session timeout for testing TTL (1 second)
|
||||
props.setProperty(GroupCoordinatorConfig.GROUP_MAX_SESSION_TIMEOUT_MS_CONFIG, "1000")
|
||||
|
||||
config = KafkaConfig.fromProps(props)
|
||||
val aliveBrokers = util.List.of(new Node(0, "host0", 0), new Node(1, "host1", 1))
|
||||
|
@ -115,7 +120,9 @@ class AutoTopicCreationManagerTest {
|
|||
brokerToController,
|
||||
groupCoordinator,
|
||||
transactionCoordinator,
|
||||
shareCoordinator)
|
||||
shareCoordinator,
|
||||
mockTime,
|
||||
topicErrorCacheCapacity = testCacheCapacity)
|
||||
|
||||
val topicsCollection = new CreateTopicsRequestData.CreatableTopicCollection
|
||||
topicsCollection.add(getNewTopic(topicName, numPartitions, replicationFactor))
|
||||
|
@ -231,9 +238,11 @@ class AutoTopicCreationManagerTest {
|
|||
brokerToController,
|
||||
groupCoordinator,
|
||||
transactionCoordinator,
|
||||
shareCoordinator)
|
||||
shareCoordinator,
|
||||
mockTime,
|
||||
topicErrorCacheCapacity = testCacheCapacity)
|
||||
|
||||
autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext)
|
||||
autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext, config.groupCoordinatorConfig.streamsGroupHeartbeatIntervalMs() * 2)
|
||||
|
||||
val argumentCaptor = ArgumentCaptor.forClass(classOf[AbstractRequest.Builder[_ <: AbstractRequest]])
|
||||
Mockito.verify(brokerToController).sendRequest(
|
||||
|
@ -267,9 +276,11 @@ class AutoTopicCreationManagerTest {
|
|||
brokerToController,
|
||||
groupCoordinator,
|
||||
transactionCoordinator,
|
||||
shareCoordinator)
|
||||
shareCoordinator,
|
||||
mockTime,
|
||||
topicErrorCacheCapacity = testCacheCapacity)
|
||||
|
||||
autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext)
|
||||
autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext, config.groupCoordinatorConfig.streamsGroupHeartbeatIntervalMs() * 2)
|
||||
|
||||
Mockito.verify(brokerToController, never()).sendRequest(
|
||||
any(classOf[AbstractRequest.Builder[_ <: AbstractRequest]]),
|
||||
|
@ -288,9 +299,11 @@ class AutoTopicCreationManagerTest {
|
|||
brokerToController,
|
||||
groupCoordinator,
|
||||
transactionCoordinator,
|
||||
shareCoordinator)
|
||||
shareCoordinator,
|
||||
mockTime,
|
||||
topicErrorCacheCapacity = testCacheCapacity)
|
||||
|
||||
autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext)
|
||||
autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext, config.groupCoordinatorConfig.streamsGroupHeartbeatIntervalMs() * 2)
|
||||
|
||||
val argumentCaptor = ArgumentCaptor.forClass(classOf[AbstractRequest.Builder[_ <: AbstractRequest]])
|
||||
Mockito.verify(brokerToController).sendRequest(
|
||||
|
@ -319,7 +332,9 @@ class AutoTopicCreationManagerTest {
|
|||
brokerToController,
|
||||
groupCoordinator,
|
||||
transactionCoordinator,
|
||||
shareCoordinator)
|
||||
shareCoordinator,
|
||||
mockTime,
|
||||
topicErrorCacheCapacity = testCacheCapacity)
|
||||
|
||||
val createTopicApiVersion = new ApiVersionsResponseData.ApiVersion()
|
||||
.setApiKey(ApiKeys.CREATE_TOPICS.id)
|
||||
|
@ -356,4 +371,217 @@ class AutoTopicCreationManagerTest {
|
|||
.setNumPartitions(numPartitions)
|
||||
.setReplicationFactor(replicationFactor)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testTopicCreationErrorCaching(): Unit = {
|
||||
autoTopicCreationManager = new DefaultAutoTopicCreationManager(
|
||||
config,
|
||||
brokerToController,
|
||||
groupCoordinator,
|
||||
transactionCoordinator,
|
||||
shareCoordinator,
|
||||
mockTime,
|
||||
topicErrorCacheCapacity = testCacheCapacity)
|
||||
|
||||
val topics = Map(
|
||||
"test-topic-1" -> new CreatableTopic().setName("test-topic-1").setNumPartitions(1).setReplicationFactor(1)
|
||||
)
|
||||
val requestContext = initializeRequestContextWithUserPrincipal()
|
||||
|
||||
autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext, config.groupCoordinatorConfig.streamsGroupHeartbeatIntervalMs() * 2)
|
||||
|
||||
val argumentCaptor = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler])
|
||||
Mockito.verify(brokerToController).sendRequest(
|
||||
any(classOf[AbstractRequest.Builder[_ <: AbstractRequest]]),
|
||||
argumentCaptor.capture())
|
||||
|
||||
// Simulate a CreateTopicsResponse with errors
|
||||
val createTopicsResponseData = new org.apache.kafka.common.message.CreateTopicsResponseData()
|
||||
val topicResult = new org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult()
|
||||
.setName("test-topic-1")
|
||||
.setErrorCode(Errors.TOPIC_ALREADY_EXISTS.code())
|
||||
.setErrorMessage("Topic 'test-topic-1' already exists.")
|
||||
createTopicsResponseData.topics().add(topicResult)
|
||||
|
||||
val createTopicsResponse = new CreateTopicsResponse(createTopicsResponseData)
|
||||
val header = new RequestHeader(ApiKeys.CREATE_TOPICS, 0, "client", 1)
|
||||
val clientResponse = new ClientResponse(header, null, null,
|
||||
0, 0, false, null, null, createTopicsResponse)
|
||||
|
||||
// Trigger the completion handler
|
||||
argumentCaptor.getValue.onComplete(clientResponse)
|
||||
|
||||
// Verify that the error was cached
|
||||
val cachedErrors = autoTopicCreationManager.getStreamsInternalTopicCreationErrors(Set("test-topic-1"), mockTime.milliseconds())
|
||||
assertEquals(1, cachedErrors.size)
|
||||
assertTrue(cachedErrors.contains("test-topic-1"))
|
||||
assertEquals("Topic 'test-topic-1' already exists.", cachedErrors("test-topic-1"))
|
||||
}
|
||||
|
||||
@Test
|
||||
def testGetTopicCreationErrorsWithMultipleTopics(): Unit = {
|
||||
autoTopicCreationManager = new DefaultAutoTopicCreationManager(
|
||||
config,
|
||||
brokerToController,
|
||||
groupCoordinator,
|
||||
transactionCoordinator,
|
||||
shareCoordinator,
|
||||
mockTime,
|
||||
topicErrorCacheCapacity = testCacheCapacity)
|
||||
|
||||
val topics = Map(
|
||||
"success-topic" -> new CreatableTopic().setName("success-topic").setNumPartitions(1).setReplicationFactor(1),
|
||||
"failed-topic" -> new CreatableTopic().setName("failed-topic").setNumPartitions(1).setReplicationFactor(1)
|
||||
)
|
||||
val requestContext = initializeRequestContextWithUserPrincipal()
|
||||
autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext, config.groupCoordinatorConfig.streamsGroupHeartbeatIntervalMs() * 2)
|
||||
|
||||
val argumentCaptor = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler])
|
||||
Mockito.verify(brokerToController).sendRequest(
|
||||
any(classOf[AbstractRequest.Builder[_ <: AbstractRequest]]),
|
||||
argumentCaptor.capture())
|
||||
|
||||
// Simulate mixed response - one success, one failure
|
||||
val createTopicsResponseData = new org.apache.kafka.common.message.CreateTopicsResponseData()
|
||||
createTopicsResponseData.topics().add(
|
||||
new org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult()
|
||||
.setName("success-topic")
|
||||
.setErrorCode(Errors.NONE.code())
|
||||
)
|
||||
createTopicsResponseData.topics().add(
|
||||
new org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult()
|
||||
.setName("failed-topic")
|
||||
.setErrorCode(Errors.POLICY_VIOLATION.code())
|
||||
.setErrorMessage("Policy violation")
|
||||
)
|
||||
|
||||
val createTopicsResponse = new CreateTopicsResponse(createTopicsResponseData)
|
||||
val header = new RequestHeader(ApiKeys.CREATE_TOPICS, 0, "client", 1)
|
||||
val clientResponse = new ClientResponse(header, null, null,
|
||||
0, 0, false, null, null, createTopicsResponse)
|
||||
|
||||
argumentCaptor.getValue.onComplete(clientResponse)
|
||||
|
||||
// Only the failed topic should be cached
|
||||
val cachedErrors = autoTopicCreationManager.getStreamsInternalTopicCreationErrors(Set("success-topic", "failed-topic", "nonexistent-topic"), mockTime.milliseconds())
|
||||
assertEquals(1, cachedErrors.size)
|
||||
assertTrue(cachedErrors.contains("failed-topic"))
|
||||
assertEquals("Policy violation", cachedErrors("failed-topic"))
|
||||
}
|
||||
|
||||
@Test
|
||||
def testErrorCacheTTL(): Unit = {
|
||||
autoTopicCreationManager = new DefaultAutoTopicCreationManager(
|
||||
config,
|
||||
brokerToController,
|
||||
groupCoordinator,
|
||||
transactionCoordinator,
|
||||
shareCoordinator,
|
||||
mockTime,
|
||||
topicErrorCacheCapacity = testCacheCapacity)
|
||||
|
||||
|
||||
// First cache an error by simulating topic creation failure
|
||||
val topics = Map(
|
||||
"test-topic" -> new CreatableTopic().setName("test-topic").setNumPartitions(1).setReplicationFactor(1)
|
||||
)
|
||||
val requestContext = initializeRequestContextWithUserPrincipal()
|
||||
val shortTtlMs = 1000L // Use 1 second TTL for faster testing
|
||||
autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext, shortTtlMs)
|
||||
|
||||
val argumentCaptor = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler])
|
||||
Mockito.verify(brokerToController).sendRequest(
|
||||
any(classOf[AbstractRequest.Builder[_ <: AbstractRequest]]),
|
||||
argumentCaptor.capture())
|
||||
|
||||
// Simulate a CreateTopicsResponse with error
|
||||
val createTopicsResponseData = new org.apache.kafka.common.message.CreateTopicsResponseData()
|
||||
val topicResult = new org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult()
|
||||
.setName("test-topic")
|
||||
.setErrorCode(Errors.INVALID_REPLICATION_FACTOR.code())
|
||||
.setErrorMessage("Invalid replication factor")
|
||||
createTopicsResponseData.topics().add(topicResult)
|
||||
|
||||
val createTopicsResponse = new CreateTopicsResponse(createTopicsResponseData)
|
||||
val header = new RequestHeader(ApiKeys.CREATE_TOPICS, 0, "client", 1)
|
||||
val clientResponse = new ClientResponse(header, null, null,
|
||||
0, 0, false, null, null, createTopicsResponse)
|
||||
|
||||
// Cache the error at T0
|
||||
argumentCaptor.getValue.onComplete(clientResponse)
|
||||
|
||||
// Verify error is cached and accessible within TTL
|
||||
val cachedErrors = autoTopicCreationManager.getStreamsInternalTopicCreationErrors(Set("test-topic"), mockTime.milliseconds())
|
||||
assertEquals(1, cachedErrors.size)
|
||||
assertEquals("Invalid replication factor", cachedErrors("test-topic"))
|
||||
|
||||
// Advance time beyond TTL
|
||||
mockTime.sleep(shortTtlMs + 100) // T0 + 1.1 seconds
|
||||
|
||||
// Verify error is now expired and proactively cleaned up
|
||||
val expiredErrors = autoTopicCreationManager.getStreamsInternalTopicCreationErrors(Set("test-topic"), mockTime.milliseconds())
|
||||
assertTrue(expiredErrors.isEmpty, "Expired errors should be proactively cleaned up")
|
||||
}
|
||||
|
||||
@Test
|
||||
def testErrorCacheExpirationBasedEviction(): Unit = {
|
||||
// Create manager with small cache size for testing
|
||||
autoTopicCreationManager = new DefaultAutoTopicCreationManager(
|
||||
config,
|
||||
brokerToController,
|
||||
groupCoordinator,
|
||||
transactionCoordinator,
|
||||
shareCoordinator,
|
||||
mockTime,
|
||||
topicErrorCacheCapacity = 3)
|
||||
|
||||
val requestContext = initializeRequestContextWithUserPrincipal()
|
||||
|
||||
// Create 5 topics to exceed the cache size of 3
|
||||
val topicNames = (1 to 5).map(i => s"test-topic-$i")
|
||||
|
||||
// Add errors for all 5 topics to the cache
|
||||
topicNames.zipWithIndex.foreach { case (topicName, idx) =>
|
||||
val topics = Map(
|
||||
topicName -> new CreatableTopic().setName(topicName).setNumPartitions(1).setReplicationFactor(1)
|
||||
)
|
||||
|
||||
autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext, config.groupCoordinatorConfig.streamsGroupHeartbeatIntervalMs() * 2)
|
||||
|
||||
val argumentCaptor = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler])
|
||||
Mockito.verify(brokerToController, Mockito.atLeastOnce()).sendRequest(
|
||||
any(classOf[AbstractRequest.Builder[_ <: AbstractRequest]]),
|
||||
argumentCaptor.capture())
|
||||
|
||||
// Simulate error response for this topic
|
||||
val createTopicsResponseData = new org.apache.kafka.common.message.CreateTopicsResponseData()
|
||||
val topicResult = new org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult()
|
||||
.setName(topicName)
|
||||
.setErrorCode(Errors.TOPIC_ALREADY_EXISTS.code())
|
||||
.setErrorMessage(s"Topic '$topicName' already exists.")
|
||||
createTopicsResponseData.topics().add(topicResult)
|
||||
|
||||
val createTopicsResponse = new CreateTopicsResponse(createTopicsResponseData)
|
||||
val header = new RequestHeader(ApiKeys.CREATE_TOPICS, 0, "client", 1)
|
||||
val clientResponse = new ClientResponse(header, null, null,
|
||||
0, 0, false, null, null, createTopicsResponse)
|
||||
|
||||
argumentCaptor.getValue.onComplete(clientResponse)
|
||||
|
||||
// Advance time slightly between additions to ensure different timestamps
|
||||
mockTime.sleep(10)
|
||||
|
||||
}
|
||||
|
||||
// With cache size of 3, topics 1 and 2 should have been evicted
|
||||
val cachedErrors = autoTopicCreationManager.getStreamsInternalTopicCreationErrors(topicNames.toSet, mockTime.milliseconds())
|
||||
|
||||
// Only the last 3 topics should be in the cache (topics 3, 4, 5)
|
||||
assertEquals(3, cachedErrors.size, "Cache should contain only the most recent 3 entries")
|
||||
assertTrue(cachedErrors.contains("test-topic-3"), "test-topic-3 should be in cache")
|
||||
assertTrue(cachedErrors.contains("test-topic-4"), "test-topic-4 should be in cache")
|
||||
assertTrue(cachedErrors.contains("test-topic-5"), "test-topic-5 should be in cache")
|
||||
assertTrue(!cachedErrors.contains("test-topic-1"), "test-topic-1 should have been evicted")
|
||||
assertTrue(!cachedErrors.contains("test-topic-2"), "test-topic-2 should have been evicted")
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
package kafka.server
|
||||
|
||||
import kafka.network.RequestChannel
|
||||
import kafka.raft.RaftManager
|
||||
import kafka.server.QuotaFactory.QuotaManagers
|
||||
import kafka.server.metadata.KRaftMetadataCache
|
||||
import org.apache.kafka.clients.admin.AlterConfigOp
|
||||
|
@ -56,7 +55,7 @@ import org.apache.kafka.image.publisher.ControllerRegistrationsPublisher
|
|||
import org.apache.kafka.network.SocketServerConfigs
|
||||
import org.apache.kafka.network.metrics.RequestChannelMetrics
|
||||
import org.apache.kafka.network.Session
|
||||
import org.apache.kafka.raft.QuorumConfig
|
||||
import org.apache.kafka.raft.{QuorumConfig, RaftManager}
|
||||
import org.apache.kafka.server.SimpleApiVersionManager
|
||||
import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult, Authorizer}
|
||||
import org.apache.kafka.server.common.{ApiMessageAndVersion, FinalizedFeatures, KRaftVersion, MetadataVersion, ProducerIdsBlock, RequestLocal}
|
||||
|
|
|
@ -0,0 +1,400 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package kafka.server
|
||||
|
||||
import org.apache.kafka.server.util.MockTime
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.api.{BeforeEach, Test}
|
||||
import scala.concurrent.Future
|
||||
import scala.concurrent.ExecutionContext.Implicits.global
|
||||
import scala.util.Random
|
||||
import java.util.concurrent.{CountDownLatch, TimeUnit}
|
||||
|
||||
class ExpiringErrorCacheTest {
|
||||
|
||||
private var mockTime: MockTime = _
|
||||
private var cache: ExpiringErrorCache = _
|
||||
|
||||
@BeforeEach
|
||||
def setUp(): Unit = {
|
||||
mockTime = new MockTime()
|
||||
}
|
||||
|
||||
// Basic Functionality Tests
|
||||
|
||||
@Test
|
||||
def testPutAndGet(): Unit = {
|
||||
cache = new ExpiringErrorCache(10, mockTime)
|
||||
|
||||
cache.put("topic1", "error1", 1000L)
|
||||
cache.put("topic2", "error2", 2000L)
|
||||
|
||||
val errors = cache.getErrorsForTopics(Set("topic1", "topic2"), mockTime.milliseconds())
|
||||
assertEquals(2, errors.size)
|
||||
assertEquals("error1", errors("topic1"))
|
||||
assertEquals("error2", errors("topic2"))
|
||||
}
|
||||
|
||||
@Test
|
||||
def testGetNonExistentTopic(): Unit = {
|
||||
cache = new ExpiringErrorCache(10, mockTime)
|
||||
|
||||
cache.put("topic1", "error1", 1000L)
|
||||
|
||||
val errors = cache.getErrorsForTopics(Set("topic1", "topic2"), mockTime.milliseconds())
|
||||
assertEquals(1, errors.size)
|
||||
assertEquals("error1", errors("topic1"))
|
||||
assertFalse(errors.contains("topic2"))
|
||||
}
|
||||
|
||||
@Test
|
||||
def testUpdateExistingEntry(): Unit = {
|
||||
cache = new ExpiringErrorCache(10, mockTime)
|
||||
|
||||
cache.put("topic1", "error1", 1000L)
|
||||
assertEquals("error1", cache.getErrorsForTopics(Set("topic1"), mockTime.milliseconds())("topic1"))
|
||||
|
||||
// Update with new error
|
||||
cache.put("topic1", "error2", 2000L)
|
||||
assertEquals("error2", cache.getErrorsForTopics(Set("topic1"), mockTime.milliseconds())("topic1"))
|
||||
}
|
||||
|
||||
@Test
|
||||
def testGetMultipleTopics(): Unit = {
|
||||
cache = new ExpiringErrorCache(10, mockTime)
|
||||
|
||||
cache.put("topic1", "error1", 1000L)
|
||||
cache.put("topic2", "error2", 1000L)
|
||||
cache.put("topic3", "error3", 1000L)
|
||||
|
||||
val errors = cache.getErrorsForTopics(Set("topic1", "topic3", "topic4"), mockTime.milliseconds())
|
||||
assertEquals(2, errors.size)
|
||||
assertEquals("error1", errors("topic1"))
|
||||
assertEquals("error3", errors("topic3"))
|
||||
assertFalse(errors.contains("topic2"))
|
||||
assertFalse(errors.contains("topic4"))
|
||||
}
|
||||
|
||||
// Expiration Tests
|
||||
|
||||
@Test
|
||||
def testExpiredEntryNotReturned(): Unit = {
|
||||
cache = new ExpiringErrorCache(10, mockTime)
|
||||
|
||||
cache.put("topic1", "error1", 1000L)
|
||||
|
||||
// Entry should be available before expiration
|
||||
assertEquals(1, cache.getErrorsForTopics(Set("topic1"), mockTime.milliseconds()).size)
|
||||
|
||||
// Advance time past expiration
|
||||
mockTime.sleep(1001L)
|
||||
|
||||
// Entry should not be returned after expiration
|
||||
assertTrue(cache.getErrorsForTopics(Set("topic1"), mockTime.milliseconds()).isEmpty)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testExpiredEntriesCleanedOnPut(): Unit = {
|
||||
cache = new ExpiringErrorCache(10, mockTime)
|
||||
|
||||
// Add entries with different TTLs
|
||||
cache.put("topic1", "error1", 1000L)
|
||||
cache.put("topic2", "error2", 2000L)
|
||||
|
||||
// Advance time to expire topic1 but not topic2
|
||||
mockTime.sleep(1500L)
|
||||
|
||||
// Add a new entry - this should trigger cleanup
|
||||
cache.put("topic3", "error3", 1000L)
|
||||
|
||||
// Verify only non-expired entries remain
|
||||
val errors = cache.getErrorsForTopics(Set("topic1", "topic2", "topic3"), mockTime.milliseconds())
|
||||
assertEquals(2, errors.size)
|
||||
assertFalse(errors.contains("topic1"))
|
||||
assertEquals("error2", errors("topic2"))
|
||||
assertEquals("error3", errors("topic3"))
|
||||
}
|
||||
|
||||
@Test
|
||||
def testMixedExpiredAndValidEntries(): Unit = {
|
||||
cache = new ExpiringErrorCache(10, mockTime)
|
||||
|
||||
cache.put("topic1", "error1", 500L)
|
||||
cache.put("topic2", "error2", 1000L)
|
||||
cache.put("topic3", "error3", 1500L)
|
||||
|
||||
// Advance time to expire only topic1
|
||||
mockTime.sleep(600L)
|
||||
|
||||
val errors = cache.getErrorsForTopics(Set("topic1", "topic2", "topic3"), mockTime.milliseconds())
|
||||
assertEquals(2, errors.size)
|
||||
assertFalse(errors.contains("topic1"))
|
||||
assertTrue(errors.contains("topic2"))
|
||||
assertTrue(errors.contains("topic3"))
|
||||
}
|
||||
|
||||
// Capacity Enforcement Tests
|
||||
|
||||
@Test
|
||||
def testCapacityEnforcement(): Unit = {
|
||||
cache = new ExpiringErrorCache(3, mockTime)
|
||||
|
||||
// Add 5 entries, exceeding capacity of 3
|
||||
for (i <- 1 to 5) {
|
||||
cache.put(s"topic$i", s"error$i", 1000L)
|
||||
// Small time advance between entries to ensure different insertion order
|
||||
mockTime.sleep(10L)
|
||||
}
|
||||
|
||||
val errors = cache.getErrorsForTopics((1 to 5).map(i => s"topic$i").toSet, mockTime.milliseconds())
|
||||
assertEquals(3, errors.size)
|
||||
|
||||
// The cache evicts by earliest expiration time
|
||||
// Since all have same TTL, earliest inserted (topic1, topic2) should be evicted
|
||||
assertFalse(errors.contains("topic1"))
|
||||
assertFalse(errors.contains("topic2"))
|
||||
assertTrue(errors.contains("topic3"))
|
||||
assertTrue(errors.contains("topic4"))
|
||||
assertTrue(errors.contains("topic5"))
|
||||
}
|
||||
|
||||
@Test
|
||||
def testEvictionOrder(): Unit = {
|
||||
cache = new ExpiringErrorCache(3, mockTime)
|
||||
|
||||
// Add entries with different TTLs
|
||||
cache.put("topic1", "error1", 3000L) // Expires at 3000
|
||||
mockTime.sleep(100L)
|
||||
cache.put("topic2", "error2", 1000L) // Expires at 1100
|
||||
mockTime.sleep(100L)
|
||||
cache.put("topic3", "error3", 2000L) // Expires at 2200
|
||||
mockTime.sleep(100L)
|
||||
cache.put("topic4", "error4", 500L) // Expires at 800
|
||||
|
||||
// With capacity 3, topic4 (earliest expiration) should be evicted
|
||||
val errors = cache.getErrorsForTopics(Set("topic1", "topic2", "topic3", "topic4"), mockTime.milliseconds())
|
||||
assertEquals(3, errors.size)
|
||||
assertTrue(errors.contains("topic1"))
|
||||
assertTrue(errors.contains("topic2"))
|
||||
assertTrue(errors.contains("topic3"))
|
||||
assertFalse(errors.contains("topic4"))
|
||||
}
|
||||
|
||||
@Test
|
||||
def testCapacityWithDifferentTTLs(): Unit = {
|
||||
cache = new ExpiringErrorCache(2, mockTime)
|
||||
|
||||
cache.put("topic1", "error1", 5000L) // Long TTL
|
||||
cache.put("topic2", "error2", 100L) // Short TTL
|
||||
cache.put("topic3", "error3", 3000L) // Medium TTL
|
||||
|
||||
// topic2 has earliest expiration, so it should be evicted
|
||||
val errors = cache.getErrorsForTopics(Set("topic1", "topic2", "topic3"), mockTime.milliseconds())
|
||||
assertEquals(2, errors.size)
|
||||
assertTrue(errors.contains("topic1"))
|
||||
assertFalse(errors.contains("topic2"))
|
||||
assertTrue(errors.contains("topic3"))
|
||||
}
|
||||
|
||||
// Update and Stale Entry Tests
|
||||
|
||||
@Test
|
||||
def testUpdateDoesNotLeaveStaleEntries(): Unit = {
|
||||
cache = new ExpiringErrorCache(3, mockTime)
|
||||
|
||||
// Fill cache to capacity
|
||||
cache.put("topic1", "error1", 1000L)
|
||||
cache.put("topic2", "error2", 1000L)
|
||||
cache.put("topic3", "error3", 1000L)
|
||||
|
||||
// Update topic2 with longer TTL
|
||||
cache.put("topic2", "error2_updated", 5000L)
|
||||
|
||||
// Add new entry to trigger eviction
|
||||
cache.put("topic4", "error4", 1000L)
|
||||
|
||||
// Should evict topic1 or topic3 (earliest expiration), not the updated topic2
|
||||
val errors = cache.getErrorsForTopics(Set("topic1", "topic2", "topic3", "topic4"), mockTime.milliseconds())
|
||||
assertEquals(3, errors.size)
|
||||
assertTrue(errors.contains("topic2"))
|
||||
assertEquals("error2_updated", errors("topic2"))
|
||||
}
|
||||
|
||||
@Test
|
||||
def testStaleEntriesInQueueHandledCorrectly(): Unit = {
|
||||
cache = new ExpiringErrorCache(10, mockTime)
|
||||
|
||||
// Add and update same topic multiple times
|
||||
cache.put("topic1", "error1", 1000L)
|
||||
cache.put("topic1", "error2", 2000L)
|
||||
cache.put("topic1", "error3", 3000L)
|
||||
|
||||
// Only latest value should be returned
|
||||
val errors = cache.getErrorsForTopics(Set("topic1"), mockTime.milliseconds())
|
||||
assertEquals(1, errors.size)
|
||||
assertEquals("error3", errors("topic1"))
|
||||
|
||||
// Advance time to expire first two entries
|
||||
mockTime.sleep(2500L)
|
||||
|
||||
// Force cleanup by adding new entry
|
||||
cache.put("topic2", "error_new", 1000L)
|
||||
|
||||
// topic1 should still be available with latest value
|
||||
val errorsAfterCleanup = cache.getErrorsForTopics(Set("topic1"), mockTime.milliseconds())
|
||||
assertEquals(1, errorsAfterCleanup.size)
|
||||
assertEquals("error3", errorsAfterCleanup("topic1"))
|
||||
}
|
||||
|
||||
// Edge Cases
|
||||
|
||||
@Test
|
||||
def testEmptyCache(): Unit = {
|
||||
cache = new ExpiringErrorCache(10, mockTime)
|
||||
|
||||
val errors = cache.getErrorsForTopics(Set("topic1", "topic2"), mockTime.milliseconds())
|
||||
assertTrue(errors.isEmpty)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testSingleEntryCache(): Unit = {
|
||||
cache = new ExpiringErrorCache(1, mockTime)
|
||||
|
||||
cache.put("topic1", "error1", 1000L)
|
||||
cache.put("topic2", "error2", 1000L)
|
||||
|
||||
// Only most recent should remain
|
||||
val errors = cache.getErrorsForTopics(Set("topic1", "topic2"), mockTime.milliseconds())
|
||||
assertEquals(1, errors.size)
|
||||
assertFalse(errors.contains("topic1"))
|
||||
assertTrue(errors.contains("topic2"))
|
||||
}
|
||||
|
||||
@Test
|
||||
def testZeroTTL(): Unit = {
|
||||
cache = new ExpiringErrorCache(10, mockTime)
|
||||
|
||||
cache.put("topic1", "error1", 0L)
|
||||
|
||||
// Entry expires immediately
|
||||
assertTrue(cache.getErrorsForTopics(Set("topic1"), mockTime.milliseconds()).isEmpty)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testClearOperation(): Unit = {
|
||||
cache = new ExpiringErrorCache(10, mockTime)
|
||||
|
||||
cache.put("topic1", "error1", 1000L)
|
||||
cache.put("topic2", "error2", 1000L)
|
||||
|
||||
assertEquals(2, cache.getErrorsForTopics(Set("topic1", "topic2"), mockTime.milliseconds()).size)
|
||||
|
||||
cache.clear()
|
||||
|
||||
assertTrue(cache.getErrorsForTopics(Set("topic1", "topic2"), mockTime.milliseconds()).isEmpty)
|
||||
}
|
||||
|
||||
// Concurrent Access Tests
|
||||
|
||||
@Test
|
||||
def testConcurrentPutOperations(): Unit = {
|
||||
cache = new ExpiringErrorCache(100, mockTime)
|
||||
val numThreads = 10
|
||||
val numTopicsPerThread = 20
|
||||
val latch = new CountDownLatch(numThreads)
|
||||
|
||||
(1 to numThreads).foreach { threadId =>
|
||||
Future {
|
||||
try {
|
||||
for (i <- 1 to numTopicsPerThread) {
|
||||
cache.put(s"topic_${threadId}_$i", s"error_${threadId}_$i", 1000L)
|
||||
}
|
||||
} finally {
|
||||
latch.countDown()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
assertTrue(latch.await(5, TimeUnit.SECONDS))
|
||||
|
||||
// Verify all entries were added
|
||||
val allTopics = (1 to numThreads).flatMap { threadId =>
|
||||
(1 to numTopicsPerThread).map(i => s"topic_${threadId}_$i")
|
||||
}.toSet
|
||||
|
||||
val errors = cache.getErrorsForTopics(allTopics, mockTime.milliseconds())
|
||||
assertEquals(100, errors.size) // Limited by cache capacity
|
||||
}
|
||||
|
||||
@Test
|
||||
def testConcurrentPutAndGet(): Unit = {
|
||||
cache = new ExpiringErrorCache(100, mockTime)
|
||||
val numOperations = 1000
|
||||
val random = new Random()
|
||||
val topics = (1 to 50).map(i => s"topic$i").toArray
|
||||
|
||||
val futures = (1 to numOperations).map { _ =>
|
||||
Future {
|
||||
if (random.nextBoolean()) {
|
||||
// Put operation
|
||||
val topic = topics(random.nextInt(topics.length))
|
||||
cache.put(topic, s"error_${random.nextInt()}", 1000L)
|
||||
} else {
|
||||
// Get operation
|
||||
val topicsToGet = Set(topics(random.nextInt(topics.length)))
|
||||
cache.getErrorsForTopics(topicsToGet, mockTime.milliseconds())
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Wait for all operations to complete
|
||||
Future.sequence(futures).map(_ => ())
|
||||
}
|
||||
|
||||
@Test
|
||||
def testConcurrentUpdates(): Unit = {
|
||||
cache = new ExpiringErrorCache(50, mockTime)
|
||||
val numThreads = 10
|
||||
val numUpdatesPerThread = 100
|
||||
val sharedTopics = (1 to 10).map(i => s"shared_topic$i").toArray
|
||||
val latch = new CountDownLatch(numThreads)
|
||||
|
||||
(1 to numThreads).foreach { threadId =>
|
||||
Future {
|
||||
try {
|
||||
val random = new Random()
|
||||
for (i <- 1 to numUpdatesPerThread) {
|
||||
val topic = sharedTopics(random.nextInt(sharedTopics.length))
|
||||
cache.put(topic, s"error_thread${threadId}_update$i", 1000L)
|
||||
}
|
||||
} finally {
|
||||
latch.countDown()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
assertTrue(latch.await(5, TimeUnit.SECONDS))
|
||||
|
||||
// Verify all shared topics have some value
|
||||
val errors = cache.getErrorsForTopics(sharedTopics.toSet, mockTime.milliseconds())
|
||||
sharedTopics.foreach { topic =>
|
||||
assertTrue(errors.contains(topic), s"Topic $topic should have a value")
|
||||
assertTrue(errors(topic).startsWith("error_thread"), s"Value should be from one of the threads")
|
||||
}
|
||||
}
|
||||
}
|
|
@ -168,7 +168,8 @@ class KafkaApisTest extends Logging {
|
|||
authorizer: Option[Authorizer] = None,
|
||||
configRepository: ConfigRepository = new MockConfigRepository(),
|
||||
overrideProperties: Map[String, String] = Map.empty,
|
||||
featureVersions: Seq[FeatureVersion] = Seq.empty
|
||||
featureVersions: Seq[FeatureVersion] = Seq.empty,
|
||||
autoTopicCreationManager: Option[AutoTopicCreationManager] = None
|
||||
): KafkaApis = {
|
||||
|
||||
val properties = TestUtils.createBrokerConfig(brokerId)
|
||||
|
@ -194,7 +195,7 @@ class KafkaApisTest extends Logging {
|
|||
groupCoordinator = groupCoordinator,
|
||||
txnCoordinator = txnCoordinator,
|
||||
shareCoordinator = shareCoordinator,
|
||||
autoTopicCreationManager = autoTopicCreationManager,
|
||||
autoTopicCreationManager = autoTopicCreationManager.getOrElse(this.autoTopicCreationManager),
|
||||
brokerId = brokerId,
|
||||
config = config,
|
||||
configRepository = configRepository,
|
||||
|
@ -10887,7 +10888,7 @@ class KafkaApisTest extends Logging {
|
|||
future.complete(new StreamsGroupHeartbeatResult(streamsGroupHeartbeatResponse, missingTopics.asJava))
|
||||
val response = verifyNoThrottling[StreamsGroupHeartbeatResponse](requestChannelRequest)
|
||||
assertEquals(streamsGroupHeartbeatResponse, response.data)
|
||||
verify(autoTopicCreationManager).createStreamsInternalTopics(missingTopics, requestChannelRequest.context)
|
||||
verify(autoTopicCreationManager).createStreamsInternalTopics(any(), any(), anyLong())
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -10947,6 +10948,61 @@ class KafkaApisTest extends Logging {
|
|||
)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testStreamsGroupHeartbeatRequestWithCachedTopicCreationErrors(): Unit = {
|
||||
val features = mock(classOf[FinalizedFeatures])
|
||||
when(features.finalizedFeatures()).thenReturn(util.Map.of(StreamsVersion.FEATURE_NAME, 1.toShort))
|
||||
|
||||
metadataCache = mock(classOf[KRaftMetadataCache])
|
||||
when(metadataCache.features()).thenReturn(features)
|
||||
|
||||
val streamsGroupHeartbeatRequest = new StreamsGroupHeartbeatRequestData().setGroupId("group")
|
||||
val requestChannelRequest = buildRequest(new StreamsGroupHeartbeatRequest.Builder(streamsGroupHeartbeatRequest, true).build())
|
||||
|
||||
val future = new CompletableFuture[StreamsGroupHeartbeatResult]()
|
||||
when(groupCoordinator.streamsGroupHeartbeat(
|
||||
requestChannelRequest.context,
|
||||
streamsGroupHeartbeatRequest
|
||||
)).thenReturn(future)
|
||||
|
||||
// Mock AutoTopicCreationManager to return cached errors
|
||||
val mockAutoTopicCreationManager = mock(classOf[AutoTopicCreationManager])
|
||||
when(mockAutoTopicCreationManager.getStreamsInternalTopicCreationErrors(ArgumentMatchers.eq(Set("test-topic")), any()))
|
||||
.thenReturn(Map("test-topic" -> "INVALID_REPLICATION_FACTOR"))
|
||||
// Mock the createStreamsInternalTopics method to do nothing (simulate topic creation attempt)
|
||||
doNothing().when(mockAutoTopicCreationManager).createStreamsInternalTopics(any(), any(), anyLong())
|
||||
|
||||
kafkaApis = createKafkaApis(autoTopicCreationManager = Some(mockAutoTopicCreationManager))
|
||||
kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching)
|
||||
|
||||
// Group coordinator returns MISSING_INTERNAL_TOPICS status and topics to create
|
||||
val missingTopics = util.Map.of("test-topic", new CreatableTopic())
|
||||
val streamsGroupHeartbeatResponse = new StreamsGroupHeartbeatResponseData()
|
||||
.setMemberId("member")
|
||||
.setStatus(util.List.of(
|
||||
new StreamsGroupHeartbeatResponseData.Status()
|
||||
.setStatusCode(StreamsGroupHeartbeatResponse.Status.MISSING_INTERNAL_TOPICS.code())
|
||||
.setStatusDetail("Internal topics are missing: [test-topic]")
|
||||
))
|
||||
|
||||
future.complete(new StreamsGroupHeartbeatResult(streamsGroupHeartbeatResponse, missingTopics))
|
||||
val response = verifyNoThrottling[StreamsGroupHeartbeatResponse](requestChannelRequest)
|
||||
|
||||
assertEquals(Errors.NONE.code, response.data.errorCode())
|
||||
assertEquals(null, response.data.errorMessage())
|
||||
|
||||
// Verify that the cached error was appended to the existing status detail
|
||||
assertEquals(1, response.data.status().size())
|
||||
val status = response.data.status().get(0)
|
||||
assertEquals(StreamsGroupHeartbeatResponse.Status.MISSING_INTERNAL_TOPICS.code(), status.statusCode())
|
||||
assertTrue(status.statusDetail().contains("Internal topics are missing: [test-topic]"))
|
||||
assertTrue(status.statusDetail().contains("Creation failed: test-topic (INVALID_REPLICATION_FACTOR)"))
|
||||
|
||||
// Verify that createStreamsInternalTopics was called
|
||||
verify(mockAutoTopicCreationManager).createStreamsInternalTopics(any(), any(), anyLong())
|
||||
verify(mockAutoTopicCreationManager).getStreamsInternalTopicCreationErrors(ArgumentMatchers.eq(Set("test-topic")), any())
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = Array(true, false))
|
||||
def testConsumerGroupDescribe(includeAuthorizedOperations: Boolean): Unit = {
|
||||
|
|
|
@ -293,7 +293,7 @@ class LeaderEpochIntegrationTest extends QuorumTestHarness with Logging {
|
|||
}
|
||||
|
||||
private def waitUntilQuorumLeaderElected(controllerServer: ControllerServer, timeout: Long = JTestUtils.DEFAULT_MAX_WAIT_MS): Int = {
|
||||
val (leaderAndEpoch, _) = computeUntilTrue(controllerServer.raftManager.leaderAndEpoch, waitTime = timeout)(_.leaderId().isPresent)
|
||||
val (leaderAndEpoch, _) = computeUntilTrue(controllerServer.raftManager.client.leaderAndEpoch, waitTime = timeout)(_.leaderId().isPresent)
|
||||
leaderAndEpoch.leaderId().orElseThrow(() => new AssertionError(s"Quorum Controller leader not elected after $timeout ms"))
|
||||
}
|
||||
|
||||
|
|
|
@ -147,7 +147,7 @@ Single Node
|
|||
- To produce messages using client scripts (Ensure that java version >= 17):
|
||||
```
|
||||
# Run from root of the repo
|
||||
$ bin/kafka-console-producer.sh --topic test --bootstrap-server localhost:9093 --producer.config ./docker/examples/fixtures/client-secrets/client-ssl.properties
|
||||
$ bin/kafka-console-producer.sh --topic test --bootstrap-server localhost:9093 --command-config ./docker/examples/fixtures/client-secrets/client-ssl.properties
|
||||
```
|
||||
- File Input:
|
||||
- Here ssl configs are provided via file input.
|
||||
|
@ -167,7 +167,7 @@ Single Node
|
|||
- To produce messages using client scripts (Ensure that java version >= 17):
|
||||
```
|
||||
# Run from root of the repo
|
||||
$ bin/kafka-console-producer.sh --topic test --bootstrap-server localhost:9093 --producer.config ./docker/examples/fixtures/client-secrets/client-ssl.properties
|
||||
$ bin/kafka-console-producer.sh --topic test --bootstrap-server localhost:9093 --command-config ./docker/examples/fixtures/client-secrets/client-ssl.properties
|
||||
```
|
||||
|
||||
Multi Node Cluster
|
||||
|
@ -219,7 +219,7 @@ Multi Node Cluster
|
|||
- To produce messages using client scripts (Ensure that java version >= 17):
|
||||
```
|
||||
# Run from root of the repo
|
||||
$ bin/kafka-console-producer.sh --topic test --bootstrap-server localhost:29093 --producer.config ./docker/examples/fixtures/client-secrets/client-ssl.properties
|
||||
$ bin/kafka-console-producer.sh --topic test --bootstrap-server localhost:29093 --command-config ./docker/examples/fixtures/client-secrets/client-ssl.properties
|
||||
```
|
||||
- Isolated:
|
||||
- Examples are present in `docker-compose-files/cluster/isolated` directory.
|
||||
|
@ -258,7 +258,7 @@ Multi Node Cluster
|
|||
- To produce messages using client scripts (Ensure that java version >= 17):
|
||||
```
|
||||
# Run from root of the repo
|
||||
$ bin/kafka-console-producer.sh --topic test --bootstrap-server localhost:29093 --producer.config ./docker/examples/fixtures/client-secrets/client-ssl.properties
|
||||
$ bin/kafka-console-producer.sh --topic test --bootstrap-server localhost:29093 --command-config ./docker/examples/fixtures/client-secrets/client-ssl.properties
|
||||
```
|
||||
|
||||
- Note that the examples are meant to be tried one at a time, make sure you close an example server before trying out the other to avoid conflicts.
|
||||
|
|
|
@ -65,7 +65,7 @@ class DockerSanityTest(unittest.TestCase):
|
|||
subprocess.run(["bash", "-c", " ".join(command)])
|
||||
|
||||
def consume_message(self, topic, consumer_config):
|
||||
command = [f"{self.FIXTURES_DIR}/{constants.KAFKA_CONSOLE_CONSUMER}", "--topic", topic, "--property", "'print.key=true'", "--property", "'key.separator=:'", "--from-beginning", "--max-messages", "1", "--timeout-ms", f"{constants.CLIENT_TIMEOUT}"]
|
||||
command = [f"{self.FIXTURES_DIR}/{constants.KAFKA_CONSOLE_CONSUMER}", "--topic", topic, "--formatter-property", "'print.key=true'", "--formatter-property", "'key.separator=:'", "--from-beginning", "--max-messages", "1", "--timeout-ms", f"{constants.CLIENT_TIMEOUT}"]
|
||||
command.extend(consumer_config)
|
||||
message = subprocess.check_output(["bash", "-c", " ".join(command)])
|
||||
return message.decode("utf-8").strip()
|
||||
|
@ -93,9 +93,9 @@ class DockerSanityTest(unittest.TestCase):
|
|||
errors.append(constants.BROKER_METRICS_ERROR_PREFIX + str(e))
|
||||
return errors
|
||||
|
||||
producer_config = ["--bootstrap-server", "localhost:9092", "--property", "client.id=host"]
|
||||
producer_config = ["--bootstrap-server", "localhost:9092", "--command-property", "client.id=host"]
|
||||
self.produce_message(constants.BROKER_METRICS_TEST_TOPIC, producer_config, "key", "message")
|
||||
consumer_config = ["--bootstrap-server", "localhost:9092", "--property", "auto.offset.reset=earliest"]
|
||||
consumer_config = ["--bootstrap-server", "localhost:9092", "--command-property", "auto.offset.reset=earliest"]
|
||||
message = self.consume_message(constants.BROKER_METRICS_TEST_TOPIC, consumer_config)
|
||||
try:
|
||||
self.assertEqual(message, "key:message")
|
||||
|
@ -129,13 +129,13 @@ class DockerSanityTest(unittest.TestCase):
|
|||
return errors
|
||||
|
||||
producer_config = ["--bootstrap-server", ssl_broker_port,
|
||||
"--producer.config", f"{self.FIXTURES_DIR}/{constants.SSL_CLIENT_CONFIG}"]
|
||||
"--command-config", f"{self.FIXTURES_DIR}/{constants.SSL_CLIENT_CONFIG}"]
|
||||
self.produce_message(topic, producer_config, "key", "message")
|
||||
|
||||
consumer_config = [
|
||||
"--bootstrap-server", ssl_broker_port,
|
||||
"--property", "auto.offset.reset=earliest",
|
||||
"--consumer.config", f"{self.FIXTURES_DIR}/{constants.SSL_CLIENT_CONFIG}",
|
||||
"--command-property", "auto.offset.reset=earliest",
|
||||
"--command-config", f"{self.FIXTURES_DIR}/{constants.SSL_CLIENT_CONFIG}",
|
||||
]
|
||||
message = self.consume_message(topic, consumer_config)
|
||||
try:
|
||||
|
@ -155,7 +155,7 @@ class DockerSanityTest(unittest.TestCase):
|
|||
errors.append(constants.BROKER_RESTART_ERROR_PREFIX + str(e))
|
||||
return errors
|
||||
|
||||
producer_config = ["--bootstrap-server", "localhost:9092", "--property", "client.id=host"]
|
||||
producer_config = ["--bootstrap-server", "localhost:9092", "--command-property", "client.id=host"]
|
||||
self.produce_message(constants.BROKER_RESTART_TEST_TOPIC, producer_config, "key", "message")
|
||||
|
||||
print("Stopping Container")
|
||||
|
@ -163,7 +163,7 @@ class DockerSanityTest(unittest.TestCase):
|
|||
print("Resuming Container")
|
||||
self.resume_container()
|
||||
|
||||
consumer_config = ["--bootstrap-server", "localhost:9092", "--property", "auto.offset.reset=earliest"]
|
||||
consumer_config = ["--bootstrap-server", "localhost:9092", "--command-property", "auto.offset.reset=earliest"]
|
||||
message = self.consume_message(constants.BROKER_RESTART_TEST_TOPIC, consumer_config)
|
||||
try:
|
||||
self.assertEqual(message, "key:message")
|
||||
|
|
|
@ -510,8 +510,8 @@ ssl.key.password=test1234</code></pre>
|
|||
</ol>
|
||||
<br>
|
||||
Examples using console-producer and console-consumer:
|
||||
<pre><code class="language-bash">$ bin/kafka-console-producer.sh --bootstrap-server localhost:9093 --topic test --producer.config client-ssl.properties
|
||||
$ bin/kafka-console-consumer.sh --bootstrap-server localhost:9093 --topic test --consumer.config client-ssl.properties</code></pre>
|
||||
<pre><code class="language-bash">$ bin/kafka-console-producer.sh --bootstrap-server localhost:9093 --topic test --command-config client-ssl.properties
|
||||
$ bin/kafka-console-consumer.sh --bootstrap-server localhost:9093 --topic test --command-config client-ssl.properties</code></pre>
|
||||
</li>
|
||||
</ol>
|
||||
<h3 class="anchor-heading"><a id="security_sasl" class="anchor-link"></a><a href="#security_sasl">7.4 Authentication using SASL</a></h3>
|
||||
|
|
|
@ -142,6 +142,11 @@
|
|||
</li>
|
||||
</ul>
|
||||
</li>
|
||||
<li>
|
||||
The <code>controller.quorum.auto.join.enable</code> has been added to <code>QuorumConfig</code>, enabling KRaft controllers to automatically join the cluster's voter set,
|
||||
and defaults to false.
|
||||
For further details, please refer to <a href="https://cwiki.apache.org/confluence/x/nyH1D">KIP-853</a>.
|
||||
</li>
|
||||
</ul>
|
||||
|
||||
<h4><a id="upgrade_4_1_0" href="#upgrade_4_1_0">Upgrading to 4.1.0</a></h4>
|
||||
|
|
|
@ -124,11 +124,11 @@ public class CoordinatorRecordTypeGenerator implements TypeClassGenerator {
|
|||
for (Map.Entry<Short, CoordinatorRecord> entry : records.entrySet()) {
|
||||
MessageSpec key = entry.getValue().key;
|
||||
if (key == null) {
|
||||
throw new RuntimeException("Coordinator record " + entry.getKey() + " has not key.");
|
||||
throw new RuntimeException("Coordinator record " + entry.getKey() + " has no key.");
|
||||
}
|
||||
MessageSpec value = entry.getValue().value;
|
||||
if (value == null) {
|
||||
throw new RuntimeException("Coordinator record " + entry.getKey() + " has not key.");
|
||||
throw new RuntimeException("Coordinator record " + entry.getKey() + " has no value.");
|
||||
}
|
||||
String name = cleanName(key.name());
|
||||
numProcessed++;
|
||||
|
|
|
@ -813,10 +813,10 @@ public class GroupMetadataManager {
|
|||
}
|
||||
|
||||
if (group == null) {
|
||||
return new ConsumerGroup(snapshotRegistry, groupId, metrics);
|
||||
return new ConsumerGroup(snapshotRegistry, groupId);
|
||||
} else if (createIfNotExists && maybeDeleteEmptyClassicGroup(group, records)) {
|
||||
log.info("[GroupId {}] Converted the empty classic group to a consumer group.", groupId);
|
||||
return new ConsumerGroup(snapshotRegistry, groupId, metrics);
|
||||
return new ConsumerGroup(snapshotRegistry, groupId);
|
||||
} else {
|
||||
if (group.type() == CONSUMER) {
|
||||
return (ConsumerGroup) group;
|
||||
|
@ -851,10 +851,10 @@ public class GroupMetadataManager {
|
|||
Group group = groups.get(groupId);
|
||||
|
||||
if (group == null) {
|
||||
return new StreamsGroup(logContext, snapshotRegistry, groupId, metrics);
|
||||
return new StreamsGroup(logContext, snapshotRegistry, groupId);
|
||||
} else if (maybeDeleteEmptyClassicGroup(group, records)) {
|
||||
log.info("[GroupId {}] Converted the empty classic group to a streams group.", groupId);
|
||||
return new StreamsGroup(logContext, snapshotRegistry, groupId, metrics);
|
||||
return new StreamsGroup(logContext, snapshotRegistry, groupId);
|
||||
} else {
|
||||
return castToStreamsGroup(group);
|
||||
}
|
||||
|
@ -980,7 +980,7 @@ public class GroupMetadataManager {
|
|||
}
|
||||
|
||||
if (group == null) {
|
||||
ConsumerGroup consumerGroup = new ConsumerGroup(snapshotRegistry, groupId, metrics);
|
||||
ConsumerGroup consumerGroup = new ConsumerGroup(snapshotRegistry, groupId);
|
||||
groups.put(groupId, consumerGroup);
|
||||
return consumerGroup;
|
||||
} else if (group.type() == CONSUMER) {
|
||||
|
@ -990,7 +990,7 @@ public class GroupMetadataManager {
|
|||
// offsets if no group existed. Simple classic groups are not backed by any records
|
||||
// in the __consumer_offsets topic hence we can safely replace it here. Without this,
|
||||
// replaying consumer group records after offset commit records would not work.
|
||||
ConsumerGroup consumerGroup = new ConsumerGroup(snapshotRegistry, groupId, metrics);
|
||||
ConsumerGroup consumerGroup = new ConsumerGroup(snapshotRegistry, groupId);
|
||||
groups.put(groupId, consumerGroup);
|
||||
return consumerGroup;
|
||||
} else {
|
||||
|
@ -1023,7 +1023,7 @@ public class GroupMetadataManager {
|
|||
}
|
||||
|
||||
if (group == null) {
|
||||
StreamsGroup streamsGroup = new StreamsGroup(logContext, snapshotRegistry, groupId, metrics);
|
||||
StreamsGroup streamsGroup = new StreamsGroup(logContext, snapshotRegistry, groupId);
|
||||
groups.put(groupId, streamsGroup);
|
||||
return streamsGroup;
|
||||
} else if (group.type() == STREAMS) {
|
||||
|
@ -1364,7 +1364,6 @@ public class GroupMetadataManager {
|
|||
try {
|
||||
consumerGroup = ConsumerGroup.fromClassicGroup(
|
||||
snapshotRegistry,
|
||||
metrics,
|
||||
classicGroup,
|
||||
topicHashCache,
|
||||
metadataImage
|
||||
|
|
|
@ -38,7 +38,6 @@ import org.apache.kafka.coordinator.group.Utils;
|
|||
import org.apache.kafka.coordinator.group.api.assignor.SubscriptionType;
|
||||
import org.apache.kafka.coordinator.group.classic.ClassicGroup;
|
||||
import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataValue;
|
||||
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard;
|
||||
import org.apache.kafka.coordinator.group.modern.Assignment;
|
||||
import org.apache.kafka.coordinator.group.modern.MemberState;
|
||||
import org.apache.kafka.coordinator.group.modern.ModernGroup;
|
||||
|
@ -119,11 +118,6 @@ public class ConsumerGroup extends ModernGroup<ConsumerGroupMember> {
|
|||
*/
|
||||
private final TimelineHashMap<String, Integer> serverAssignors;
|
||||
|
||||
/**
|
||||
* The coordinator metrics.
|
||||
*/
|
||||
private final GroupCoordinatorMetricsShard metrics;
|
||||
|
||||
/**
|
||||
* The number of members that use the classic protocol.
|
||||
*/
|
||||
|
@ -155,14 +149,12 @@ public class ConsumerGroup extends ModernGroup<ConsumerGroupMember> {
|
|||
|
||||
public ConsumerGroup(
|
||||
SnapshotRegistry snapshotRegistry,
|
||||
String groupId,
|
||||
GroupCoordinatorMetricsShard metrics
|
||||
String groupId
|
||||
) {
|
||||
super(snapshotRegistry, groupId);
|
||||
this.state = new TimelineObject<>(snapshotRegistry, EMPTY);
|
||||
this.staticMembers = new TimelineHashMap<>(snapshotRegistry, 0);
|
||||
this.serverAssignors = new TimelineHashMap<>(snapshotRegistry, 0);
|
||||
this.metrics = Objects.requireNonNull(metrics);
|
||||
this.numClassicProtocolMembers = new TimelineInteger(snapshotRegistry);
|
||||
this.classicProtocolMembersSupportedProtocols = new TimelineHashMap<>(snapshotRegistry, 0);
|
||||
this.currentPartitionEpoch = new TimelineHashMap<>(snapshotRegistry, 0);
|
||||
|
@ -1130,7 +1122,6 @@ public class ConsumerGroup extends ModernGroup<ConsumerGroupMember> {
|
|||
* Create a new consumer group according to the given classic group.
|
||||
*
|
||||
* @param snapshotRegistry The SnapshotRegistry.
|
||||
* @param metrics The GroupCoordinatorMetricsShard.
|
||||
* @param classicGroup The converted classic group.
|
||||
* @param topicHashCache The cache for topic hashes.
|
||||
* @param metadataImage The current metadata image for the Kafka cluster.
|
||||
|
@ -1141,13 +1132,12 @@ public class ConsumerGroup extends ModernGroup<ConsumerGroupMember> {
|
|||
*/
|
||||
public static ConsumerGroup fromClassicGroup(
|
||||
SnapshotRegistry snapshotRegistry,
|
||||
GroupCoordinatorMetricsShard metrics,
|
||||
ClassicGroup classicGroup,
|
||||
Map<String, Long> topicHashCache,
|
||||
CoordinatorMetadataImage metadataImage
|
||||
) {
|
||||
String groupId = classicGroup.groupId();
|
||||
ConsumerGroup consumerGroup = new ConsumerGroup(snapshotRegistry, groupId, metrics);
|
||||
ConsumerGroup consumerGroup = new ConsumerGroup(snapshotRegistry, groupId);
|
||||
consumerGroup.setGroupEpoch(classicGroup.generationId());
|
||||
consumerGroup.setTargetAssignmentEpoch(classicGroup.generationId());
|
||||
|
||||
|
|
|
@ -31,7 +31,6 @@ import org.apache.kafka.coordinator.group.Group;
|
|||
import org.apache.kafka.coordinator.group.OffsetExpirationCondition;
|
||||
import org.apache.kafka.coordinator.group.OffsetExpirationConditionImpl;
|
||||
import org.apache.kafka.coordinator.group.Utils;
|
||||
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard;
|
||||
import org.apache.kafka.coordinator.group.streams.topics.ConfiguredSubtopology;
|
||||
import org.apache.kafka.coordinator.group.streams.topics.ConfiguredTopology;
|
||||
import org.apache.kafka.timeline.SnapshotRegistry;
|
||||
|
@ -179,11 +178,6 @@ public class StreamsGroup implements Group {
|
|||
private final TimelineHashMap<String, TimelineHashMap<Integer, Set<String>>> currentStandbyTaskToProcessIds;
|
||||
private final TimelineHashMap<String, TimelineHashMap<Integer, Set<String>>> currentWarmupTaskToProcessIds;
|
||||
|
||||
/**
|
||||
* The coordinator metrics.
|
||||
*/
|
||||
private final GroupCoordinatorMetricsShard metrics;
|
||||
|
||||
/**
|
||||
* The Streams topology.
|
||||
*/
|
||||
|
@ -220,8 +214,7 @@ public class StreamsGroup implements Group {
|
|||
public StreamsGroup(
|
||||
LogContext logContext,
|
||||
SnapshotRegistry snapshotRegistry,
|
||||
String groupId,
|
||||
GroupCoordinatorMetricsShard metrics
|
||||
String groupId
|
||||
) {
|
||||
this.log = logContext.logger(StreamsGroup.class);
|
||||
this.logContext = logContext;
|
||||
|
@ -238,7 +231,6 @@ public class StreamsGroup implements Group {
|
|||
this.currentActiveTaskToProcessId = new TimelineHashMap<>(snapshotRegistry, 0);
|
||||
this.currentStandbyTaskToProcessIds = new TimelineHashMap<>(snapshotRegistry, 0);
|
||||
this.currentWarmupTaskToProcessIds = new TimelineHashMap<>(snapshotRegistry, 0);
|
||||
this.metrics = Objects.requireNonNull(metrics);
|
||||
this.topology = new TimelineObject<>(snapshotRegistry, Optional.empty());
|
||||
this.configuredTopology = new TimelineObject<>(snapshotRegistry, Optional.empty());
|
||||
}
|
||||
|
|
|
@ -84,7 +84,6 @@ import org.apache.kafka.coordinator.group.generated.StreamsGroupTargetAssignment
|
|||
import org.apache.kafka.coordinator.group.generated.StreamsGroupTargetAssignmentMetadataValue;
|
||||
import org.apache.kafka.coordinator.group.generated.StreamsGroupTopologyKey;
|
||||
import org.apache.kafka.coordinator.group.generated.StreamsGroupTopologyValue;
|
||||
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard;
|
||||
import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroup;
|
||||
import org.apache.kafka.coordinator.group.modern.share.ShareGroup;
|
||||
import org.apache.kafka.coordinator.group.streams.StreamsGroupHeartbeatResult;
|
||||
|
@ -1384,10 +1383,9 @@ public class GroupCoordinatorShardTest {
|
|||
ArgumentCaptor<List<CoordinatorRecord>> recordsCapture = ArgumentCaptor.forClass(List.class);
|
||||
|
||||
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
|
||||
GroupCoordinatorMetricsShard metricsShard = mock(GroupCoordinatorMetricsShard.class);
|
||||
|
||||
ConsumerGroup group1 = new ConsumerGroup(snapshotRegistry, "group-id", metricsShard);
|
||||
ConsumerGroup group2 = new ConsumerGroup(snapshotRegistry, "other-group-id", metricsShard);
|
||||
ConsumerGroup group1 = new ConsumerGroup(snapshotRegistry, "group-id");
|
||||
ConsumerGroup group2 = new ConsumerGroup(snapshotRegistry, "other-group-id");
|
||||
|
||||
when(groupMetadataManager.groupIds()).thenReturn(Set.of("group-id", "other-group-id"));
|
||||
when(groupMetadataManager.group("group-id")).thenReturn(group1);
|
||||
|
|
|
@ -45,7 +45,6 @@ import org.apache.kafka.coordinator.group.OffsetAndMetadata;
|
|||
import org.apache.kafka.coordinator.group.OffsetExpirationCondition;
|
||||
import org.apache.kafka.coordinator.group.OffsetExpirationConditionImpl;
|
||||
import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataValue;
|
||||
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard;
|
||||
import org.apache.kafka.coordinator.group.modern.Assignment;
|
||||
import org.apache.kafka.coordinator.group.modern.MemberState;
|
||||
import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroup;
|
||||
|
@ -81,7 +80,6 @@ import static org.junit.jupiter.api.Assertions.assertFalse;
|
|||
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.mockito.Mockito.mock;
|
||||
|
||||
public class ClassicGroupTest {
|
||||
private final String protocolType = "consumer";
|
||||
|
@ -1383,8 +1381,7 @@ public class ClassicGroupTest {
|
|||
|
||||
ConsumerGroup consumerGroup = new ConsumerGroup(
|
||||
new SnapshotRegistry(logContext),
|
||||
groupId,
|
||||
mock(GroupCoordinatorMetricsShard.class)
|
||||
groupId
|
||||
);
|
||||
consumerGroup.setGroupEpoch(10);
|
||||
consumerGroup.setTargetAssignmentEpoch(10);
|
||||
|
@ -1536,8 +1533,7 @@ public class ClassicGroupTest {
|
|||
|
||||
ConsumerGroup consumerGroup = new ConsumerGroup(
|
||||
new SnapshotRegistry(logContext),
|
||||
groupId,
|
||||
mock(GroupCoordinatorMetricsShard.class)
|
||||
groupId
|
||||
);
|
||||
consumerGroup.setGroupEpoch(10);
|
||||
consumerGroup.setTargetAssignmentEpoch(10);
|
||||
|
|
|
@ -25,7 +25,6 @@ import org.apache.kafka.common.errors.IllegalGenerationException;
|
|||
import org.apache.kafka.common.errors.StaleMemberEpochException;
|
||||
import org.apache.kafka.common.errors.UnknownMemberIdException;
|
||||
import org.apache.kafka.common.errors.UnsupportedVersionException;
|
||||
import org.apache.kafka.common.internals.Topic;
|
||||
import org.apache.kafka.common.message.ConsumerGroupDescribeResponseData;
|
||||
import org.apache.kafka.common.message.JoinGroupRequestData;
|
||||
import org.apache.kafka.common.protocol.ApiKeys;
|
||||
|
@ -45,7 +44,6 @@ import org.apache.kafka.coordinator.group.OffsetExpirationConditionImpl;
|
|||
import org.apache.kafka.coordinator.group.classic.ClassicGroup;
|
||||
import org.apache.kafka.coordinator.group.classic.ClassicGroupMember;
|
||||
import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataValue;
|
||||
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard;
|
||||
import org.apache.kafka.coordinator.group.modern.Assignment;
|
||||
import org.apache.kafka.coordinator.group.modern.MemberState;
|
||||
import org.apache.kafka.coordinator.group.modern.ModernGroup;
|
||||
|
@ -83,7 +81,6 @@ import static org.junit.jupiter.api.Assertions.assertFalse;
|
|||
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.mockito.Mockito.mock;
|
||||
|
||||
public class ConsumerGroupTest {
|
||||
|
||||
|
@ -91,8 +88,7 @@ public class ConsumerGroupTest {
|
|||
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
|
||||
return new ConsumerGroup(
|
||||
snapshotRegistry,
|
||||
groupId,
|
||||
mock(GroupCoordinatorMetricsShard.class)
|
||||
groupId
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -700,8 +696,7 @@ public class ConsumerGroupTest {
|
|||
@Test
|
||||
public void testUpdateInvertedAssignment() {
|
||||
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
|
||||
GroupCoordinatorMetricsShard metricsShard = mock(GroupCoordinatorMetricsShard.class);
|
||||
ConsumerGroup consumerGroup = new ConsumerGroup(snapshotRegistry, "test-group", metricsShard);
|
||||
ConsumerGroup consumerGroup = new ConsumerGroup(snapshotRegistry, "test-group");
|
||||
Uuid topicId = Uuid.randomUuid();
|
||||
String memberId1 = "member1";
|
||||
String memberId2 = "member2";
|
||||
|
@ -916,12 +911,7 @@ public class ConsumerGroupTest {
|
|||
@Test
|
||||
public void testAsListedGroup() {
|
||||
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
|
||||
GroupCoordinatorMetricsShard metricsShard = new GroupCoordinatorMetricsShard(
|
||||
snapshotRegistry,
|
||||
Map.of(),
|
||||
new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 0)
|
||||
);
|
||||
ConsumerGroup group = new ConsumerGroup(snapshotRegistry, "group-foo", metricsShard);
|
||||
ConsumerGroup group = new ConsumerGroup(snapshotRegistry, "group-foo");
|
||||
snapshotRegistry.idempotentCreateSnapshot(0);
|
||||
assertEquals(ConsumerGroup.ConsumerGroupState.EMPTY.toString(), group.stateAsString(0));
|
||||
group.updateMember(new ConsumerGroupMember.Builder("member1")
|
||||
|
@ -937,8 +927,7 @@ public class ConsumerGroupTest {
|
|||
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
|
||||
ConsumerGroup group = new ConsumerGroup(
|
||||
snapshotRegistry,
|
||||
"group-foo",
|
||||
mock(GroupCoordinatorMetricsShard.class)
|
||||
"group-foo"
|
||||
);
|
||||
|
||||
// Simulate a call from the admin client without member id and member epoch.
|
||||
|
@ -997,7 +986,7 @@ public class ConsumerGroupTest {
|
|||
long commitTimestamp = 20000L;
|
||||
long offsetsRetentionMs = 10000L;
|
||||
OffsetAndMetadata offsetAndMetadata = new OffsetAndMetadata(15000L, OptionalInt.empty(), "", commitTimestamp, OptionalLong.empty(), Uuid.ZERO_UUID);
|
||||
ConsumerGroup group = new ConsumerGroup(new SnapshotRegistry(new LogContext()), "group-id", mock(GroupCoordinatorMetricsShard.class));
|
||||
ConsumerGroup group = new ConsumerGroup(new SnapshotRegistry(new LogContext()), "group-id");
|
||||
|
||||
Optional<OffsetExpirationCondition> offsetExpirationCondition = group.offsetExpirationCondition();
|
||||
assertTrue(offsetExpirationCondition.isPresent());
|
||||
|
@ -1034,7 +1023,7 @@ public class ConsumerGroupTest {
|
|||
@Test
|
||||
public void testAsDescribedGroup() {
|
||||
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
|
||||
ConsumerGroup group = new ConsumerGroup(snapshotRegistry, "group-id-1", mock(GroupCoordinatorMetricsShard.class));
|
||||
ConsumerGroup group = new ConsumerGroup(snapshotRegistry, "group-id-1");
|
||||
snapshotRegistry.idempotentCreateSnapshot(0);
|
||||
assertEquals(ConsumerGroup.ConsumerGroupState.EMPTY.toString(), group.stateAsString(0));
|
||||
|
||||
|
@ -1071,12 +1060,7 @@ public class ConsumerGroupTest {
|
|||
@Test
|
||||
public void testIsInStatesCaseInsensitive() {
|
||||
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
|
||||
GroupCoordinatorMetricsShard metricsShard = new GroupCoordinatorMetricsShard(
|
||||
snapshotRegistry,
|
||||
Map.of(),
|
||||
new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 0)
|
||||
);
|
||||
ConsumerGroup group = new ConsumerGroup(snapshotRegistry, "group-foo", metricsShard);
|
||||
ConsumerGroup group = new ConsumerGroup(snapshotRegistry, "group-foo");
|
||||
snapshotRegistry.idempotentCreateSnapshot(0);
|
||||
assertTrue(group.isInStates(Set.of("empty"), 0));
|
||||
assertFalse(group.isInStates(Set.of("Empty"), 0));
|
||||
|
@ -1307,7 +1291,6 @@ public class ConsumerGroupTest {
|
|||
|
||||
ConsumerGroup consumerGroup = ConsumerGroup.fromClassicGroup(
|
||||
new SnapshotRegistry(logContext),
|
||||
mock(GroupCoordinatorMetricsShard.class),
|
||||
classicGroup,
|
||||
new HashMap<>(),
|
||||
metadataImage
|
||||
|
@ -1315,8 +1298,7 @@ public class ConsumerGroupTest {
|
|||
|
||||
ConsumerGroup expectedConsumerGroup = new ConsumerGroup(
|
||||
new SnapshotRegistry(logContext),
|
||||
groupId,
|
||||
mock(GroupCoordinatorMetricsShard.class)
|
||||
groupId
|
||||
);
|
||||
expectedConsumerGroup.setGroupEpoch(10);
|
||||
expectedConsumerGroup.setTargetAssignmentEpoch(10);
|
||||
|
|
|
@ -16,7 +16,6 @@
|
|||
*/
|
||||
package org.apache.kafka.coordinator.group.streams;
|
||||
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.common.errors.GroupNotEmptyException;
|
||||
import org.apache.kafka.common.errors.StaleMemberEpochException;
|
||||
|
@ -43,7 +42,6 @@ import org.apache.kafka.coordinator.group.generated.StreamsGroupTargetAssignment
|
|||
import org.apache.kafka.coordinator.group.generated.StreamsGroupTargetAssignmentMetadataKey;
|
||||
import org.apache.kafka.coordinator.group.generated.StreamsGroupTopologyKey;
|
||||
import org.apache.kafka.coordinator.group.generated.StreamsGroupTopologyValue;
|
||||
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard;
|
||||
import org.apache.kafka.coordinator.group.streams.StreamsGroup.StreamsGroupState;
|
||||
import org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.TaskRole;
|
||||
import org.apache.kafka.coordinator.group.streams.topics.ConfiguredTopology;
|
||||
|
@ -90,8 +88,7 @@ public class StreamsGroupTest {
|
|||
return new StreamsGroup(
|
||||
LOG_CONTEXT,
|
||||
snapshotRegistry,
|
||||
groupId,
|
||||
mock(GroupCoordinatorMetricsShard.class)
|
||||
groupId
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -693,8 +690,7 @@ public class StreamsGroupTest {
|
|||
StreamsGroup group = new StreamsGroup(
|
||||
LOG_CONTEXT,
|
||||
snapshotRegistry,
|
||||
"group-foo",
|
||||
mock(GroupCoordinatorMetricsShard.class)
|
||||
"group-foo"
|
||||
);
|
||||
group.setGroupEpoch(1);
|
||||
group.setTopology(new StreamsTopology(1, Map.of()));
|
||||
|
@ -719,8 +715,7 @@ public class StreamsGroupTest {
|
|||
StreamsGroup group = new StreamsGroup(
|
||||
LOG_CONTEXT,
|
||||
snapshotRegistry,
|
||||
"group-foo",
|
||||
mock(GroupCoordinatorMetricsShard.class)
|
||||
"group-foo"
|
||||
);
|
||||
|
||||
// Simulate a call from the admin client without member ID and member epoch.
|
||||
|
@ -790,7 +785,7 @@ public class StreamsGroupTest {
|
|||
long commitTimestamp = 20000L;
|
||||
long offsetsRetentionMs = 10000L;
|
||||
OffsetAndMetadata offsetAndMetadata = new OffsetAndMetadata(15000L, OptionalInt.empty(), "", commitTimestamp, OptionalLong.empty(), Uuid.ZERO_UUID);
|
||||
StreamsGroup group = new StreamsGroup(LOG_CONTEXT, new SnapshotRegistry(LOG_CONTEXT), "group-id", mock(GroupCoordinatorMetricsShard.class));
|
||||
StreamsGroup group = new StreamsGroup(LOG_CONTEXT, new SnapshotRegistry(LOG_CONTEXT), "group-id");
|
||||
|
||||
Optional<OffsetExpirationCondition> offsetExpirationCondition = group.offsetExpirationCondition();
|
||||
assertTrue(offsetExpirationCondition.isPresent());
|
||||
|
@ -803,7 +798,7 @@ public class StreamsGroupTest {
|
|||
@Test
|
||||
public void testAsDescribedGroup() {
|
||||
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
|
||||
StreamsGroup group = new StreamsGroup(LOG_CONTEXT, snapshotRegistry, "group-id-1", mock(GroupCoordinatorMetricsShard.class));
|
||||
StreamsGroup group = new StreamsGroup(LOG_CONTEXT, snapshotRegistry, "group-id-1");
|
||||
snapshotRegistry.idempotentCreateSnapshot(0);
|
||||
assertEquals(StreamsGroup.StreamsGroupState.EMPTY.toString(), group.stateAsString(0));
|
||||
|
||||
|
@ -887,12 +882,7 @@ public class StreamsGroupTest {
|
|||
@Test
|
||||
public void testIsInStatesCaseInsensitiveAndUnderscored() {
|
||||
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(LOG_CONTEXT);
|
||||
GroupCoordinatorMetricsShard metricsShard = new GroupCoordinatorMetricsShard(
|
||||
snapshotRegistry,
|
||||
Map.of(),
|
||||
new TopicPartition("__consumer_offsets", 0)
|
||||
);
|
||||
StreamsGroup group = new StreamsGroup(LOG_CONTEXT, snapshotRegistry, "group-foo", metricsShard);
|
||||
StreamsGroup group = new StreamsGroup(LOG_CONTEXT, snapshotRegistry, "group-foo");
|
||||
snapshotRegistry.idempotentCreateSnapshot(0);
|
||||
assertTrue(group.isInStates(Set.of("empty"), 0));
|
||||
assertFalse(group.isInStates(Set.of("Empty"), 0));
|
||||
|
@ -911,8 +901,7 @@ public class StreamsGroupTest {
|
|||
StreamsGroup streamsGroup = new StreamsGroup(
|
||||
LOG_CONTEXT,
|
||||
snapshotRegistry,
|
||||
"group-foo",
|
||||
mock(GroupCoordinatorMetricsShard.class)
|
||||
"group-foo"
|
||||
);
|
||||
|
||||
MetadataImage metadataImage = new MetadataImageBuilder()
|
||||
|
@ -933,8 +922,7 @@ public class StreamsGroupTest {
|
|||
StreamsGroup streamsGroup = new StreamsGroup(
|
||||
LOG_CONTEXT,
|
||||
snapshotRegistry,
|
||||
"test-group",
|
||||
mock(GroupCoordinatorMetricsShard.class)
|
||||
"test-group"
|
||||
);
|
||||
streamsGroup.updateMember(new StreamsGroupMember.Builder("member1")
|
||||
.setMemberEpoch(1)
|
||||
|
@ -961,8 +949,7 @@ public class StreamsGroupTest {
|
|||
public void testIsSubscribedToTopic() {
|
||||
LogContext logContext = new LogContext();
|
||||
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(logContext);
|
||||
GroupCoordinatorMetricsShard metricsShard = mock(GroupCoordinatorMetricsShard.class);
|
||||
StreamsGroup streamsGroup = new StreamsGroup(logContext, snapshotRegistry, "test-group", metricsShard);
|
||||
StreamsGroup streamsGroup = new StreamsGroup(logContext, snapshotRegistry, "test-group");
|
||||
|
||||
assertFalse(streamsGroup.isSubscribedToTopic("test-topic1"));
|
||||
assertFalse(streamsGroup.isSubscribedToTopic("test-topic2"));
|
||||
|
@ -1008,8 +995,7 @@ public class StreamsGroupTest {
|
|||
String memberId2 = "test-member-id2";
|
||||
LogContext logContext = new LogContext();
|
||||
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(logContext);
|
||||
GroupCoordinatorMetricsShard metricsShard = mock(GroupCoordinatorMetricsShard.class);
|
||||
StreamsGroup streamsGroup = new StreamsGroup(logContext, snapshotRegistry, "test-group", metricsShard);
|
||||
StreamsGroup streamsGroup = new StreamsGroup(logContext, snapshotRegistry, "test-group");
|
||||
|
||||
streamsGroup.updateMember(streamsGroup.getOrCreateDefaultMember(memberId1));
|
||||
streamsGroup.updateMember(streamsGroup.getOrCreateDefaultMember(memberId2));
|
||||
|
|
|
@ -0,0 +1,355 @@
|
|||
/*
|
||||
* 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.jmh.coordinator;
|
||||
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.common.compress.Compression;
|
||||
import org.apache.kafka.common.internals.Topic;
|
||||
import org.apache.kafka.common.metrics.Metrics;
|
||||
import org.apache.kafka.common.record.ControlRecordType;
|
||||
import org.apache.kafka.common.record.EndTransactionMarker;
|
||||
import org.apache.kafka.common.record.MemoryRecords;
|
||||
import org.apache.kafka.common.record.SimpleRecord;
|
||||
import org.apache.kafka.common.utils.LogContext;
|
||||
import org.apache.kafka.common.utils.MockTime;
|
||||
import org.apache.kafka.coordinator.common.runtime.CoordinatorLoader;
|
||||
import org.apache.kafka.coordinator.common.runtime.CoordinatorLoaderImpl;
|
||||
import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
|
||||
import org.apache.kafka.coordinator.common.runtime.MockCoordinatorExecutor;
|
||||
import org.apache.kafka.coordinator.common.runtime.MockCoordinatorTimer;
|
||||
import org.apache.kafka.coordinator.common.runtime.SnapshottableCoordinator;
|
||||
import org.apache.kafka.coordinator.group.GroupConfigManager;
|
||||
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig;
|
||||
import org.apache.kafka.coordinator.group.GroupCoordinatorRecordHelpers;
|
||||
import org.apache.kafka.coordinator.group.GroupCoordinatorRecordSerde;
|
||||
import org.apache.kafka.coordinator.group.GroupCoordinatorShard;
|
||||
import org.apache.kafka.coordinator.group.OffsetAndMetadata;
|
||||
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics;
|
||||
import org.apache.kafka.server.storage.log.FetchIsolation;
|
||||
import org.apache.kafka.storage.internals.log.FetchDataInfo;
|
||||
import org.apache.kafka.storage.internals.log.LogOffsetMetadata;
|
||||
import org.apache.kafka.storage.internals.log.UnifiedLog;
|
||||
import org.apache.kafka.timeline.SnapshotRegistry;
|
||||
|
||||
import com.yammer.metrics.core.MetricsRegistry;
|
||||
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
import org.openjdk.jmh.annotations.BenchmarkMode;
|
||||
import org.openjdk.jmh.annotations.Fork;
|
||||
import org.openjdk.jmh.annotations.Level;
|
||||
import org.openjdk.jmh.annotations.Measurement;
|
||||
import org.openjdk.jmh.annotations.Mode;
|
||||
import org.openjdk.jmh.annotations.OutputTimeUnit;
|
||||
import org.openjdk.jmh.annotations.Param;
|
||||
import org.openjdk.jmh.annotations.Scope;
|
||||
import org.openjdk.jmh.annotations.Setup;
|
||||
import org.openjdk.jmh.annotations.State;
|
||||
import org.openjdk.jmh.annotations.Threads;
|
||||
import org.openjdk.jmh.annotations.Warmup;
|
||||
import org.openjdk.jmh.infra.BenchmarkParams;
|
||||
import org.openjdk.jmh.infra.IterationParams;
|
||||
import org.openjdk.jmh.runner.IterationType;
|
||||
import org.openjdk.jmh.runner.Runner;
|
||||
import org.openjdk.jmh.runner.RunnerException;
|
||||
import org.openjdk.jmh.runner.options.Options;
|
||||
import org.openjdk.jmh.runner.options.OptionsBuilder;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.OptionalInt;
|
||||
import java.util.OptionalLong;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.function.Function;
|
||||
|
||||
@State(Scope.Benchmark)
|
||||
@Fork(value = 1)
|
||||
@Warmup(iterations = 1)
|
||||
@Measurement(iterations = 1)
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MILLISECONDS)
|
||||
public class GroupCoordinatorShardLoadingBenchmark {
|
||||
|
||||
private static final String GROUP_ID = "test-group";
|
||||
|
||||
@Param({"1", "4", "16", "64", "256", "1024", "4096", "16384", "65536", "262144", "1048576"})
|
||||
private long commitInterval;
|
||||
|
||||
@Param({"8192"})
|
||||
private int batchCount;
|
||||
|
||||
@Param({"2048"})
|
||||
private int batchSize;
|
||||
|
||||
private TopicPartition topicPartition;
|
||||
private MockTime time;
|
||||
private GroupCoordinatorConfig config;
|
||||
private GroupCoordinatorRecordSerde serde;
|
||||
private GroupCoordinatorShard coordinatorShard;
|
||||
private SnapshottableCoordinator<GroupCoordinatorShard, CoordinatorRecord> snapshottableCoordinator;
|
||||
private UnifiedLog offsetCommitLog;
|
||||
private UnifiedLog transactionalOffsetCommitLog;
|
||||
|
||||
static class OffsetCommitLog extends MockLog {
|
||||
private final int batchCount;
|
||||
private final SimpleRecord[] batch;
|
||||
|
||||
public OffsetCommitLog(TopicPartition tp, int batchSize, int batchCount) throws IOException {
|
||||
super(tp);
|
||||
|
||||
this.batchCount = batchCount;
|
||||
|
||||
List<SimpleRecord> batchRecords = new ArrayList<>();
|
||||
for (int i = 0; i < batchSize; i++) {
|
||||
String topic = "topic-" + i;
|
||||
int partition = 0;
|
||||
|
||||
OffsetAndMetadata offsetAndMetadata = new OffsetAndMetadata(
|
||||
0L,
|
||||
OptionalInt.of(0),
|
||||
OffsetAndMetadata.NO_METADATA,
|
||||
0L,
|
||||
OptionalLong.empty(),
|
||||
Uuid.randomUuid()
|
||||
);
|
||||
|
||||
CoordinatorRecord coordinatorRecord = GroupCoordinatorRecordHelpers.newOffsetCommitRecord(
|
||||
GROUP_ID, topic, partition, offsetAndMetadata
|
||||
);
|
||||
|
||||
byte[] keyBytes = new GroupCoordinatorRecordSerde().serializeKey(coordinatorRecord);
|
||||
byte[] valueBytes = new GroupCoordinatorRecordSerde().serializeValue(coordinatorRecord);
|
||||
SimpleRecord simpleRecord = new SimpleRecord(keyBytes, valueBytes);
|
||||
batchRecords.add(simpleRecord);
|
||||
}
|
||||
|
||||
this.batch = batchRecords.toArray(new SimpleRecord[0]);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long logStartOffset() {
|
||||
return 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long logEndOffset() {
|
||||
if (batch == null) {
|
||||
return 0L;
|
||||
}
|
||||
|
||||
return (long) batchCount * (long) batch.length;
|
||||
}
|
||||
|
||||
@Override
|
||||
public FetchDataInfo read(long startOffset, int maxLength, FetchIsolation isolation, boolean minOneMessage) {
|
||||
if (startOffset < 0 || startOffset >= logEndOffset()) {
|
||||
return new FetchDataInfo(new LogOffsetMetadata(startOffset), MemoryRecords.EMPTY);
|
||||
}
|
||||
|
||||
MemoryRecords records = MemoryRecords.withRecords(
|
||||
startOffset,
|
||||
Compression.NONE,
|
||||
batch
|
||||
);
|
||||
return new FetchDataInfo(new LogOffsetMetadata(startOffset), records);
|
||||
}
|
||||
}
|
||||
|
||||
static class TransactionalOffsetCommitLog extends MockLog {
|
||||
private final int batchCount;
|
||||
private final SimpleRecord[] batch;
|
||||
private final long producerId;
|
||||
private final short producerEpoch;
|
||||
private final int coordinatorEpoch;
|
||||
|
||||
public TransactionalOffsetCommitLog(TopicPartition tp, int batchSize, int batchCount) throws IOException {
|
||||
super(tp);
|
||||
|
||||
this.batchCount = batchCount;
|
||||
this.producerId = 1000L;
|
||||
this.producerEpoch = 0;
|
||||
this.coordinatorEpoch = 100;
|
||||
|
||||
List<SimpleRecord> batchRecords = new ArrayList<>();
|
||||
for (int i = 0; i < batchSize - 1; i++) {
|
||||
String topic = "topic-" + i;
|
||||
int partition = 0;
|
||||
|
||||
OffsetAndMetadata offsetAndMetadata = new OffsetAndMetadata(
|
||||
0L,
|
||||
OptionalInt.of(0),
|
||||
OffsetAndMetadata.NO_METADATA,
|
||||
0L,
|
||||
OptionalLong.empty(),
|
||||
Uuid.randomUuid()
|
||||
);
|
||||
|
||||
CoordinatorRecord coordinatorRecord = GroupCoordinatorRecordHelpers.newOffsetCommitRecord(
|
||||
GROUP_ID, topic, partition, offsetAndMetadata
|
||||
);
|
||||
|
||||
byte[] keyBytes = new GroupCoordinatorRecordSerde().serializeKey(coordinatorRecord);
|
||||
byte[] valueBytes = new GroupCoordinatorRecordSerde().serializeValue(coordinatorRecord);
|
||||
SimpleRecord simpleRecord = new SimpleRecord(keyBytes, valueBytes);
|
||||
batchRecords.add(simpleRecord);
|
||||
}
|
||||
|
||||
this.batch = batchRecords.toArray(new SimpleRecord[0]);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long logStartOffset() {
|
||||
return 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long logEndOffset() {
|
||||
if (batch == null) {
|
||||
return 0L;
|
||||
}
|
||||
|
||||
return (long) (batch.length + 1) * (long) batchCount;
|
||||
}
|
||||
|
||||
@Override
|
||||
public FetchDataInfo read(long startOffset, int maxLength, FetchIsolation isolation, boolean minOneMessage) {
|
||||
if (startOffset < 0 || startOffset >= logEndOffset()) {
|
||||
return new FetchDataInfo(new LogOffsetMetadata(startOffset), MemoryRecords.EMPTY);
|
||||
}
|
||||
|
||||
// Repeat the batch followed by a commit marker.
|
||||
long patternLength = batch.length + 1;
|
||||
if (startOffset % patternLength < batch.length) {
|
||||
MemoryRecords records = MemoryRecords.withTransactionalRecords(
|
||||
startOffset,
|
||||
Compression.NONE,
|
||||
producerId,
|
||||
producerEpoch,
|
||||
0,
|
||||
0,
|
||||
batch
|
||||
);
|
||||
return new FetchDataInfo(new LogOffsetMetadata(startOffset), records);
|
||||
} else {
|
||||
MemoryRecords records = MemoryRecords.withEndTransactionMarker(
|
||||
startOffset,
|
||||
0L,
|
||||
0,
|
||||
producerId,
|
||||
producerEpoch,
|
||||
new EndTransactionMarker(ControlRecordType.COMMIT, coordinatorEpoch)
|
||||
);
|
||||
return new FetchDataInfo(new LogOffsetMetadata(startOffset), records);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Setup(Level.Trial)
|
||||
public void setup() throws Exception {
|
||||
topicPartition = new TopicPartition("__consumer_offsets", 0);
|
||||
time = new MockTime();
|
||||
Map<String, Object> props = new HashMap<>();
|
||||
config = GroupCoordinatorConfig.fromProps(props);
|
||||
serde = new GroupCoordinatorRecordSerde();
|
||||
}
|
||||
|
||||
@Setup(Level.Iteration)
|
||||
public void setupIteration(BenchmarkParams benchmarkParams, IterationParams iterationParams) throws IOException {
|
||||
// Reduce the data size for warmup iterations, since transactional offset commit loading
|
||||
// takes longer than 20 seconds.
|
||||
int iterationBatchCount = batchCount;
|
||||
if (iterationParams.getType() == IterationType.WARMUP) {
|
||||
iterationBatchCount = Math.min(iterationBatchCount, 1024);
|
||||
}
|
||||
|
||||
offsetCommitLog = new OffsetCommitLog(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 0), batchSize, iterationBatchCount);
|
||||
transactionalOffsetCommitLog = new TransactionalOffsetCommitLog(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 0), batchSize, iterationBatchCount);
|
||||
}
|
||||
|
||||
@Setup(Level.Invocation)
|
||||
public void setupInvocation() {
|
||||
GroupConfigManager configManager = new GroupConfigManager(new HashMap<>());
|
||||
LogContext logContext = new LogContext();
|
||||
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(logContext);
|
||||
|
||||
MetricsRegistry metricsRegistry = new MetricsRegistry();
|
||||
Metrics metrics = new Metrics();
|
||||
GroupCoordinatorMetrics coordinatorMetrics = new GroupCoordinatorMetrics(metricsRegistry, metrics);
|
||||
|
||||
coordinatorShard = new GroupCoordinatorShard.Builder(config, configManager)
|
||||
.withAuthorizerPlugin(Optional.empty())
|
||||
.withLogContext(logContext)
|
||||
.withSnapshotRegistry(snapshotRegistry)
|
||||
.withTime(time)
|
||||
.withTimer(new MockCoordinatorTimer<>(time))
|
||||
.withExecutor(new MockCoordinatorExecutor<>())
|
||||
.withCoordinatorMetrics(coordinatorMetrics)
|
||||
.withTopicPartition(topicPartition)
|
||||
.build();
|
||||
|
||||
snapshottableCoordinator = new SnapshottableCoordinator<>(
|
||||
logContext,
|
||||
snapshotRegistry,
|
||||
coordinatorShard,
|
||||
topicPartition
|
||||
);
|
||||
}
|
||||
|
||||
private CoordinatorLoader.LoadSummary loadRecords(UnifiedLog log) throws ExecutionException, InterruptedException {
|
||||
Function<TopicPartition, Optional<UnifiedLog>> partitionLogSupplier = tp -> Optional.of(log);
|
||||
Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier = tp -> Optional.of(log.logEndOffset());
|
||||
|
||||
CoordinatorLoaderImpl<CoordinatorRecord> loader = new CoordinatorLoaderImpl<>(
|
||||
time,
|
||||
partitionLogSupplier,
|
||||
partitionLogEndOffsetSupplier,
|
||||
serde,
|
||||
config.offsetsLoadBufferSize(),
|
||||
commitInterval
|
||||
);
|
||||
|
||||
return loader.load(topicPartition, snapshottableCoordinator).get();
|
||||
}
|
||||
|
||||
@Benchmark
|
||||
@Threads(1)
|
||||
@OutputTimeUnit(TimeUnit.MILLISECONDS)
|
||||
public CoordinatorLoader.LoadSummary loadOffsetCommitRecords() throws ExecutionException, InterruptedException {
|
||||
return loadRecords(offsetCommitLog);
|
||||
}
|
||||
|
||||
@Benchmark
|
||||
@Threads(1)
|
||||
@OutputTimeUnit(TimeUnit.MILLISECONDS)
|
||||
public CoordinatorLoader.LoadSummary loadTransactionalOffsetCommitRecords() throws ExecutionException, InterruptedException {
|
||||
return loadRecords(transactionalOffsetCommitLog);
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws RunnerException {
|
||||
Options opt = new OptionsBuilder()
|
||||
.include(GroupCoordinatorShardLoadingBenchmark.class.getSimpleName())
|
||||
.forks(1)
|
||||
.build();
|
||||
|
||||
new Runner(opt).run();
|
||||
}
|
||||
}
|
|
@ -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.jmh.coordinator;
|
||||
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.apache.kafka.server.storage.log.FetchIsolation;
|
||||
import org.apache.kafka.server.util.Scheduler;
|
||||
import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache;
|
||||
import org.apache.kafka.storage.internals.log.FetchDataInfo;
|
||||
import org.apache.kafka.storage.internals.log.LocalLog;
|
||||
import org.apache.kafka.storage.internals.log.LogOffsetsListener;
|
||||
import org.apache.kafka.storage.internals.log.LogSegments;
|
||||
import org.apache.kafka.storage.internals.log.ProducerStateManager;
|
||||
import org.apache.kafka.storage.internals.log.UnifiedLog;
|
||||
import org.apache.kafka.storage.log.metrics.BrokerTopicStats;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Optional;
|
||||
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
public abstract class MockLog extends UnifiedLog {
|
||||
|
||||
public MockLog(TopicPartition tp) throws IOException {
|
||||
super(
|
||||
0,
|
||||
createMockLocalLog(tp),
|
||||
mock(BrokerTopicStats.class),
|
||||
Integer.MAX_VALUE,
|
||||
mock(LeaderEpochFileCache.class),
|
||||
mock(ProducerStateManager.class),
|
||||
Optional.empty(),
|
||||
false,
|
||||
LogOffsetsListener.NO_OP_OFFSETS_LISTENER
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public abstract long logStartOffset();
|
||||
|
||||
@Override
|
||||
public abstract long logEndOffset();
|
||||
|
||||
@Override
|
||||
public long highWatermark() {
|
||||
return logEndOffset();
|
||||
}
|
||||
|
||||
@Override
|
||||
public abstract FetchDataInfo read(long startOffset, int maxLength, FetchIsolation isolation, boolean minOneMessage);
|
||||
|
||||
private static LocalLog createMockLocalLog(TopicPartition tp) {
|
||||
LocalLog localLog = mock(LocalLog.class);
|
||||
when(localLog.scheduler()).thenReturn(mock(Scheduler.class));
|
||||
when(localLog.segments()).thenReturn(mock(LogSegments.class));
|
||||
when(localLog.topicPartition()).thenReturn(tp);
|
||||
return localLog;
|
||||
}
|
||||
}
|
|
@ -17,6 +17,8 @@
|
|||
|
||||
package org.apache.kafka.controller;
|
||||
|
||||
import org.apache.kafka.common.Node;
|
||||
import org.apache.kafka.common.network.ListenerName;
|
||||
import org.apache.kafka.common.protocol.ObjectSerializationCache;
|
||||
import org.apache.kafka.common.utils.BufferSupplier;
|
||||
import org.apache.kafka.common.utils.LogContext;
|
||||
|
@ -758,6 +760,11 @@ public final class MockRaftClient implements RaftClient<ApiMessageAndVersion>, A
|
|||
return OptionalInt.of(nodeId);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<Node> voterNode(int id, ListenerName listenerName) {
|
||||
return Optional.empty();
|
||||
}
|
||||
|
||||
public List<RaftClient.Listener<ApiMessageAndVersion>> listeners() {
|
||||
final CompletableFuture<List<RaftClient.Listener<ApiMessageAndVersion>>> future = new CompletableFuture<>();
|
||||
eventQueue.append(() ->
|
||||
|
|
|
@ -17,6 +17,8 @@
|
|||
|
||||
package org.apache.kafka.image.publisher;
|
||||
|
||||
import org.apache.kafka.common.Node;
|
||||
import org.apache.kafka.common.network.ListenerName;
|
||||
import org.apache.kafka.common.utils.MockTime;
|
||||
import org.apache.kafka.image.FakeSnapshotWriter;
|
||||
import org.apache.kafka.image.MetadataImageTest;
|
||||
|
@ -73,6 +75,11 @@ public class SnapshotEmitterTest {
|
|||
return OptionalInt.empty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<Node> voterNode(int id, ListenerName listenerName) {
|
||||
return Optional.empty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long prepareAppend(int epoch, List<ApiMessageAndVersion> records) {
|
||||
return 0;
|
||||
|
|
|
@ -3793,6 +3793,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<Node> voterNode(int id, ListenerName listenerName) {
|
||||
return partitionState.lastVoterSet().voterNode(id, listenerName);
|
||||
}
|
||||
|
|
|
@ -16,7 +16,9 @@
|
|||
*/
|
||||
package org.apache.kafka.raft;
|
||||
|
||||
import org.apache.kafka.common.Node;
|
||||
import org.apache.kafka.common.errors.ApiException;
|
||||
import org.apache.kafka.common.network.ListenerName;
|
||||
import org.apache.kafka.raft.errors.BufferAllocationException;
|
||||
import org.apache.kafka.raft.errors.NotLeaderException;
|
||||
import org.apache.kafka.server.common.KRaftVersion;
|
||||
|
@ -129,6 +131,15 @@ public interface RaftClient<T> extends AutoCloseable {
|
|||
*/
|
||||
OptionalInt nodeId();
|
||||
|
||||
/**
|
||||
* Returns the node information for a given voter id and listener.
|
||||
*
|
||||
* @param id the id of the voter
|
||||
* @param listenerName the name of the listener
|
||||
* @return the node information if it exists, otherwise {@code Optional.empty()}
|
||||
*/
|
||||
Optional<Node> voterNode(int id, ListenerName listenerName);
|
||||
|
||||
/**
|
||||
* Prepare a list of records to be appended to the log.
|
||||
*
|
||||
|
|
|
@ -0,0 +1,54 @@
|
|||
/*
|
||||
* 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.raft;
|
||||
|
||||
import org.apache.kafka.common.protocol.ApiMessage;
|
||||
import org.apache.kafka.common.requests.RequestContext;
|
||||
import org.apache.kafka.common.requests.RequestHeader;
|
||||
import org.apache.kafka.server.common.serialization.RecordSerde;
|
||||
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
|
||||
public interface RaftManager<T> {
|
||||
|
||||
CompletableFuture<ApiMessage> handleRequest(
|
||||
RequestContext context,
|
||||
RequestHeader header,
|
||||
ApiMessage request,
|
||||
long createdTimeMs
|
||||
);
|
||||
|
||||
/**
|
||||
* Returns a Raft client.
|
||||
* <p>
|
||||
* Always returns the same instance. Callers must NOT close it.
|
||||
*/
|
||||
RaftClient<T> client();
|
||||
|
||||
/**
|
||||
* Returns a replicated log.
|
||||
* <p>
|
||||
* Always returns the same instance. Callers must NOT close it.
|
||||
*/
|
||||
ReplicatedLog replicatedLog();
|
||||
|
||||
/**
|
||||
* Returns the record Serde.
|
||||
*/
|
||||
RecordSerde<T> recordSerde();
|
||||
}
|
|
@ -57,11 +57,12 @@ import org.apache.kafka.test.TestUtils;
|
|||
import org.junit.jupiter.api.AfterAll;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Tag;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.TestInfo;
|
||||
import org.junit.jupiter.api.Timeout;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.Arguments;
|
||||
import org.junit.jupiter.params.provider.MethodSource;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.Field;
|
||||
|
@ -76,6 +77,7 @@ import java.util.concurrent.ExecutionException;
|
|||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static java.util.Collections.singleton;
|
||||
import static org.apache.kafka.streams.query.StateQueryRequest.inStore;
|
||||
|
@ -99,6 +101,7 @@ public class IQv2IntegrationTest {
|
|||
public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(NUM_BROKERS);
|
||||
|
||||
private KafkaStreams kafkaStreams;
|
||||
private String groupProtocol;
|
||||
|
||||
@BeforeAll
|
||||
public static void before()
|
||||
|
@ -149,8 +152,8 @@ public class IQv2IntegrationTest {
|
|||
));
|
||||
}
|
||||
|
||||
@BeforeEach
|
||||
public void beforeTest(final TestInfo testInfo) {
|
||||
private void setup(final String groupProtocol, final TestInfo testInfo) {
|
||||
this.groupProtocol = groupProtocol;
|
||||
final StreamsBuilder builder = new StreamsBuilder();
|
||||
|
||||
builder.table(
|
||||
|
@ -159,7 +162,6 @@ public class IQv2IntegrationTest {
|
|||
Materialized.as(STORE_NAME)
|
||||
);
|
||||
|
||||
|
||||
final String safeTestName = safeUniqueTestName(testInfo);
|
||||
kafkaStreams = new KafkaStreams(builder.build(), streamsConfiguration(safeTestName));
|
||||
kafkaStreams.cleanUp();
|
||||
|
@ -167,17 +169,21 @@ public class IQv2IntegrationTest {
|
|||
|
||||
@AfterEach
|
||||
public void afterTest() {
|
||||
if (kafkaStreams != null) {
|
||||
kafkaStreams.close(Duration.ofSeconds(60));
|
||||
kafkaStreams.cleanUp();
|
||||
}
|
||||
}
|
||||
|
||||
@AfterAll
|
||||
public static void after() {
|
||||
CLUSTER.stop();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldFailUnknownStore() {
|
||||
@ParameterizedTest(name = "{1}")
|
||||
@MethodSource("groupProtocolParameters")
|
||||
public void shouldFailUnknownStore(final String groupProtocol, final String testName, final TestInfo testInfo) {
|
||||
setup(groupProtocol, testInfo);
|
||||
final KeyQuery<Integer, ValueAndTimestamp<Integer>> query = KeyQuery.withKey(1);
|
||||
final StateQueryRequest<ValueAndTimestamp<Integer>> request =
|
||||
inStore("unknown-store").withQuery(query);
|
||||
|
@ -185,8 +191,10 @@ public class IQv2IntegrationTest {
|
|||
assertThrows(UnknownStateStoreException.class, () -> kafkaStreams.query(request));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldFailNotStarted() {
|
||||
@ParameterizedTest(name = "{1}")
|
||||
@MethodSource("groupProtocolParameters")
|
||||
public void shouldFailNotStarted(final String groupProtocol, final String testName, final TestInfo testInfo) {
|
||||
setup(groupProtocol, testInfo);
|
||||
final KeyQuery<Integer, ValueAndTimestamp<Integer>> query = KeyQuery.withKey(1);
|
||||
final StateQueryRequest<ValueAndTimestamp<Integer>> request =
|
||||
inStore(STORE_NAME).withQuery(query);
|
||||
|
@ -194,8 +202,10 @@ public class IQv2IntegrationTest {
|
|||
assertThrows(StreamsNotStartedException.class, () -> kafkaStreams.query(request));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldFailStopped() {
|
||||
@ParameterizedTest(name = "{1}")
|
||||
@MethodSource("groupProtocolParameters")
|
||||
public void shouldFailStopped(final String groupProtocol, final String testName, final TestInfo testInfo) {
|
||||
setup(groupProtocol, testInfo);
|
||||
final KeyQuery<Integer, ValueAndTimestamp<Integer>> query = KeyQuery.withKey(1);
|
||||
final StateQueryRequest<ValueAndTimestamp<Integer>> request =
|
||||
inStore(STORE_NAME).withQuery(query);
|
||||
|
@ -205,9 +215,11 @@ public class IQv2IntegrationTest {
|
|||
assertThrows(StreamsStoppedException.class, () -> kafkaStreams.query(request));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldRejectNonRunningActive()
|
||||
@ParameterizedTest(name = "{1}")
|
||||
@MethodSource("groupProtocolParameters")
|
||||
public void shouldRejectNonRunningActive(final String groupProtocol, final String testName, final TestInfo testInfo)
|
||||
throws NoSuchFieldException, IllegalAccessException {
|
||||
setup(groupProtocol, testInfo);
|
||||
final KeyQuery<Integer, ValueAndTimestamp<Integer>> query = KeyQuery.withKey(1);
|
||||
final StateQueryRequest<ValueAndTimestamp<Integer>> request =
|
||||
inStore(STORE_NAME).withQuery(query).requireActive();
|
||||
|
@ -261,8 +273,10 @@ public class IQv2IntegrationTest {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldFetchFromPartition() {
|
||||
@ParameterizedTest(name = "{1}")
|
||||
@MethodSource("groupProtocolParameters")
|
||||
public void shouldFetchFromPartition(final String groupProtocol, final String testName, final TestInfo testInfo) {
|
||||
setup(groupProtocol, testInfo);
|
||||
final KeyQuery<Integer, ValueAndTimestamp<Integer>> query = KeyQuery.withKey(1);
|
||||
final int partition = 1;
|
||||
final Set<Integer> partitions = singleton(partition);
|
||||
|
@ -276,8 +290,10 @@ public class IQv2IntegrationTest {
|
|||
assertThat(result.getPartitionResults().keySet(), equalTo(partitions));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldFetchExplicitlyFromAllPartitions() {
|
||||
@ParameterizedTest(name = "{1}")
|
||||
@MethodSource("groupProtocolParameters")
|
||||
public void shouldFetchExplicitlyFromAllPartitions(final String groupProtocol, final String testName, final TestInfo testInfo) {
|
||||
setup(groupProtocol, testInfo);
|
||||
final KeyQuery<Integer, ValueAndTimestamp<Integer>> query = KeyQuery.withKey(1);
|
||||
final Set<Integer> partitions = Set.of(0, 1);
|
||||
final StateQueryRequest<ValueAndTimestamp<Integer>> request =
|
||||
|
@ -290,8 +306,10 @@ public class IQv2IntegrationTest {
|
|||
assertThat(result.getPartitionResults().keySet(), equalTo(partitions));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldNotRequireQueryHandler(final TestInfo testInfo) {
|
||||
@ParameterizedTest(name = "{1}")
|
||||
@MethodSource("groupProtocolParameters")
|
||||
public void shouldNotRequireQueryHandler(final String groupProtocol, final String testName, final TestInfo testInfo) {
|
||||
setup(groupProtocol, testInfo);
|
||||
final KeyQuery<Integer, ValueAndTimestamp<Integer>> query = KeyQuery.withKey(1);
|
||||
final int partition = 1;
|
||||
final Set<Integer> partitions = singleton(partition);
|
||||
|
@ -423,8 +441,11 @@ public class IQv2IntegrationTest {
|
|||
);
|
||||
|
||||
// Discard the basic streams and replace with test-specific topology
|
||||
if (kafkaStreams != null) {
|
||||
kafkaStreams.close();
|
||||
}
|
||||
final String safeTestName = safeUniqueTestName(testInfo);
|
||||
this.groupProtocol = groupProtocol;
|
||||
kafkaStreams = new KafkaStreams(builder.build(), streamsConfiguration(safeTestName));
|
||||
kafkaStreams.cleanUp();
|
||||
|
||||
|
@ -446,7 +467,7 @@ public class IQv2IntegrationTest {
|
|||
private Properties streamsConfiguration(final String safeTestName) {
|
||||
final Properties config = new Properties();
|
||||
config.put(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG, StreamsConfig.OPTIMIZE);
|
||||
config.put(StreamsConfig.APPLICATION_ID_CONFIG, "app-" + safeTestName);
|
||||
config.put(StreamsConfig.APPLICATION_ID_CONFIG, "app-" + safeTestName + "-" + groupProtocol);
|
||||
config.put(StreamsConfig.APPLICATION_SERVER_CONFIG, "localhost:" + (++port));
|
||||
config.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
||||
config.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
|
||||
|
@ -458,6 +479,14 @@ public class IQv2IntegrationTest {
|
|||
config.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 1000);
|
||||
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
||||
config.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1);
|
||||
config.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, groupProtocol);
|
||||
return config;
|
||||
}
|
||||
|
||||
private static Stream<Arguments> groupProtocolParameters() {
|
||||
return Stream.of(
|
||||
Arguments.of("classic", "CLASSIC protocol"),
|
||||
Arguments.of("streams", "STREAMS protocol")
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -360,7 +360,9 @@ public class IQv2StoreIntegrationTest {
|
|||
for (final boolean logEnabled : Arrays.asList(true, false)) {
|
||||
for (final StoresToTest toTest : StoresToTest.values()) {
|
||||
for (final String kind : Arrays.asList("DSL", "PAPI")) {
|
||||
values.add(Arguments.of(cacheEnabled, logEnabled, toTest.name(), kind));
|
||||
for (final String groupProtocol : Arrays.asList("classic", "streams")) {
|
||||
values.add(Arguments.of(cacheEnabled, logEnabled, toTest.name(), kind, groupProtocol));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -426,13 +428,14 @@ public class IQv2StoreIntegrationTest {
|
|||
));
|
||||
}
|
||||
|
||||
public void setup(final boolean cache, final boolean log, final StoresToTest storeToTest, final String kind) {
|
||||
public void setup(final boolean cache, final boolean log, final StoresToTest storeToTest, final String kind, final String groupProtocol) {
|
||||
final StoreSupplier<?> supplier = storeToTest.supplier();
|
||||
final Properties streamsConfig = streamsConfiguration(
|
||||
cache,
|
||||
log,
|
||||
storeToTest.name(),
|
||||
kind
|
||||
kind,
|
||||
groupProtocol
|
||||
);
|
||||
|
||||
final StreamsBuilder builder = new StreamsBuilder();
|
||||
|
@ -765,8 +768,8 @@ public class IQv2StoreIntegrationTest {
|
|||
|
||||
@ParameterizedTest
|
||||
@MethodSource("data")
|
||||
public void verifyStore(final boolean cache, final boolean log, final StoresToTest storeToTest, final String kind) {
|
||||
setup(cache, log, storeToTest, kind);
|
||||
public void verifyStore(final boolean cache, final boolean log, final StoresToTest storeToTest, final String kind, final String groupProtocol) {
|
||||
setup(cache, log, storeToTest, kind, groupProtocol);
|
||||
try {
|
||||
if (storeToTest.global()) {
|
||||
// See KAFKA-13523
|
||||
|
@ -2030,10 +2033,10 @@ public class IQv2StoreIntegrationTest {
|
|||
}
|
||||
|
||||
private static Properties streamsConfiguration(final boolean cache, final boolean log,
|
||||
final String supplier, final String kind) {
|
||||
final String supplier, final String kind, final String groupProtocol) {
|
||||
final String safeTestName =
|
||||
IQv2StoreIntegrationTest.class.getName() + "-" + cache + "-" + log + "-" + supplier
|
||||
+ "-" + kind + "-" + RANDOM.nextInt();
|
||||
+ "-" + kind + "-" + groupProtocol + "-" + RANDOM.nextInt();
|
||||
final Properties config = new Properties();
|
||||
config.put(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG, StreamsConfig.OPTIMIZE);
|
||||
config.put(StreamsConfig.APPLICATION_ID_CONFIG, "app-" + safeTestName);
|
||||
|
@ -2048,6 +2051,7 @@ public class IQv2StoreIntegrationTest {
|
|||
config.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 1000);
|
||||
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
|
||||
config.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1);
|
||||
config.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, groupProtocol);
|
||||
return config;
|
||||
}
|
||||
}
|
|
@ -47,7 +47,10 @@ import org.junit.jupiter.api.AfterEach;
|
|||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Tag;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.TestInfo;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.Arguments;
|
||||
import org.junit.jupiter.params.provider.MethodSource;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.time.Instant;
|
||||
|
@ -57,7 +60,9 @@ import java.util.Map;
|
|||
import java.util.Map.Entry;
|
||||
import java.util.Optional;
|
||||
import java.util.Properties;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static org.apache.kafka.streams.utils.TestUtils.safeUniqueTestName;
|
||||
import static org.hamcrest.CoreMatchers.nullValue;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.hamcrest.Matchers.empty;
|
||||
|
@ -83,16 +88,25 @@ public class IQv2VersionedStoreIntegrationTest {
|
|||
private static final Long[] RECORD_TIMESTAMPS = {BASE_TIMESTAMP_LONG, BASE_TIMESTAMP_LONG + 10, BASE_TIMESTAMP_LONG + 20, BASE_TIMESTAMP_LONG + 30};
|
||||
private static final int RECORD_NUMBER = RECORD_VALUES.length;
|
||||
private static final int LAST_INDEX = RECORD_NUMBER - 1;
|
||||
private static final Position INPUT_POSITION = Position.emptyPosition();
|
||||
private Position inputPosition;
|
||||
|
||||
public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(NUM_BROKERS, Utils.mkProperties(Collections.singletonMap("auto.create.topics.enable", "true")));
|
||||
|
||||
private KafkaStreams kafkaStreams;
|
||||
private String groupProtocol;
|
||||
|
||||
@BeforeAll
|
||||
public static void before() throws Exception {
|
||||
public static void beforeAll() throws Exception {
|
||||
CLUSTER.start();
|
||||
}
|
||||
|
||||
@BeforeEach
|
||||
public void beforeEach() throws Exception {
|
||||
// Delete and recreate the topic to ensure clean state for each test
|
||||
CLUSTER.deleteTopic(INPUT_TOPIC_NAME);
|
||||
CLUSTER.createTopic(INPUT_TOPIC_NAME, 1, 1);
|
||||
|
||||
// Set up fresh test data
|
||||
final Properties producerProps = new Properties();
|
||||
producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
||||
producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, IntegerSerializer.class);
|
||||
|
@ -103,19 +117,21 @@ public class IQv2VersionedStoreIntegrationTest {
|
|||
producer.send(new ProducerRecord<>(INPUT_TOPIC_NAME, 0, RECORD_TIMESTAMPS[2], RECORD_KEY, RECORD_VALUES[2])).get();
|
||||
producer.send(new ProducerRecord<>(INPUT_TOPIC_NAME, 0, RECORD_TIMESTAMPS[3], RECORD_KEY, RECORD_VALUES[3])).get();
|
||||
}
|
||||
INPUT_POSITION.withComponent(INPUT_TOPIC_NAME, 0, 3);
|
||||
inputPosition = Position.emptyPosition().withComponent(INPUT_TOPIC_NAME, 0, 3);
|
||||
}
|
||||
|
||||
@BeforeEach
|
||||
public void beforeTest() {
|
||||
private void setup(final String groupProtocol, final TestInfo testInfo) {
|
||||
this.groupProtocol = groupProtocol;
|
||||
final StreamsBuilder builder = new StreamsBuilder();
|
||||
builder.table(INPUT_TOPIC_NAME,
|
||||
Materialized.as(Stores.persistentVersionedKeyValueStore(STORE_NAME, HISTORY_RETENTION, SEGMENT_INTERVAL)));
|
||||
final Properties configs = new Properties();
|
||||
configs.put(StreamsConfig.APPLICATION_ID_CONFIG, "app");
|
||||
final String safeTestName = safeUniqueTestName(testInfo);
|
||||
configs.put(StreamsConfig.APPLICATION_ID_CONFIG, "app-" + safeTestName);
|
||||
configs.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
||||
configs.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.IntegerSerde.class.getName());
|
||||
configs.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.IntegerSerde.class.getName());
|
||||
configs.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, groupProtocol);
|
||||
kafkaStreams = IntegrationTestUtils.getStartedStreams(configs, builder, true);
|
||||
}
|
||||
|
||||
|
@ -132,8 +148,19 @@ public class IQv2VersionedStoreIntegrationTest {
|
|||
CLUSTER.stop();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void verifyStore() {
|
||||
private static Stream<Arguments> groupProtocolParameters() {
|
||||
return Stream.of(
|
||||
Arguments.of("classic", "CLASSIC protocol"),
|
||||
Arguments.of("streams", "STREAMS protocol")
|
||||
);
|
||||
}
|
||||
|
||||
@ParameterizedTest(name = "{1}")
|
||||
@MethodSource("groupProtocolParameters")
|
||||
public void verifyStore(final String groupProtocol, final String testName, final TestInfo testInfo) throws Exception {
|
||||
// Set up streams
|
||||
setup(groupProtocol, testInfo);
|
||||
|
||||
/* Test Versioned Key Queries */
|
||||
// retrieve the latest value
|
||||
shouldHandleVersionedKeyQuery(Optional.empty(), RECORD_VALUES[3], RECORD_TIMESTAMPS[3], Optional.empty());
|
||||
|
@ -255,7 +282,10 @@ public class IQv2VersionedStoreIntegrationTest {
|
|||
private void shouldHandleRaceCondition() {
|
||||
final MultiVersionedKeyQuery<Integer, Integer> query = defineQuery(RECORD_KEY, Optional.empty(), Optional.empty(), ResultOrder.ANY);
|
||||
|
||||
final Map<Integer, QueryResult<VersionedRecordIterator<Integer>>> partitionResults = sendRequestAndReceiveResults(query, kafkaStreams);
|
||||
// For race condition test, we don't use position bounds since we're testing concurrent updates
|
||||
final StateQueryRequest<VersionedRecordIterator<Integer>> request = StateQueryRequest.inStore(STORE_NAME).withQuery(query);
|
||||
final StateQueryResult<VersionedRecordIterator<Integer>> result = IntegrationTestUtils.iqv2WaitForResult(kafkaStreams, request);
|
||||
final Map<Integer, QueryResult<VersionedRecordIterator<Integer>>> partitionResults = result.getPartitionResults();
|
||||
|
||||
// verify results in two steps
|
||||
for (final Entry<Integer, QueryResult<VersionedRecordIterator<Integer>>> partitionResultsEntry : partitionResults.entrySet()) {
|
||||
|
@ -327,14 +357,14 @@ public class IQv2VersionedStoreIntegrationTest {
|
|||
return query;
|
||||
}
|
||||
|
||||
private static Map<Integer, QueryResult<VersionedRecordIterator<Integer>>> sendRequestAndReceiveResults(final MultiVersionedKeyQuery<Integer, Integer> query, final KafkaStreams kafkaStreams) {
|
||||
final StateQueryRequest<VersionedRecordIterator<Integer>> request = StateQueryRequest.inStore(STORE_NAME).withQuery(query).withPositionBound(PositionBound.at(INPUT_POSITION));
|
||||
private Map<Integer, QueryResult<VersionedRecordIterator<Integer>>> sendRequestAndReceiveResults(final MultiVersionedKeyQuery<Integer, Integer> query, final KafkaStreams kafkaStreams) {
|
||||
final StateQueryRequest<VersionedRecordIterator<Integer>> request = StateQueryRequest.inStore(STORE_NAME).withQuery(query).withPositionBound(PositionBound.at(inputPosition));
|
||||
final StateQueryResult<VersionedRecordIterator<Integer>> result = IntegrationTestUtils.iqv2WaitForResult(kafkaStreams, request);
|
||||
return result.getPartitionResults();
|
||||
}
|
||||
|
||||
private static QueryResult<VersionedRecord<Integer>> sendRequestAndReceiveResults(final VersionedKeyQuery<Integer, Integer> query, final KafkaStreams kafkaStreams) {
|
||||
final StateQueryRequest<VersionedRecord<Integer>> request = StateQueryRequest.inStore(STORE_NAME).withQuery(query).withPositionBound(PositionBound.at(INPUT_POSITION));
|
||||
private QueryResult<VersionedRecord<Integer>> sendRequestAndReceiveResults(final VersionedKeyQuery<Integer, Integer> query, final KafkaStreams kafkaStreams) {
|
||||
final StateQueryRequest<VersionedRecord<Integer>> request = StateQueryRequest.inStore(STORE_NAME).withQuery(query).withPositionBound(PositionBound.at(inputPosition));
|
||||
final StateQueryResult<VersionedRecord<Integer>> result = IntegrationTestUtils.iqv2WaitForResult(kafkaStreams, request);
|
||||
return result.getOnlyPartitionResult();
|
||||
}
|
||||
|
@ -352,7 +382,7 @@ public class IQv2VersionedStoreIntegrationTest {
|
|||
/**
|
||||
* This method inserts a new value (999999) for the key in the oldest timestamp (RECORD_TIMESTAMPS[0]).
|
||||
*/
|
||||
private static void updateRecordValue() {
|
||||
private void updateRecordValue() {
|
||||
// update the record value at RECORD_TIMESTAMPS[0]
|
||||
final Properties producerProps = new Properties();
|
||||
producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
|
||||
|
@ -361,8 +391,9 @@ public class IQv2VersionedStoreIntegrationTest {
|
|||
try (final KafkaProducer<Integer, Integer> producer = new KafkaProducer<>(producerProps)) {
|
||||
producer.send(new ProducerRecord<>(INPUT_TOPIC_NAME, 0, RECORD_TIMESTAMPS[0], RECORD_KEY, 999999));
|
||||
}
|
||||
INPUT_POSITION.withComponent(INPUT_TOPIC_NAME, 0, 4);
|
||||
assertThat(INPUT_POSITION, equalTo(Position.emptyPosition().withComponent(INPUT_TOPIC_NAME, 0, 4)));
|
||||
|
||||
inputPosition = inputPosition.withComponent(INPUT_TOPIC_NAME, 0, 4);
|
||||
assertThat(inputPosition, equalTo(Position.emptyPosition().withComponent(INPUT_TOPIC_NAME, 0, 4)));
|
||||
|
||||
// make sure that the new value is picked up by the store
|
||||
final Properties consumerProps = new Properties();
|
||||
|
|
|
@ -1122,15 +1122,15 @@ public class KStreamAggregationIntegrationTest {
|
|||
final String[] args = new String[] {
|
||||
"--bootstrap-server", CLUSTER.bootstrapServers(),
|
||||
"--from-beginning",
|
||||
"--property", "print.key=true",
|
||||
"--property", "print.timestamp=" + printTimestamp,
|
||||
"--formatter-property", "print.key=true",
|
||||
"--formatter-property", "print.timestamp=" + printTimestamp,
|
||||
"--topic", outputTopic,
|
||||
"--max-messages", String.valueOf(numMessages),
|
||||
"--property", "key.deserializer=" + keyDeserializer.getClass().getName(),
|
||||
"--property", "value.deserializer=" + valueDeserializer.getClass().getName(),
|
||||
"--property", "key.separator=" + keySeparator,
|
||||
"--property", "key.deserializer." + TimeWindowedDeserializer.WINDOWED_INNER_DESERIALIZER_CLASS + "=" + Serdes.serdeFrom(innerClass).getClass().getName(),
|
||||
"--property", "key.deserializer.window.size.ms=500",
|
||||
"--formatter-property", "key.deserializer=" + keyDeserializer.getClass().getName(),
|
||||
"--formatter-property", "value.deserializer=" + valueDeserializer.getClass().getName(),
|
||||
"--formatter-property", "key.separator=" + keySeparator,
|
||||
"--formatter-property", "key.deserializer." + TimeWindowedDeserializer.WINDOWED_INNER_DESERIALIZER_CLASS + "=" + Serdes.serdeFrom(innerClass).getClass().getName(),
|
||||
"--formatter-property", "key.deserializer.window.size.ms=500",
|
||||
};
|
||||
|
||||
ConsoleConsumer.run(new ConsoleConsumerOptions(args));
|
||||
|
|
|
@ -89,6 +89,7 @@ public class DefaultStreamsRebalanceListener implements StreamsRebalanceListener
|
|||
taskManager.handleAssignment(activeTasksWithPartitions, standbyTasksWithPartitions);
|
||||
streamThread.setState(StreamThread.State.PARTITIONS_ASSIGNED);
|
||||
taskManager.handleRebalanceComplete();
|
||||
streamsRebalanceData.setReconciledAssignment(assignment);
|
||||
} catch (final Exception exception) {
|
||||
return Optional.of(exception);
|
||||
}
|
||||
|
@ -99,6 +100,7 @@ public class DefaultStreamsRebalanceListener implements StreamsRebalanceListener
|
|||
public Optional<Exception> onAllTasksLost() {
|
||||
try {
|
||||
taskManager.handleLostAll();
|
||||
streamsRebalanceData.setReconciledAssignment(StreamsRebalanceData.Assignment.EMPTY);
|
||||
} catch (final Exception exception) {
|
||||
return Optional.of(exception);
|
||||
}
|
||||
|
|
|
@ -118,10 +118,8 @@ public class DefaultStreamsRebalanceListenerTest {
|
|||
|
||||
@Test
|
||||
void testOnTasksAssigned() {
|
||||
createRebalanceListenerWithRebalanceData(new StreamsRebalanceData(
|
||||
UUID.randomUUID(),
|
||||
Optional.empty(),
|
||||
Map.of(
|
||||
final StreamsRebalanceData streamsRebalanceData = mock(StreamsRebalanceData.class);
|
||||
when(streamsRebalanceData.subtopologies()).thenReturn(Map.of(
|
||||
"1",
|
||||
new StreamsRebalanceData.Subtopology(
|
||||
Set.of("source1"),
|
||||
|
@ -146,21 +144,20 @@ public class DefaultStreamsRebalanceListenerTest {
|
|||
Map.of(),
|
||||
Set.of()
|
||||
)
|
||||
),
|
||||
Map.of()
|
||||
));
|
||||
createRebalanceListenerWithRebalanceData(streamsRebalanceData);
|
||||
|
||||
final Optional<Exception> result = defaultStreamsRebalanceListener.onTasksAssigned(
|
||||
new StreamsRebalanceData.Assignment(
|
||||
final StreamsRebalanceData.Assignment assignment = new StreamsRebalanceData.Assignment(
|
||||
Set.of(new StreamsRebalanceData.TaskId("1", 0)),
|
||||
Set.of(new StreamsRebalanceData.TaskId("2", 0)),
|
||||
Set.of(new StreamsRebalanceData.TaskId("3", 0))
|
||||
)
|
||||
);
|
||||
|
||||
final Optional<Exception> result = defaultStreamsRebalanceListener.onTasksAssigned(assignment);
|
||||
|
||||
assertTrue(result.isEmpty());
|
||||
|
||||
final InOrder inOrder = inOrder(taskManager, streamThread);
|
||||
final InOrder inOrder = inOrder(taskManager, streamThread, streamsRebalanceData);
|
||||
inOrder.verify(taskManager).handleAssignment(
|
||||
Map.of(new TaskId(1, 0), Set.of(new TopicPartition("source1", 0), new TopicPartition("repartition1", 0))),
|
||||
Map.of(
|
||||
|
@ -170,6 +167,7 @@ public class DefaultStreamsRebalanceListenerTest {
|
|||
);
|
||||
inOrder.verify(streamThread).setState(StreamThread.State.PARTITIONS_ASSIGNED);
|
||||
inOrder.verify(taskManager).handleRebalanceComplete();
|
||||
inOrder.verify(streamsRebalanceData).setReconciledAssignment(assignment);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -177,21 +175,32 @@ public class DefaultStreamsRebalanceListenerTest {
|
|||
final Exception exception = new RuntimeException("sample exception");
|
||||
doThrow(exception).when(taskManager).handleAssignment(any(), any());
|
||||
|
||||
createRebalanceListenerWithRebalanceData(new StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of()));
|
||||
final Optional<Exception> result = defaultStreamsRebalanceListener.onTasksAssigned(new StreamsRebalanceData.Assignment(Set.of(), Set.of(), Set.of()));
|
||||
assertTrue(defaultStreamsRebalanceListener.onAllTasksLost().isEmpty());
|
||||
final StreamsRebalanceData streamsRebalanceData = mock(StreamsRebalanceData.class);
|
||||
when(streamsRebalanceData.subtopologies()).thenReturn(Map.of());
|
||||
createRebalanceListenerWithRebalanceData(streamsRebalanceData);
|
||||
|
||||
final Optional<Exception> result = defaultStreamsRebalanceListener.onTasksAssigned(
|
||||
new StreamsRebalanceData.Assignment(Set.of(), Set.of(), Set.of())
|
||||
);
|
||||
assertTrue(result.isPresent());
|
||||
assertEquals(exception, result.get());
|
||||
verify(taskManager).handleLostAll();
|
||||
verify(taskManager).handleAssignment(any(), any());
|
||||
verify(streamThread, never()).setState(StreamThread.State.PARTITIONS_ASSIGNED);
|
||||
verify(taskManager, never()).handleRebalanceComplete();
|
||||
verify(streamsRebalanceData, never()).setReconciledAssignment(any());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testOnAllTasksLost() {
|
||||
createRebalanceListenerWithRebalanceData(new StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of()));
|
||||
final StreamsRebalanceData streamsRebalanceData = mock(StreamsRebalanceData.class);
|
||||
when(streamsRebalanceData.subtopologies()).thenReturn(Map.of());
|
||||
createRebalanceListenerWithRebalanceData(streamsRebalanceData);
|
||||
|
||||
assertTrue(defaultStreamsRebalanceListener.onAllTasksLost().isEmpty());
|
||||
verify(taskManager).handleLostAll();
|
||||
|
||||
final InOrder inOrder = inOrder(taskManager, streamsRebalanceData);
|
||||
inOrder.verify(taskManager).handleLostAll();
|
||||
inOrder.verify(streamsRebalanceData).setReconciledAssignment(StreamsRebalanceData.Assignment.EMPTY);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -199,10 +208,13 @@ public class DefaultStreamsRebalanceListenerTest {
|
|||
final Exception exception = new RuntimeException("sample exception");
|
||||
doThrow(exception).when(taskManager).handleLostAll();
|
||||
|
||||
createRebalanceListenerWithRebalanceData(new StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of()));
|
||||
final StreamsRebalanceData streamsRebalanceData = mock(StreamsRebalanceData.class);
|
||||
when(streamsRebalanceData.subtopologies()).thenReturn(Map.of());
|
||||
createRebalanceListenerWithRebalanceData(streamsRebalanceData);
|
||||
final Optional<Exception> result = defaultStreamsRebalanceListener.onAllTasksLost();
|
||||
assertTrue(result.isPresent());
|
||||
assertEquals(exception, result.get());
|
||||
verify(taskManager).handleLostAll();
|
||||
verify(streamsRebalanceData, never()).setReconciledAssignment(any());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -152,6 +152,7 @@ public class TaskManagerTest {
|
|||
private final TopicPartition t1p1 = new TopicPartition(topic1, 1);
|
||||
private final TopicPartition t2p2 = new TopicPartition(topic2, 1);
|
||||
private final TopicPartition t1p1changelog = new TopicPartition("changelog", 1);
|
||||
private final TopicPartition t1p1changelog2 = new TopicPartition("changelog2", 1);
|
||||
private final Set<TopicPartition> taskId01Partitions = Set.of(t1p1);
|
||||
private final Set<TopicPartition> taskId01ChangelogPartitions = Set.of(t1p1changelog);
|
||||
private final Map<TaskId, Set<TopicPartition>> taskId01Assignment = singletonMap(taskId01, taskId01Partitions);
|
||||
|
@ -218,6 +219,10 @@ public class TaskManagerTest {
|
|||
taskManager = setUpTaskManager(StreamsConfigUtils.ProcessingMode.AT_LEAST_ONCE, null, false);
|
||||
}
|
||||
|
||||
private TaskManager setUpTaskManager(final ProcessingMode processingMode, final TasksRegistry tasks) {
|
||||
return setUpTaskManager(processingMode, tasks, false);
|
||||
}
|
||||
|
||||
private TaskManager setUpTaskManager(final ProcessingMode processingMode, final boolean stateUpdaterEnabled) {
|
||||
return setUpTaskManager(processingMode, null, stateUpdaterEnabled, false);
|
||||
}
|
||||
|
@ -249,52 +254,6 @@ public class TaskManagerTest {
|
|||
return taskManager;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldClassifyExistingTasksWithoutStateUpdater() {
|
||||
final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, false);
|
||||
final Map<TaskId, Set<TopicPartition>> runningActiveTasks = mkMap(mkEntry(taskId01, Set.of(t1p1)));
|
||||
final Map<TaskId, Set<TopicPartition>> standbyTasks = mkMap(mkEntry(taskId02, Set.of(t2p2)));
|
||||
final Map<TaskId, Set<TopicPartition>> restoringActiveTasks = mkMap(mkEntry(taskId03, Set.of(t1p3)));
|
||||
final Map<TaskId, Set<TopicPartition>> activeTasks = new HashMap<>(runningActiveTasks);
|
||||
activeTasks.putAll(restoringActiveTasks);
|
||||
handleAssignment(runningActiveTasks, standbyTasks, restoringActiveTasks);
|
||||
|
||||
taskManager.handleAssignment(activeTasks, standbyTasks);
|
||||
|
||||
verifyNoInteractions(stateUpdater);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldNotUpdateExistingStandbyTaskIfStandbyIsReassignedWithSameInputPartitionWithoutStateUpdater() {
|
||||
final StandbyTask standbyTask = standbyTask(taskId03, taskId03ChangelogPartitions)
|
||||
.inState(State.RUNNING)
|
||||
.withInputPartitions(taskId03Partitions).build();
|
||||
updateExistingStandbyTaskIfStandbyIsReassignedWithoutStateUpdater(standbyTask, taskId03Partitions);
|
||||
verify(standbyTask, never()).updateInputPartitions(eq(taskId03Partitions), any());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldUpdateExistingStandbyTaskIfStandbyIsReassignedWithDifferentInputPartitionWithoutStateUpdater() {
|
||||
final StandbyTask standbyTask = standbyTask(taskId03, taskId03ChangelogPartitions)
|
||||
.inState(State.RUNNING)
|
||||
.withInputPartitions(taskId03Partitions).build();
|
||||
updateExistingStandbyTaskIfStandbyIsReassignedWithoutStateUpdater(standbyTask, taskId04Partitions);
|
||||
verify(standbyTask).updateInputPartitions(eq(taskId04Partitions), any());
|
||||
}
|
||||
|
||||
private void updateExistingStandbyTaskIfStandbyIsReassignedWithoutStateUpdater(final Task standbyTask,
|
||||
final Set<TopicPartition> newInputPartition) {
|
||||
final TasksRegistry tasks = mock(TasksRegistry.class);
|
||||
when(tasks.allTasks()).thenReturn(Set.of(standbyTask));
|
||||
final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, false);
|
||||
|
||||
taskManager.handleAssignment(
|
||||
Collections.emptyMap(),
|
||||
mkMap(mkEntry(standbyTask.id(), newInputPartition))
|
||||
);
|
||||
|
||||
verify(standbyTask).resume();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldLockAllTasksOnCorruptionWithProcessingThreads() {
|
||||
|
@ -1853,14 +1812,20 @@ public class TaskManagerTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void shouldReportLatestOffsetAsOffsetSumForRunningTask() throws Exception {
|
||||
final Map<TopicPartition, Long> changelogOffsets = mkMap(
|
||||
mkEntry(new TopicPartition("changelog", 0), Task.LATEST_OFFSET),
|
||||
mkEntry(new TopicPartition("changelog", 1), Task.LATEST_OFFSET)
|
||||
);
|
||||
final Map<TaskId, Long> expectedOffsetSums = mkMap(mkEntry(taskId00, Task.LATEST_OFFSET));
|
||||
public void shouldComputeOffsetSumForRunningStatefulTask() {
|
||||
final StreamTask runningStatefulTask = statefulTask(taskId00, taskId00ChangelogPartitions)
|
||||
.inState(State.RUNNING).build();
|
||||
final long changelogOffsetOfRunningTask = Task.LATEST_OFFSET;
|
||||
when(runningStatefulTask.changelogOffsets())
|
||||
.thenReturn(mkMap(mkEntry(t1p0changelog, changelogOffsetOfRunningTask)));
|
||||
final TasksRegistry tasks = mock(TasksRegistry.class);
|
||||
final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks);
|
||||
when(tasks.allTasksPerId()).thenReturn(mkMap(mkEntry(taskId00, runningStatefulTask)));
|
||||
|
||||
computeOffsetSumAndVerify(changelogOffsets, expectedOffsetSums);
|
||||
assertThat(
|
||||
taskManager.taskOffsetSums(),
|
||||
is(mkMap(mkEntry(taskId00, changelogOffsetOfRunningTask)))
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -1911,14 +1876,14 @@ public class TaskManagerTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void shouldComputeOffsetSumForRunningStatefulTaskAndRestoringTaskWithStateUpdater() {
|
||||
public void shouldComputeOffsetSumForRunningStatefulTaskAndRestoringTask() {
|
||||
final StreamTask runningStatefulTask = statefulTask(taskId00, taskId00ChangelogPartitions)
|
||||
.inState(State.RUNNING).build();
|
||||
final StreamTask restoringStatefulTask = statefulTask(taskId01, taskId01ChangelogPartitions)
|
||||
.inState(State.RESTORING).build();
|
||||
final StandbyTask restoringStandbyTask = standbyTask(taskId02, taskId02ChangelogPartitions)
|
||||
.inState(State.RUNNING).build();
|
||||
final long changelogOffsetOfRunningTask = 42L;
|
||||
final long changelogOffsetOfRunningTask = Task.LATEST_OFFSET;
|
||||
final long changelogOffsetOfRestoringStatefulTask = 24L;
|
||||
final long changelogOffsetOfRestoringStandbyTask = 84L;
|
||||
when(runningStatefulTask.changelogOffsets())
|
||||
|
@ -1943,14 +1908,26 @@ public class TaskManagerTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void shouldSkipUnknownOffsetsWhenComputingOffsetSum() throws Exception {
|
||||
final Map<TopicPartition, Long> changelogOffsets = mkMap(
|
||||
mkEntry(new TopicPartition("changelog", 0), OffsetCheckpoint.OFFSET_UNKNOWN),
|
||||
mkEntry(new TopicPartition("changelog", 1), 10L)
|
||||
);
|
||||
final Map<TaskId, Long> expectedOffsetSums = mkMap(mkEntry(taskId00, 10L));
|
||||
public void shouldSkipUnknownOffsetsWhenComputingOffsetSum() {
|
||||
final StreamTask restoringStatefulTask = statefulTask(taskId01, taskId01ChangelogPartitions)
|
||||
.inState(State.RESTORING).build();
|
||||
final long changelogOffsetOfRestoringStandbyTask = 84L;
|
||||
when(restoringStatefulTask.changelogOffsets())
|
||||
.thenReturn(mkMap(
|
||||
mkEntry(t1p1changelog, changelogOffsetOfRestoringStandbyTask),
|
||||
mkEntry(t1p1changelog2, OffsetCheckpoint.OFFSET_UNKNOWN)
|
||||
));
|
||||
final TasksRegistry tasks = mock(TasksRegistry.class);
|
||||
final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks);
|
||||
when(tasks.allTasksPerId()).thenReturn(mkMap(mkEntry(taskId01, restoringStatefulTask)));
|
||||
when(stateUpdater.tasks()).thenReturn(Set.of(restoringStatefulTask));
|
||||
|
||||
computeOffsetSumAndVerify(changelogOffsets, expectedOffsetSums);
|
||||
assertThat(
|
||||
taskManager.taskOffsetSums(),
|
||||
is(mkMap(
|
||||
mkEntry(taskId01, changelogOffsetOfRestoringStandbyTask)
|
||||
))
|
||||
);
|
||||
}
|
||||
|
||||
private void computeOffsetSumAndVerify(final Map<TopicPartition, Long> changelogOffsets,
|
||||
|
|
|
@ -88,7 +88,7 @@ class ConsoleConsumer(KafkaPathResolverMixin, JmxMixin, BackgroundThreadService)
|
|||
jaas_override_variables A dict of variables to be used in the jaas.conf template file
|
||||
kafka_opts_override Override parameters of the KAFKA_OPTS environment variable
|
||||
client_prop_file_override Override client.properties file used by the consumer
|
||||
consumer_properties A dict of values to pass in as --consumer-property key=value
|
||||
consumer_properties A dict of values to pass in as --command-property key=value. For versions older than KAFKA_4_2_0, these will be passed as --consumer-property key=value
|
||||
"""
|
||||
JmxMixin.__init__(self, num_nodes=num_nodes, jmx_object_names=jmx_object_names, jmx_attributes=(jmx_attributes or []),
|
||||
root=ConsoleConsumer.PERSISTENT_ROOT)
|
||||
|
@ -163,8 +163,11 @@ class ConsoleConsumer(KafkaPathResolverMixin, JmxMixin, BackgroundThreadService)
|
|||
"export KAFKA_LOG4J_OPTS=\"%(log4j_param)s%(log4j_config)s\"; " \
|
||||
"export KAFKA_OPTS=%(kafka_opts)s; " \
|
||||
"%(console_consumer)s " \
|
||||
"--topic %(topic)s " \
|
||||
"--consumer.config %(config_file)s " % args
|
||||
"--topic %(topic)s " % args
|
||||
|
||||
version = get_version(node)
|
||||
command_config_arg = "--command-config" if version.supports_command_config() else "--consumer.config"
|
||||
cmd += "%s %s" % (command_config_arg, args['config_file'])
|
||||
cmd += " --bootstrap-server %(broker_list)s" % args
|
||||
cmd += " --isolation-level %s" % self.isolation_level
|
||||
|
||||
|
@ -176,14 +179,15 @@ class ConsoleConsumer(KafkaPathResolverMixin, JmxMixin, BackgroundThreadService)
|
|||
# This will be added in the properties file instead
|
||||
cmd += " --timeout-ms %s" % self.consumer_timeout_ms
|
||||
|
||||
formatter_property_arg = "--formatter-property" if version.supports_formatter_property() else "--property"
|
||||
if self.print_timestamp:
|
||||
cmd += " --property print.timestamp=true"
|
||||
cmd += " %s print.timestamp=true" % formatter_property_arg
|
||||
|
||||
if self.print_key:
|
||||
cmd += " --property print.key=true"
|
||||
cmd += " %s print.key=true" % formatter_property_arg
|
||||
|
||||
if self.print_partition:
|
||||
cmd += " --property print.partition=true"
|
||||
cmd += " %s print.partition=true" % formatter_property_arg
|
||||
|
||||
# LoggingMessageFormatter was introduced after 0.9
|
||||
if node.version > LATEST_3_7:
|
||||
|
@ -194,9 +198,10 @@ class ConsoleConsumer(KafkaPathResolverMixin, JmxMixin, BackgroundThreadService)
|
|||
if self.enable_systest_events:
|
||||
cmd += " --enable-systest-events"
|
||||
|
||||
command_property_arg = "--command-property" if version.supports_command_property() else "--consumer-property"
|
||||
if self.consumer_properties is not None:
|
||||
for k, v in self.consumer_properties.items():
|
||||
cmd += " --consumer-property %s=%s" % (k, v)
|
||||
cmd += " %s %s=%s" % (command_property_arg, k, v)
|
||||
|
||||
cmd += " 2>> %(stderr)s | tee -a %(stdout)s &" % args
|
||||
return cmd
|
||||
|
|
|
@ -21,7 +21,7 @@ from ducktape.utils.util import wait_until
|
|||
|
||||
from kafkatest.directory_layout.kafka_path import KafkaPathResolverMixin
|
||||
from kafkatest.services.monitor.jmx import JmxMixin
|
||||
from kafkatest.version import DEV_BRANCH, LATEST_4_1
|
||||
from kafkatest.version import DEV_BRANCH, LATEST_4_1, get_version
|
||||
from kafkatest.services.kafka.util import fix_opts_for_new_jvm, get_log4j_config_param, get_log4j_config_for_tools
|
||||
|
||||
"""
|
||||
|
@ -84,7 +84,7 @@ class ConsoleShareConsumer(KafkaPathResolverMixin, JmxMixin, BackgroundThreadSer
|
|||
jaas_override_variables A dict of variables to be used in the jaas.conf template file
|
||||
kafka_opts_override Override parameters of the KAFKA_OPTS environment variable
|
||||
client_prop_file_override Override client.properties file used by the consumer
|
||||
share_consumer_properties A dict of values to pass in as --consumer-property key=value
|
||||
share_consumer_properties A dict of values to pass in as --command-property key=value. For versions older than KAFKA_4_2_0, these will be passed as --consumer-property key=value
|
||||
"""
|
||||
JmxMixin.__init__(self, num_nodes=num_nodes, jmx_object_names=jmx_object_names, jmx_attributes=(jmx_attributes or []),
|
||||
root=ConsoleShareConsumer.PERSISTENT_ROOT)
|
||||
|
@ -156,31 +156,36 @@ class ConsoleShareConsumer(KafkaPathResolverMixin, JmxMixin, BackgroundThreadSer
|
|||
"export KAFKA_LOG4J_OPTS=\"%(log4j_param)s%(log4j_config)s\"; " \
|
||||
"export KAFKA_OPTS=%(kafka_opts)s; " \
|
||||
"%(console_share_consumer)s " \
|
||||
"--topic %(topic)s " \
|
||||
"--consumer-config %(config_file)s " % args
|
||||
"--topic %(topic)s " % args
|
||||
|
||||
version = get_version(node)
|
||||
command_config_arg = "--command-config" if version.supports_command_config() else "--consumer-config"
|
||||
cmd += "%s %s" % (command_config_arg, args['config_file'])
|
||||
cmd += " --bootstrap-server %(broker_list)s" % args
|
||||
|
||||
if self.share_consumer_timeout_ms is not None:
|
||||
# This will be added in the properties file instead
|
||||
cmd += " --timeout-ms %s" % self.share_consumer_timeout_ms
|
||||
|
||||
formatter_property_arg = "--formatter-property" if version.supports_formatter_property() else "--property"
|
||||
if self.print_timestamp:
|
||||
cmd += " --property print.timestamp=true"
|
||||
cmd += " %s print.timestamp=true" % formatter_property_arg
|
||||
|
||||
if self.print_key:
|
||||
cmd += " --property print.key=true"
|
||||
cmd += " %s print.key=true" % formatter_property_arg
|
||||
|
||||
if self.print_partition:
|
||||
cmd += " --property print.partition=true"
|
||||
cmd += " %s print.partition=true" % formatter_property_arg
|
||||
|
||||
cmd += " --formatter org.apache.kafka.tools.consumer.LoggingMessageFormatter"
|
||||
|
||||
if self.enable_systest_events:
|
||||
cmd += " --enable-systest-events"
|
||||
|
||||
command_property_arg = "--command-property" if version.supports_command_property() else "--consumer-property"
|
||||
if self.share_consumer_properties is not None:
|
||||
for k, v in self.share_consumer_properties.items():
|
||||
cmd += " --consumer-property %s=%s" % (k, v)
|
||||
cmd += " %s %s=%s" % (command_property_arg, k, v)
|
||||
|
||||
cmd += " 2>> %(stderr)s | tee -a %(stdout)s &" % args
|
||||
return cmd
|
||||
|
|
|
@ -104,6 +104,20 @@ class KafkaVersion(LooseVersion):
|
|||
# - For older versions, continue using --producer.config or --consumer.config
|
||||
return self >= V_4_2_0
|
||||
|
||||
def supports_command_property(self):
|
||||
# According to KIP-1147, --producer-property and --consumer-property have been deprecated and will be removed in future versions
|
||||
# For backward compatibility, we select the configuration based on node version:
|
||||
# - For versions 4.2.0 and above, use --command-property
|
||||
# - For older versions, continue using --producer-property or --consumer-property
|
||||
return self >= V_4_2_0
|
||||
|
||||
def supports_formatter_property(self):
|
||||
# According to KIP-1147, --property has been deprecated and will be removed in future versions
|
||||
# For backward compatibility, we select the configuration based on node version:
|
||||
# - For versions 4.2.0 and above, use --formatter-property
|
||||
# - For older versions, continue using --property
|
||||
return self >= V_4_2_0
|
||||
|
||||
def get_version(node=None):
|
||||
"""Return the version attached to the given node.
|
||||
Default to DEV_BRANCH if node or node.version is undefined (aka None)
|
||||
|
|
|
@ -131,8 +131,12 @@ public class ConsoleProducer {
|
|||
private final OptionSpec<Integer> socketBufferSizeOpt;
|
||||
private final OptionSpec<String> propertyOpt;
|
||||
private final OptionSpec<String> readerConfigOpt;
|
||||
@Deprecated(since = "4.2", forRemoval = true)
|
||||
private final OptionSpec<String> producerPropertyOpt;
|
||||
private OptionSpec<String> commandPropertyOpt;
|
||||
@Deprecated(since = "4.2", forRemoval = true)
|
||||
private final OptionSpec<String> producerConfigOpt;
|
||||
private OptionSpec<String> commandConfigOpt;
|
||||
|
||||
public ConsoleProducerOptions(String[] args) {
|
||||
super(args);
|
||||
|
@ -250,11 +254,20 @@ public class ConsoleProducer {
|
|||
.withRequiredArg()
|
||||
.describedAs("config file")
|
||||
.ofType(String.class);
|
||||
producerPropertyOpt = parser.accepts("producer-property", "A mechanism to pass user-defined properties in the form key=value to the producer. ")
|
||||
producerPropertyOpt = parser.accepts("producer-property", "(DEPRECATED) Producer config properties in the form key=value. " +
|
||||
"This option will be removed in a future version. Use --command-property instead.")
|
||||
.withRequiredArg()
|
||||
.describedAs("producer_prop")
|
||||
.ofType(String.class);
|
||||
producerConfigOpt = parser.accepts("producer.config", "Producer config properties file. Note that " + producerPropertyOpt + " takes precedence over this config.")
|
||||
commandPropertyOpt = parser.accepts("command-property", "Producer config properties in the form key=value.")
|
||||
.withRequiredArg()
|
||||
.describedAs("producer_prop")
|
||||
.ofType(String.class);
|
||||
producerConfigOpt = parser.accepts("producer.config", "(DEPRECATED) Producer config properties file. Note that " + commandPropertyOpt + " takes precedence over this config. This option will be removed in a future version. Use --command-config instead.")
|
||||
.withRequiredArg()
|
||||
.describedAs("config file")
|
||||
.ofType(String.class);
|
||||
commandConfigOpt = parser.accepts("command-config", "Producer config properties file. Note that " + commandPropertyOpt + " takes precedence over this config.")
|
||||
.withRequiredArg()
|
||||
.describedAs("config file")
|
||||
.ofType(String.class);
|
||||
|
@ -273,6 +286,23 @@ public class ConsoleProducer {
|
|||
|
||||
CommandLineUtils.checkRequiredArgs(parser, options, topicOpt);
|
||||
|
||||
if (options.has(commandConfigOpt) && options.has(producerConfigOpt)) {
|
||||
CommandLineUtils.printUsageAndExit(parser, "Options --command-config and --producer.config cannot be specified together.");
|
||||
}
|
||||
if (options.has(commandPropertyOpt) && options.has(producerPropertyOpt)) {
|
||||
CommandLineUtils.printUsageAndExit(parser, "Options --command-property and --producer-property cannot be specified together.");
|
||||
}
|
||||
|
||||
if (options.has(producerPropertyOpt)) {
|
||||
System.out.println("Warning: --producer-property is deprecated and will be removed in a future version. Use --command-property instead.");
|
||||
commandPropertyOpt = producerPropertyOpt;
|
||||
}
|
||||
|
||||
if (options.has(producerConfigOpt)) {
|
||||
System.out.println("Warning: --producer.config is deprecated and will be removed in a future version. Use --command-config instead.");
|
||||
commandConfigOpt = producerConfigOpt;
|
||||
}
|
||||
|
||||
try {
|
||||
ToolsUtils.validateBootstrapServer(options.valueOf(bootstrapServerOpt));
|
||||
} catch (IllegalArgumentException e) {
|
||||
|
@ -314,11 +344,11 @@ public class ConsoleProducer {
|
|||
Properties producerProps() throws IOException {
|
||||
Properties props = new Properties();
|
||||
|
||||
if (options.has(producerConfigOpt)) {
|
||||
props.putAll(loadProps(options.valueOf(producerConfigOpt)));
|
||||
if (options.has(commandConfigOpt)) {
|
||||
props.putAll(loadProps(options.valueOf(commandConfigOpt)));
|
||||
}
|
||||
|
||||
props.putAll(parseKeyValueArgs(options.valuesOf(producerPropertyOpt)));
|
||||
props.putAll(parseKeyValueArgs(options.valuesOf(commandPropertyOpt)));
|
||||
props.put(BOOTSTRAP_SERVERS_CONFIG, options.valueOf(bootstrapServerOpt));
|
||||
props.put(COMPRESSION_TYPE_CONFIG, compressionCodec());
|
||||
|
||||
|
|
|
@ -48,7 +48,9 @@ public final class ConsoleConsumerOptions extends CommandDefaultOptions {
|
|||
private final OptionSpec<Integer> partitionIdOpt;
|
||||
private final OptionSpec<String> offsetOpt;
|
||||
private final OptionSpec<String> messageFormatterOpt;
|
||||
private final OptionSpec<String> messageFormatterArgOpt;
|
||||
@Deprecated(since = "4.2", forRemoval = true)
|
||||
private final OptionSpec<String> messageFormatterArgOptDeprecated;
|
||||
private OptionSpec<String> messageFormatterArgOpt;
|
||||
private final OptionSpec<String> messageFormatterConfigOpt;
|
||||
private final OptionSpec<?> resetBeginningOpt;
|
||||
private final OptionSpec<Integer> maxMessagesOpt;
|
||||
|
@ -66,9 +68,10 @@ public final class ConsoleConsumerOptions extends CommandDefaultOptions {
|
|||
private final long timeoutMs;
|
||||
private final MessageFormatter formatter;
|
||||
|
||||
@SuppressWarnings("deprecation")
|
||||
public ConsoleConsumerOptions(String[] args) throws IOException {
|
||||
super(args);
|
||||
topicOpt = parser.accepts("topic", "The topic to consume on.")
|
||||
topicOpt = parser.accepts("topic", "The topic to consume from.")
|
||||
.withRequiredArg()
|
||||
.describedAs("topic")
|
||||
.ofType(String.class);
|
||||
|
@ -87,11 +90,23 @@ public final class ConsoleConsumerOptions extends CommandDefaultOptions {
|
|||
.describedAs("consume offset")
|
||||
.ofType(String.class)
|
||||
.defaultsTo("latest");
|
||||
OptionSpec<String> consumerPropertyOpt = parser.accepts("consumer-property", "A mechanism to pass user-defined properties in the form key=value to the consumer.")
|
||||
@Deprecated(since = "4.2", forRemoval = true)
|
||||
OptionSpec<String> consumerPropertyOpt = parser.accepts("consumer-property", "(DEPRECATED) Consumer config properties in the form key=value. " +
|
||||
"This option will be removed in a future version. Use --command-property instead.")
|
||||
.withRequiredArg()
|
||||
.describedAs("consumer_prop")
|
||||
.ofType(String.class);
|
||||
OptionSpec<String> consumerConfigOpt = parser.accepts("consumer.config", "Consumer config properties file. Note that " + consumerPropertyOpt + " takes precedence over this config.")
|
||||
OptionSpec<String> commandPropertyOpt = parser.accepts("command-property", "Consumer config properties in the form key=value.")
|
||||
.withRequiredArg()
|
||||
.describedAs("consumer_prop")
|
||||
.ofType(String.class);
|
||||
@Deprecated(since = "4.2", forRemoval = true)
|
||||
OptionSpec<String> consumerConfigOpt = parser.accepts("consumer.config", "(DEPRECATED) Consumer config properties file. Note that " + commandPropertyOpt + " takes precedence over this config. " +
|
||||
"This option will be removed in a future version. Use --command-config instead.")
|
||||
.withRequiredArg()
|
||||
.describedAs("config file")
|
||||
.ofType(String.class);
|
||||
OptionSpec<String> commandConfigOpt = parser.accepts("command-config", "Consumer config properties file. Note that " + commandPropertyOpt + " takes precedence over this config.")
|
||||
.withRequiredArg()
|
||||
.describedAs("config file")
|
||||
.ofType(String.class);
|
||||
|
@ -100,7 +115,28 @@ public final class ConsoleConsumerOptions extends CommandDefaultOptions {
|
|||
.describedAs("class")
|
||||
.ofType(String.class)
|
||||
.defaultsTo(DefaultMessageFormatter.class.getName());
|
||||
messageFormatterArgOpt = parser.accepts("property",
|
||||
messageFormatterArgOptDeprecated = parser.accepts("property",
|
||||
"(DEPRECATED) The properties to initialize the message formatter. Default properties include: \n" +
|
||||
" print.timestamp=true|false\n" +
|
||||
" print.key=true|false\n" +
|
||||
" print.offset=true|false\n" +
|
||||
" print.epoch=true|false\n" +
|
||||
" print.partition=true|false\n" +
|
||||
" print.headers=true|false\n" +
|
||||
" print.value=true|false\n" +
|
||||
" key.separator=<key.separator>\n" +
|
||||
" line.separator=<line.separator>\n" +
|
||||
" headers.separator=<headers.separator>\n" +
|
||||
" null.literal=<null.literal>\n" +
|
||||
" key.deserializer=<key.deserializer>\n" +
|
||||
" value.deserializer=<value.deserializer>\n" +
|
||||
" header.deserializer=<header.deserializer>\n" +
|
||||
"\nUsers can also pass in customized properties for their formatter; more specifically, users can pass in properties keyed with 'key.deserializer.', 'value.deserializer.' and 'headers.deserializer.' prefixes to configure their deserializers. " +
|
||||
"\nThis option will be removed in a future version. Use --formatter-property instead.")
|
||||
.withRequiredArg()
|
||||
.describedAs("prop")
|
||||
.ofType(String.class);
|
||||
messageFormatterArgOpt = parser.accepts("formatter-property",
|
||||
"The properties to initialize the message formatter. Default properties include: \n" +
|
||||
" print.timestamp=true|false\n" +
|
||||
" print.key=true|false\n" +
|
||||
|
@ -140,11 +176,11 @@ public final class ConsoleConsumerOptions extends CommandDefaultOptions {
|
|||
.withRequiredArg()
|
||||
.describedAs("server to connect to")
|
||||
.ofType(String.class);
|
||||
keyDeserializerOpt = parser.accepts("key-deserializer")
|
||||
keyDeserializerOpt = parser.accepts("key-deserializer", "The name of the class to use for deserializing keys.")
|
||||
.withRequiredArg()
|
||||
.describedAs("deserializer for key")
|
||||
.describedAs("deserializer for keys")
|
||||
.ofType(String.class);
|
||||
valueDeserializerOpt = parser.accepts("value-deserializer")
|
||||
valueDeserializerOpt = parser.accepts("value-deserializer", "The name of the class to use for deserializing values.")
|
||||
.withRequiredArg()
|
||||
.describedAs("deserializer for values")
|
||||
.ofType(String.class);
|
||||
|
@ -170,11 +206,25 @@ public final class ConsoleConsumerOptions extends CommandDefaultOptions {
|
|||
CommandLineUtils.maybePrintHelpOrVersion(this, "This tool helps to read data from Kafka topics and outputs it to standard output.");
|
||||
|
||||
checkRequiredArgs();
|
||||
if (options.has(consumerPropertyOpt) && options.has(commandPropertyOpt)) {
|
||||
CommandLineUtils.printUsageAndExit(parser, "Options --consumer-property and --command-property cannot be specified together.");
|
||||
}
|
||||
if (options.has(consumerConfigOpt) && options.has(commandConfigOpt)) {
|
||||
CommandLineUtils.printUsageAndExit(parser, "Options --consumer.config and --command-config cannot be specified together.");
|
||||
}
|
||||
|
||||
Properties consumerPropsFromFile = options.has(consumerConfigOpt)
|
||||
? Utils.loadProps(options.valueOf(consumerConfigOpt))
|
||||
if (options.has(consumerPropertyOpt)) {
|
||||
System.out.println("Option --consumer-property is deprecated and will be removed in a future version. Use --command-property instead.");
|
||||
commandPropertyOpt = consumerPropertyOpt;
|
||||
}
|
||||
if (options.has(consumerConfigOpt)) {
|
||||
System.out.println("Option --consumer.config is deprecated and will be removed in a future version. Use --command-config instead.");
|
||||
commandConfigOpt = consumerConfigOpt;
|
||||
}
|
||||
Properties consumerPropsFromFile = options.has(commandConfigOpt)
|
||||
? Utils.loadProps(options.valueOf(commandConfigOpt))
|
||||
: new Properties();
|
||||
Properties extraConsumerProps = CommandLineUtils.parseKeyValueArgs(options.valuesOf(consumerPropertyOpt));
|
||||
Properties extraConsumerProps = CommandLineUtils.parseKeyValueArgs(options.valuesOf(commandPropertyOpt));
|
||||
Set<String> groupIdsProvided = checkConsumerGroup(consumerPropsFromFile, extraConsumerProps);
|
||||
consumerProps = buildConsumerProps(consumerPropsFromFile, extraConsumerProps, groupIdsProvided);
|
||||
offset = parseOffset();
|
||||
|
@ -323,6 +373,13 @@ public final class ConsoleConsumerOptions extends CommandDefaultOptions {
|
|||
Class<?> messageFormatterClass = Class.forName(options.valueOf(messageFormatterOpt));
|
||||
formatter = (MessageFormatter) messageFormatterClass.getDeclaredConstructor().newInstance();
|
||||
|
||||
if (options.has(messageFormatterArgOpt) && options.has(messageFormatterArgOptDeprecated)) {
|
||||
CommandLineUtils.printUsageAndExit(parser, "Options --property and --formatter-property cannot be specified together.");
|
||||
}
|
||||
if (options.has(messageFormatterArgOptDeprecated)) {
|
||||
System.out.println("Option --property is deprecated and will be removed in a future version. Use --formatter-property instead.");
|
||||
messageFormatterArgOpt = messageFormatterArgOptDeprecated;
|
||||
}
|
||||
Properties formatterArgs = formatterArgs();
|
||||
Map<String, String> formatterConfigs = new HashMap<>();
|
||||
for (final String name : formatterArgs.stringPropertyNames()) {
|
||||
|
|
|
@ -37,7 +37,9 @@ import joptsimple.OptionSpec;
|
|||
public final class ConsoleShareConsumerOptions extends CommandDefaultOptions {
|
||||
private final OptionSpec<String> messageFormatterOpt;
|
||||
private final OptionSpec<String> messageFormatterConfigOpt;
|
||||
private final OptionSpec<String> messageFormatterArgOpt;
|
||||
@Deprecated(since = "4.2", forRemoval = true)
|
||||
private final OptionSpec<String> messageFormatterArgOptDeprecated;
|
||||
private OptionSpec<String> messageFormatterArgOpt;
|
||||
private final OptionSpec<String> keyDeserializerOpt;
|
||||
private final OptionSpec<String> valueDeserializerOpt;
|
||||
private final OptionSpec<Integer> maxMessagesOpt;
|
||||
|
@ -52,17 +54,30 @@ public final class ConsoleShareConsumerOptions extends CommandDefaultOptions {
|
|||
private final MessageFormatter formatter;
|
||||
private final OptionSpec<?> enableSystestEventsLoggingOpt;
|
||||
|
||||
@SuppressWarnings("deprecation")
|
||||
public ConsoleShareConsumerOptions(String[] args) throws IOException {
|
||||
super(args);
|
||||
topicOpt = parser.accepts("topic", "The topic to consume from.")
|
||||
.withRequiredArg()
|
||||
.describedAs("topic")
|
||||
.ofType(String.class);
|
||||
OptionSpec<String> consumerPropertyOpt = parser.accepts("consumer-property", "A mechanism to pass user-defined properties in the form key=value to the consumer.")
|
||||
@Deprecated(since = "4.2", forRemoval = true)
|
||||
OptionSpec<String> consumerPropertyOpt = parser.accepts("consumer-property", "(DEPRECATED) Consumer config properties in the form key=value. " +
|
||||
"This option will be removed in a future version. Use --command-property instead.")
|
||||
.withRequiredArg()
|
||||
.describedAs("consumer_prop")
|
||||
.ofType(String.class);
|
||||
OptionSpec<String> consumerConfigOpt = parser.accepts("consumer-config", "Consumer config properties file. Note that " + consumerPropertyOpt + " takes precedence over this config.")
|
||||
OptionSpec<String> commandPropertyOpt = parser.accepts("command-property", "Consumer config properties in the form key=value.")
|
||||
.withRequiredArg()
|
||||
.describedAs("consumer_prop")
|
||||
.ofType(String.class);
|
||||
@Deprecated(since = "4.2", forRemoval = true)
|
||||
OptionSpec<String> consumerConfigOpt = parser.accepts("consumer-config", "(DEPRECATED) Consumer config properties file. Note that " + commandPropertyOpt + " takes precedence over this config. " +
|
||||
"This option will be removed in a future version. Use --command-config instead.")
|
||||
.withRequiredArg()
|
||||
.describedAs("config file")
|
||||
.ofType(String.class);
|
||||
OptionSpec<String> commandConfigOpt = parser.accepts("command-config", "Consumer config properties file. Note that " + commandPropertyOpt + " takes precedence over this config.")
|
||||
.withRequiredArg()
|
||||
.describedAs("config file")
|
||||
.ofType(String.class);
|
||||
|
@ -71,7 +86,29 @@ public final class ConsoleShareConsumerOptions extends CommandDefaultOptions {
|
|||
.describedAs("class")
|
||||
.ofType(String.class)
|
||||
.defaultsTo(DefaultMessageFormatter.class.getName());
|
||||
messageFormatterArgOpt = parser.accepts("property",
|
||||
messageFormatterArgOptDeprecated = parser.accepts("property",
|
||||
"(DEPRECATED) The properties to initialize the message formatter. Default properties include: \n" +
|
||||
" print.timestamp=true|false\n" +
|
||||
" print.key=true|false\n" +
|
||||
" print.offset=true|false\n" +
|
||||
" print.delivery=true|false\n" +
|
||||
" print.epoch=true|false\n" +
|
||||
" print.partition=true|false\n" +
|
||||
" print.headers=true|false\n" +
|
||||
" print.value=true|false\n" +
|
||||
" key.separator=<key.separator>\n" +
|
||||
" line.separator=<line.separator>\n" +
|
||||
" headers.separator=<headers.separator>\n" +
|
||||
" null.literal=<null.literal>\n" +
|
||||
" key.deserializer=<key.deserializer>\n" +
|
||||
" value.deserializer=<value.deserializer>\n" +
|
||||
" header.deserializer=<header.deserializer>\n" +
|
||||
"\nUsers can also pass in customized properties for their formatter; more specifically, users can pass in properties keyed with 'key.deserializer.', 'value.deserializer.' and 'headers.deserializer.' prefixes to configure their deserializers. " +
|
||||
"\nThis option will be removed in a future version. Use --formatter-property instead.")
|
||||
.withRequiredArg()
|
||||
.describedAs("prop")
|
||||
.ofType(String.class);
|
||||
messageFormatterArgOpt = parser.accepts("formatter-property",
|
||||
"The properties to initialize the message formatter. Default properties include: \n" +
|
||||
" print.timestamp=true|false\n" +
|
||||
" print.key=true|false\n" +
|
||||
|
@ -114,7 +151,7 @@ public final class ConsoleShareConsumerOptions extends CommandDefaultOptions {
|
|||
.ofType(String.class);
|
||||
keyDeserializerOpt = parser.accepts("key-deserializer", "The name of the class to use for deserializing keys.")
|
||||
.withRequiredArg()
|
||||
.describedAs("deserializer for key")
|
||||
.describedAs("deserializer for keys")
|
||||
.ofType(String.class);
|
||||
valueDeserializerOpt = parser.accepts("value-deserializer", "The name of the class to use for deserializing values.")
|
||||
.withRequiredArg()
|
||||
|
@ -141,10 +178,26 @@ public final class ConsoleShareConsumerOptions extends CommandDefaultOptions {
|
|||
CommandLineUtils.printUsageAndExit(parser, "At most one of --reject and --release may be specified.");
|
||||
}
|
||||
|
||||
Properties consumerPropsFromFile = options.has(consumerConfigOpt)
|
||||
? Utils.loadProps(options.valueOf(consumerConfigOpt))
|
||||
if (options.has(consumerPropertyOpt) && options.has(commandPropertyOpt)) {
|
||||
CommandLineUtils.printUsageAndExit(parser, "Options --consumer-property and --command-property cannot be specified together.");
|
||||
}
|
||||
if (options.has(consumerConfigOpt) && options.has(commandConfigOpt)) {
|
||||
CommandLineUtils.printUsageAndExit(parser, "Options --consumer-config and --command-config cannot be specified together.");
|
||||
}
|
||||
|
||||
if (options.has(consumerPropertyOpt)) {
|
||||
System.out.println("Option --consumer-property is deprecated and will be removed in a future version. Use --command-property instead.");
|
||||
commandPropertyOpt = consumerPropertyOpt;
|
||||
}
|
||||
if (options.has(consumerConfigOpt)) {
|
||||
System.out.println("Option --consumer-config is deprecated and will be removed in a future version. Use --command-config instead.");
|
||||
commandConfigOpt = consumerConfigOpt;
|
||||
}
|
||||
|
||||
Properties consumerPropsFromFile = options.has(commandConfigOpt)
|
||||
? Utils.loadProps(options.valueOf(commandConfigOpt))
|
||||
: new Properties();
|
||||
Properties extraConsumerProps = CommandLineUtils.parseKeyValueArgs(options.valuesOf(consumerPropertyOpt));
|
||||
Properties extraConsumerProps = CommandLineUtils.parseKeyValueArgs(options.valuesOf(commandPropertyOpt));
|
||||
|
||||
Set<String> groupIdsProvided = checkShareGroup(consumerPropsFromFile, extraConsumerProps);
|
||||
consumerProps = buildConsumerProps(consumerPropsFromFile, extraConsumerProps, groupIdsProvided);
|
||||
|
@ -203,6 +256,13 @@ public final class ConsoleShareConsumerOptions extends CommandDefaultOptions {
|
|||
Class<?> messageFormatterClass = Class.forName(options.valueOf(messageFormatterOpt));
|
||||
formatter = (MessageFormatter) messageFormatterClass.getDeclaredConstructor().newInstance();
|
||||
|
||||
if (options.has(messageFormatterArgOpt) && options.has(messageFormatterArgOptDeprecated)) {
|
||||
CommandLineUtils.printUsageAndExit(parser, "Options --property and --formatter-property cannot be specified together.");
|
||||
}
|
||||
if (options.has(messageFormatterArgOptDeprecated)) {
|
||||
System.out.println("Option --property is deprecated and will be removed in a future version. Use --formatter-property instead.");
|
||||
messageFormatterArgOpt = messageFormatterArgOptDeprecated;
|
||||
}
|
||||
Properties formatterArgs = formatterArgs();
|
||||
Map<String, String> formatterConfigs = new HashMap<>();
|
||||
for (final String name : formatterArgs.stringPropertyNames()) {
|
||||
|
|
|
@ -34,6 +34,7 @@ import java.io.InputStream;
|
|||
import java.io.OutputStream;
|
||||
import java.nio.file.Files;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -57,11 +58,16 @@ public class ConsoleProducerTest {
|
|||
"--bootstrap-server", "localhost:1002",
|
||||
"--topic", "t3",
|
||||
};
|
||||
private static final String[] CLIENT_ID_OVERRIDE = new String[]{
|
||||
private static final String[] CLIENT_ID_OVERRIDE_DEPRECATED = new String[]{
|
||||
"--bootstrap-server", "localhost:1001",
|
||||
"--topic", "t3",
|
||||
"--producer-property", "client.id=producer-1"
|
||||
};
|
||||
private static final String[] CLIENT_ID_OVERRIDE = new String[]{
|
||||
"--bootstrap-server", "localhost:1001",
|
||||
"--topic", "t3",
|
||||
"--command-property", "client.id=producer-1"
|
||||
};
|
||||
private static final String[] BATCH_SIZE_OVERRIDDEN_BY_MAX_PARTITION_MEMORY_BYTES_VALUE = new String[]{
|
||||
"--bootstrap-server", "localhost:1002",
|
||||
"--topic", "t3",
|
||||
|
@ -151,8 +157,8 @@ public class ConsoleProducerTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testClientIdOverride() throws IOException {
|
||||
ConsoleProducerOptions opts = new ConsoleProducerOptions(CLIENT_ID_OVERRIDE);
|
||||
public void testClientIdOverrideDeprecated() throws IOException {
|
||||
ConsoleProducerOptions opts = new ConsoleProducerOptions(CLIENT_ID_OVERRIDE_DEPRECATED);
|
||||
ProducerConfig producerConfig = new ProducerConfig(opts.producerProps());
|
||||
|
||||
assertEquals("producer-1", producerConfig.getString(ProducerConfig.CLIENT_ID_CONFIG));
|
||||
|
@ -222,6 +228,107 @@ public class ConsoleProducerTest {
|
|||
assertEquals(1, reader.closeCount());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldExitOnBothProducerPropertyAndCommandProperty() {
|
||||
Exit.setExitProcedure((code, message) -> {
|
||||
throw new IllegalArgumentException(message);
|
||||
});
|
||||
|
||||
String[] args = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
"--producer-property", "acks=all",
|
||||
"--command-property", "batch.size=16384"
|
||||
};
|
||||
|
||||
try {
|
||||
assertThrows(IllegalArgumentException.class, () -> new ConsoleProducerOptions(args));
|
||||
} finally {
|
||||
Exit.resetExitProcedure();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldExitOnBothProducerConfigAndCommandConfig() throws IOException {
|
||||
Exit.setExitProcedure((code, message) -> {
|
||||
throw new IllegalArgumentException(message);
|
||||
});
|
||||
|
||||
Map<String, String> configs = new HashMap<>();
|
||||
configs.put("acks", "all");
|
||||
File propsFile = ToolsTestUtils.tempPropertiesFile(configs);
|
||||
|
||||
Map<String, String> configs2 = new HashMap<>();
|
||||
configs2.put("batch.size", "16384");
|
||||
File propsFile2 = ToolsTestUtils.tempPropertiesFile(configs2);
|
||||
|
||||
String[] args = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
"--producer.config", propsFile.getAbsolutePath(),
|
||||
"--command-config", propsFile2.getAbsolutePath()
|
||||
};
|
||||
|
||||
try {
|
||||
assertThrows(IllegalArgumentException.class, () -> new ConsoleProducerOptions(args));
|
||||
} finally {
|
||||
Exit.resetExitProcedure();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testClientIdOverrideUsingCommandProperty() throws IOException {
|
||||
ConsoleProducerOptions opts = new ConsoleProducerOptions(CLIENT_ID_OVERRIDE);
|
||||
ProducerConfig producerConfig = new ProducerConfig(opts.producerProps());
|
||||
|
||||
assertEquals("producer-1", producerConfig.getString(ProducerConfig.CLIENT_ID_CONFIG));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testProducerConfigFromFileUsingCommandConfig() throws IOException {
|
||||
Map<String, String> configs = new HashMap<>();
|
||||
configs.put("acks", "all");
|
||||
configs.put("batch.size", "32768");
|
||||
File propsFile = ToolsTestUtils.tempPropertiesFile(configs);
|
||||
|
||||
String[] args = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
"--command-config", propsFile.getAbsolutePath()
|
||||
};
|
||||
|
||||
ConsoleProducerOptions opts = new ConsoleProducerOptions(args);
|
||||
ProducerConfig producerConfig = new ProducerConfig(opts.producerProps());
|
||||
|
||||
// "all" gets converted to "-1" internally by ProducerConfig
|
||||
assertEquals("-1", producerConfig.getString(ProducerConfig.ACKS_CONFIG));
|
||||
assertEquals(32768, producerConfig.getInt(ProducerConfig.BATCH_SIZE_CONFIG));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCommandPropertyOverridesConfig() throws IOException {
|
||||
Map<String, String> configs = new HashMap<>();
|
||||
configs.put("acks", "1");
|
||||
configs.put("batch.size", "16384");
|
||||
File propsFile = ToolsTestUtils.tempPropertiesFile(configs);
|
||||
|
||||
String[] args = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
"--command-config", propsFile.getAbsolutePath(),
|
||||
"--command-property", "acks=all"
|
||||
};
|
||||
|
||||
ConsoleProducerOptions opts = new ConsoleProducerOptions(args);
|
||||
ProducerConfig producerConfig = new ProducerConfig(opts.producerProps());
|
||||
|
||||
// Command property should override the config file value
|
||||
// "all" gets converted to "-1" internally by ProducerConfig
|
||||
assertEquals("-1", producerConfig.getString(ProducerConfig.ACKS_CONFIG));
|
||||
// Config file value should still be present
|
||||
assertEquals(16384, producerConfig.getInt(ProducerConfig.BATCH_SIZE_CONFIG));
|
||||
}
|
||||
|
||||
public static class TestRecordReader implements RecordReader {
|
||||
private int configureCount = 0;
|
||||
private int closeCount = 0;
|
||||
|
|
|
@ -172,7 +172,7 @@ public class ConsoleConsumerOptionsTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void shouldParseValidConsumerConfigWithAutoOffsetResetLatest() throws IOException {
|
||||
public void shouldParseValidConsumerConfigWithAutoOffsetResetLatestDeprecated() throws IOException {
|
||||
String[] args = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
|
@ -189,7 +189,7 @@ public class ConsoleConsumerOptionsTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void shouldParseValidConsumerConfigWithAutoOffsetResetEarliest() throws IOException {
|
||||
public void shouldParseValidConsumerConfigWithAutoOffsetResetEarliestDeprecated() throws IOException {
|
||||
String[] args = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
|
@ -206,7 +206,7 @@ public class ConsoleConsumerOptionsTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void shouldParseValidConsumerConfigWithAutoOffsetResetAndMatchingFromBeginning() throws IOException {
|
||||
public void shouldParseValidConsumerConfigWithAutoOffsetResetAndMatchingFromBeginningDeprecated() throws IOException {
|
||||
String[] args = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
|
@ -240,7 +240,7 @@ public class ConsoleConsumerOptionsTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void shouldExitOnInvalidConfigWithAutoOffsetResetAndConflictingFromBeginning() {
|
||||
public void shouldExitOnInvalidConfigWithAutoOffsetResetAndConflictingFromBeginningDeprecated() {
|
||||
Exit.setExitProcedure((code, message) -> {
|
||||
throw new IllegalArgumentException(message);
|
||||
});
|
||||
|
@ -259,7 +259,7 @@ public class ConsoleConsumerOptionsTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void shouldParseConfigsFromFile() throws IOException {
|
||||
public void shouldParseConfigsFromFileDeprecated() throws IOException {
|
||||
Map<String, String> configs = new HashMap<>();
|
||||
configs.put("request.timeout.ms", "1000");
|
||||
configs.put("group.id", "group1");
|
||||
|
@ -276,10 +276,11 @@ public class ConsoleConsumerOptionsTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void groupIdsProvidedInDifferentPlacesMustMatch() throws IOException {
|
||||
public void groupIdsProvidedInDifferentPlacesMustMatchDeprecated() throws IOException {
|
||||
Exit.setExitProcedure((code, message) -> {
|
||||
throw new IllegalArgumentException(message);
|
||||
});
|
||||
try {
|
||||
|
||||
// different in all three places
|
||||
File propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file"));
|
||||
|
@ -348,9 +349,10 @@ public class ConsoleConsumerOptionsTest {
|
|||
config = new ConsoleConsumerOptions(args5);
|
||||
props = config.consumerProps();
|
||||
assertEquals("group-from-arguments", props.getProperty("group.id"));
|
||||
|
||||
} finally {
|
||||
Exit.resetExitProcedure();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCustomPropertyShouldBePassedToConfigureMethod() throws Exception {
|
||||
|
@ -508,7 +510,7 @@ public class ConsoleConsumerOptionsTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testClientIdOverride() throws IOException {
|
||||
public void testClientIdOverrideDeprecated() throws IOException {
|
||||
String[] args = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
|
@ -618,4 +620,234 @@ public class ConsoleConsumerOptionsTest {
|
|||
"--formatter", formatter,
|
||||
};
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldExitOnBothConsumerPropertyAndCommandProperty() {
|
||||
Exit.setExitProcedure((code, message) -> {
|
||||
throw new IllegalArgumentException(message);
|
||||
});
|
||||
|
||||
String[] args = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
"--consumer-property", "auto.offset.reset=latest",
|
||||
"--command-property", "session.timeout.ms=10000"
|
||||
};
|
||||
|
||||
try {
|
||||
assertThrows(IllegalArgumentException.class, () -> new ConsoleConsumerOptions(args));
|
||||
} finally {
|
||||
Exit.resetExitProcedure();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldExitOnBothConsumerConfigAndCommandConfig() throws IOException {
|
||||
Exit.setExitProcedure((code, message) -> {
|
||||
throw new IllegalArgumentException(message);
|
||||
});
|
||||
|
||||
Map<String, String> configs = new HashMap<>();
|
||||
configs.put("request.timeout.ms", "1000");
|
||||
File propsFile = ToolsTestUtils.tempPropertiesFile(configs);
|
||||
|
||||
Map<String, String> configs2 = new HashMap<>();
|
||||
configs2.put("session.timeout.ms", "10000");
|
||||
File propsFile2 = ToolsTestUtils.tempPropertiesFile(configs2);
|
||||
|
||||
String[] args = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
"--consumer.config", propsFile.getAbsolutePath(),
|
||||
"--command-config", propsFile2.getAbsolutePath()
|
||||
};
|
||||
|
||||
try {
|
||||
assertThrows(IllegalArgumentException.class, () -> new ConsoleConsumerOptions(args));
|
||||
} finally {
|
||||
Exit.resetExitProcedure();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldParseValidConsumerConfigWithAutoOffsetResetLatestUsingCommandProperty() throws IOException {
|
||||
String[] args = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
"--command-property", "auto.offset.reset=latest"
|
||||
};
|
||||
|
||||
ConsoleConsumerOptions config = new ConsoleConsumerOptions(args);
|
||||
Properties consumerProperties = config.consumerProps();
|
||||
|
||||
assertEquals("localhost:9092", config.bootstrapServer());
|
||||
assertEquals("test", config.topicArg().orElse(""));
|
||||
assertFalse(config.fromBeginning());
|
||||
assertEquals("latest", consumerProperties.getProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldParseValidConsumerConfigWithAutoOffsetResetEarliestUsingCommandProperty() throws IOException {
|
||||
String[] args = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
"--command-property", "auto.offset.reset=earliest"
|
||||
};
|
||||
|
||||
ConsoleConsumerOptions config = new ConsoleConsumerOptions(args);
|
||||
Properties consumerProperties = config.consumerProps();
|
||||
|
||||
assertEquals("localhost:9092", config.bootstrapServer());
|
||||
assertEquals("test", config.topicArg().orElse(""));
|
||||
assertFalse(config.fromBeginning());
|
||||
assertEquals("earliest", consumerProperties.getProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldParseValidConsumerConfigWithAutoOffsetResetAndMatchingFromBeginningUsingCommandProperty() throws IOException {
|
||||
String[] args = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
"--command-property", "auto.offset.reset=earliest",
|
||||
"--from-beginning"
|
||||
};
|
||||
|
||||
ConsoleConsumerOptions config = new ConsoleConsumerOptions(args);
|
||||
Properties consumerProperties = config.consumerProps();
|
||||
|
||||
assertEquals("localhost:9092", config.bootstrapServer());
|
||||
assertEquals("test", config.topicArg().orElse(""));
|
||||
assertTrue(config.fromBeginning());
|
||||
assertEquals("earliest", consumerProperties.getProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldExitOnInvalidConfigWithAutoOffsetResetAndConflictingFromBeginningUsingCommandProperty() {
|
||||
Exit.setExitProcedure((code, message) -> {
|
||||
throw new IllegalArgumentException(message);
|
||||
});
|
||||
|
||||
String[] args = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
"--command-property", "auto.offset.reset=latest",
|
||||
"--from-beginning"
|
||||
};
|
||||
try {
|
||||
assertThrows(IllegalArgumentException.class, () -> new ConsoleConsumerOptions(args));
|
||||
} finally {
|
||||
Exit.resetExitProcedure();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldParseConfigsFromFileUsingCommandConfig() throws IOException {
|
||||
Map<String, String> configs = new HashMap<>();
|
||||
configs.put("request.timeout.ms", "1000");
|
||||
configs.put("group.id", "group1");
|
||||
File propsFile = ToolsTestUtils.tempPropertiesFile(configs);
|
||||
String[] args = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
"--command-config", propsFile.getAbsolutePath()
|
||||
};
|
||||
|
||||
ConsoleConsumerOptions config = new ConsoleConsumerOptions(args);
|
||||
assertEquals("1000", config.consumerProps().get("request.timeout.ms"));
|
||||
assertEquals("group1", config.consumerProps().get("group.id"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void groupIdsProvidedInDifferentPlacesMustMatchUsingCommandConfig() throws IOException {
|
||||
Exit.setExitProcedure((code, message) -> {
|
||||
throw new IllegalArgumentException(message);
|
||||
});
|
||||
|
||||
try {
|
||||
// different in all three places
|
||||
File propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file"));
|
||||
final String[] args = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
"--group", "group-from-arguments",
|
||||
"--command-property", "group.id=group-from-properties",
|
||||
"--command-config", propsFile.getAbsolutePath()
|
||||
};
|
||||
|
||||
assertThrows(IllegalArgumentException.class, () -> new ConsoleConsumerOptions(args));
|
||||
|
||||
// the same in all three places
|
||||
propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "test-group"));
|
||||
final String[] args1 = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
"--group", "test-group",
|
||||
"--command-property", "group.id=test-group",
|
||||
"--command-config", propsFile.getAbsolutePath()
|
||||
};
|
||||
|
||||
ConsoleConsumerOptions config = new ConsoleConsumerOptions(args1);
|
||||
Properties props = config.consumerProps();
|
||||
assertEquals("test-group", props.getProperty("group.id"));
|
||||
|
||||
// different via --command-property and --command-config
|
||||
propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file"));
|
||||
final String[] args2 = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
"--command-property", "group.id=group-from-properties",
|
||||
"--command-config", propsFile.getAbsolutePath()
|
||||
};
|
||||
|
||||
assertThrows(IllegalArgumentException.class, () -> new ConsoleConsumerOptions(args2));
|
||||
|
||||
// different via --command-property and --group
|
||||
final String[] args3 = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
"--group", "group-from-arguments",
|
||||
"--command-property", "group.id=group-from-properties"
|
||||
};
|
||||
|
||||
assertThrows(IllegalArgumentException.class, () -> new ConsoleConsumerOptions(args3));
|
||||
|
||||
// different via --group and --command-config
|
||||
propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file"));
|
||||
final String[] args4 = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
"--group", "group-from-arguments",
|
||||
"--command-config", propsFile.getAbsolutePath()
|
||||
};
|
||||
assertThrows(IllegalArgumentException.class, () -> new ConsoleConsumerOptions(args4));
|
||||
|
||||
// via --group only
|
||||
final String[] args5 = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
"--group", "group-from-arguments"
|
||||
};
|
||||
|
||||
config = new ConsoleConsumerOptions(args5);
|
||||
props = config.consumerProps();
|
||||
assertEquals("group-from-arguments", props.getProperty("group.id"));
|
||||
} finally {
|
||||
Exit.resetExitProcedure();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testClientIdOverrideUsingCommandProperty() throws IOException {
|
||||
String[] args = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
"--from-beginning",
|
||||
"--command-property", "client.id=consumer-1"
|
||||
};
|
||||
|
||||
ConsoleConsumerOptions config = new ConsoleConsumerOptions(args);
|
||||
Properties consumerProperties = config.consumerProps();
|
||||
|
||||
assertEquals("consumer-1", consumerProperties.getProperty(ConsumerConfig.CLIENT_ID_CONFIG));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,6 +20,7 @@ import org.apache.kafka.clients.consumer.AcknowledgeType;
|
|||
import org.apache.kafka.clients.consumer.ConsumerConfig;
|
||||
import org.apache.kafka.common.utils.Exit;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.test.MockDeserializer;
|
||||
import org.apache.kafka.tools.ToolsTestUtils;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
@ -32,7 +33,9 @@ import java.util.Properties;
|
|||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class ConsoleShareConsumerOptionsTest {
|
||||
|
||||
|
@ -72,7 +75,7 @@ public class ConsoleShareConsumerOptionsTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void shouldParseValidConsumerConfigWithSessionTimeout() throws IOException {
|
||||
public void shouldParseValidConsumerConfigWithSessionTimeoutDeprecated() throws IOException {
|
||||
String[] args = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
|
@ -88,7 +91,7 @@ public class ConsoleShareConsumerOptionsTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void shouldParseConfigsFromFile() throws IOException {
|
||||
public void shouldParseConfigsFromFileDeprecated() throws IOException {
|
||||
Map<String, String> configs = new HashMap<>();
|
||||
configs.put("request.timeout.ms", "1000");
|
||||
configs.put("group.id", "group1");
|
||||
|
@ -109,11 +112,12 @@ public class ConsoleShareConsumerOptionsTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void groupIdsProvidedInDifferentPlacesMustMatch() throws IOException {
|
||||
public void groupIdsProvidedInDifferentPlacesMustMatchDeprecated() throws IOException {
|
||||
Exit.setExitProcedure((code, message) -> {
|
||||
throw new IllegalArgumentException(message);
|
||||
});
|
||||
|
||||
try {
|
||||
// different in all three places
|
||||
File propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file"));
|
||||
final String[] args = new String[]{
|
||||
|
@ -181,9 +185,10 @@ public class ConsoleShareConsumerOptionsTest {
|
|||
config = new ConsoleShareConsumerOptions(args5);
|
||||
props = config.consumerProps();
|
||||
assertEquals("group-from-arguments", props.getProperty("group.id"));
|
||||
|
||||
} finally {
|
||||
Exit.resetExitProcedure();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldExitIfNoTopicSpecified() {
|
||||
|
@ -203,7 +208,7 @@ public class ConsoleShareConsumerOptionsTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testClientIdOverride() throws IOException {
|
||||
public void testClientIdOverrideDeprecated() throws IOException {
|
||||
String[] args = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
|
@ -216,6 +221,56 @@ public class ConsoleShareConsumerOptionsTest {
|
|||
assertEquals("consumer-1", consumerProperties.getProperty(ConsumerConfig.CLIENT_ID_CONFIG));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCustomPropertyShouldBePassedToConfigureMethod() throws Exception {
|
||||
String[] args = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
"--property", "print.key=true",
|
||||
"--property", "key.deserializer=org.apache.kafka.test.MockDeserializer",
|
||||
"--property", "key.deserializer.my-props=abc"
|
||||
};
|
||||
|
||||
ConsoleShareConsumerOptions config = new ConsoleShareConsumerOptions(args);
|
||||
|
||||
assertInstanceOf(DefaultMessageFormatter.class, config.formatter());
|
||||
assertTrue(config.formatterArgs().containsKey("key.deserializer.my-props"));
|
||||
DefaultMessageFormatter formatter = (DefaultMessageFormatter) config.formatter();
|
||||
assertTrue(formatter.keyDeserializer().isPresent());
|
||||
assertInstanceOf(MockDeserializer.class, formatter.keyDeserializer().get());
|
||||
MockDeserializer keyDeserializer = (MockDeserializer) formatter.keyDeserializer().get();
|
||||
assertEquals(1, keyDeserializer.configs.size());
|
||||
assertEquals("abc", keyDeserializer.configs.get("my-props"));
|
||||
assertTrue(keyDeserializer.isKey);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCustomConfigShouldBePassedToConfigureMethod() throws Exception {
|
||||
Map<String, String> configs = new HashMap<>();
|
||||
configs.put("key.deserializer.my-props", "abc");
|
||||
configs.put("print.key", "false");
|
||||
File propsFile = ToolsTestUtils.tempPropertiesFile(configs);
|
||||
String[] args = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
"--property", "print.key=true",
|
||||
"--property", "key.deserializer=org.apache.kafka.test.MockDeserializer",
|
||||
"--formatter-config", propsFile.getAbsolutePath()
|
||||
};
|
||||
|
||||
ConsoleShareConsumerOptions config = new ConsoleShareConsumerOptions(args);
|
||||
|
||||
assertInstanceOf(DefaultMessageFormatter.class, config.formatter());
|
||||
assertTrue(config.formatterArgs().containsKey("key.deserializer.my-props"));
|
||||
DefaultMessageFormatter formatter = (DefaultMessageFormatter) config.formatter();
|
||||
assertTrue(formatter.keyDeserializer().isPresent());
|
||||
assertInstanceOf(MockDeserializer.class, formatter.keyDeserializer().get());
|
||||
MockDeserializer keyDeserializer = (MockDeserializer) formatter.keyDeserializer().get();
|
||||
assertEquals(1, keyDeserializer.configs.size());
|
||||
assertEquals("abc", keyDeserializer.configs.get("my-props"));
|
||||
assertTrue(keyDeserializer.isKey);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDefaultClientId() throws IOException {
|
||||
String[] args = new String[]{
|
||||
|
@ -271,4 +326,182 @@ public class ConsoleShareConsumerOptionsTest {
|
|||
Exit.resetExitProcedure();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldExitOnBothConsumerPropertyAndCommandProperty() {
|
||||
Exit.setExitProcedure((code, message) -> {
|
||||
throw new IllegalArgumentException(message);
|
||||
});
|
||||
|
||||
String[] args = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
"--consumer-property", "session.timeout.ms=10000",
|
||||
"--command-property", "request.timeout.ms=30000"
|
||||
};
|
||||
|
||||
try {
|
||||
assertThrows(IllegalArgumentException.class, () -> new ConsoleShareConsumerOptions(args));
|
||||
} finally {
|
||||
Exit.resetExitProcedure();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldExitOnBothConsumerConfigAndCommandConfig() throws IOException {
|
||||
Exit.setExitProcedure((code, message) -> {
|
||||
throw new IllegalArgumentException(message);
|
||||
});
|
||||
|
||||
Map<String, String> configs = new HashMap<>();
|
||||
configs.put("request.timeout.ms", "1000");
|
||||
File propsFile = ToolsTestUtils.tempPropertiesFile(configs);
|
||||
|
||||
Map<String, String> configs2 = new HashMap<>();
|
||||
configs2.put("session.timeout.ms", "10000");
|
||||
File propsFile2 = ToolsTestUtils.tempPropertiesFile(configs2);
|
||||
|
||||
String[] args = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
"--consumer-config", propsFile.getAbsolutePath(),
|
||||
"--command-config", propsFile2.getAbsolutePath()
|
||||
};
|
||||
|
||||
try {
|
||||
assertThrows(IllegalArgumentException.class, () -> new ConsoleShareConsumerOptions(args));
|
||||
} finally {
|
||||
Exit.resetExitProcedure();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldParseValidConsumerConfigWithSessionTimeout() throws IOException {
|
||||
String[] args = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
"--command-property", "session.timeout.ms=10000"
|
||||
};
|
||||
|
||||
ConsoleShareConsumerOptions config = new ConsoleShareConsumerOptions(args);
|
||||
Properties consumerProperties = config.consumerProps();
|
||||
|
||||
assertEquals("localhost:9092", config.bootstrapServer());
|
||||
assertEquals("test", config.topicArg());
|
||||
assertEquals("10000", consumerProperties.getProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldParseConfigsFromFile() throws IOException {
|
||||
Map<String, String> configs = new HashMap<>();
|
||||
configs.put("request.timeout.ms", "1000");
|
||||
configs.put("group.id", "group1");
|
||||
File propsFile = ToolsTestUtils.tempPropertiesFile(configs);
|
||||
String[] args = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
"--command-config", propsFile.getAbsolutePath()
|
||||
};
|
||||
|
||||
ConsoleShareConsumerOptions config = new ConsoleShareConsumerOptions(args);
|
||||
|
||||
// KafkaShareConsumer uses Utils.propsToMap to convert the properties to a map,
|
||||
// so using the same method to check the map has the expected values
|
||||
Map<String, Object> configMap = Utils.propsToMap(config.consumerProps());
|
||||
assertEquals("1000", configMap.get("request.timeout.ms"));
|
||||
assertEquals("group1", configMap.get("group.id"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void groupIdsProvidedInDifferentPlacesMustMatch() throws IOException {
|
||||
Exit.setExitProcedure((code, message) -> {
|
||||
throw new IllegalArgumentException(message);
|
||||
});
|
||||
|
||||
try {
|
||||
// different in all three places
|
||||
File propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file"));
|
||||
final String[] args = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
"--group", "group-from-arguments",
|
||||
"--command-property", "group.id=group-from-properties",
|
||||
"--command-config", propsFile.getAbsolutePath()
|
||||
};
|
||||
|
||||
assertThrows(IllegalArgumentException.class, () -> new ConsoleShareConsumerOptions(args));
|
||||
|
||||
// the same in all three places
|
||||
propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "test-group"));
|
||||
final String[] args1 = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
"--group", "test-group",
|
||||
"--command-property", "group.id=test-group",
|
||||
"--command-config", propsFile.getAbsolutePath()
|
||||
};
|
||||
|
||||
ConsoleShareConsumerOptions config = new ConsoleShareConsumerOptions(args1);
|
||||
Properties props = config.consumerProps();
|
||||
assertEquals("test-group", props.getProperty("group.id"));
|
||||
|
||||
// different via --command-property and --command-config
|
||||
propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file"));
|
||||
final String[] args2 = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
"--command-property", "group.id=group-from-properties",
|
||||
"--command-config", propsFile.getAbsolutePath()
|
||||
};
|
||||
|
||||
assertThrows(IllegalArgumentException.class, () -> new ConsoleShareConsumerOptions(args2));
|
||||
|
||||
// different via --command-property and --group
|
||||
final String[] args3 = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
"--group", "group-from-arguments",
|
||||
"--command-property", "group.id=group-from-properties"
|
||||
};
|
||||
|
||||
assertThrows(IllegalArgumentException.class, () -> new ConsoleShareConsumerOptions(args3));
|
||||
|
||||
// different via --group and --command-config
|
||||
propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file"));
|
||||
final String[] args4 = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
"--group", "group-from-arguments",
|
||||
"--command-config", propsFile.getAbsolutePath()
|
||||
};
|
||||
assertThrows(IllegalArgumentException.class, () -> new ConsoleShareConsumerOptions(args4));
|
||||
|
||||
// via --group only
|
||||
final String[] args5 = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
"--group", "group-from-arguments"
|
||||
};
|
||||
|
||||
config = new ConsoleShareConsumerOptions(args5);
|
||||
props = config.consumerProps();
|
||||
assertEquals("group-from-arguments", props.getProperty("group.id"));
|
||||
} finally {
|
||||
Exit.resetExitProcedure();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testClientIdOverride() throws IOException {
|
||||
String[] args = new String[]{
|
||||
"--bootstrap-server", "localhost:9092",
|
||||
"--topic", "test",
|
||||
"--command-property", "client.id=consumer-1"
|
||||
};
|
||||
|
||||
ConsoleShareConsumerOptions config = new ConsoleShareConsumerOptions(args);
|
||||
Properties consumerProperties = config.consumerProps();
|
||||
|
||||
assertEquals("consumer-1", consumerProperties.getProperty(ConsumerConfig.CLIENT_ID_CONFIG));
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue