Merge branch 'trunk' into KAFKA-18376-chain-events-in-background-thread-without-addAndGet

This commit is contained in:
Kirk True 2025-09-19 13:35:28 -07:00
commit 524782cd79
97 changed files with 3844 additions and 907 deletions

View File

@ -3350,6 +3350,7 @@ project(':jmh-benchmarks') {
implementation project(':raft') implementation project(':raft')
implementation project(':clients') implementation project(':clients')
implementation project(':coordinator-common') implementation project(':coordinator-common')
implementation project(':coordinator-common').sourceSets.test.output
implementation project(':group-coordinator') implementation project(':group-coordinator')
implementation project(':group-coordinator:group-coordinator-api') implementation project(':group-coordinator:group-coordinator-api')
implementation project(':metadata') implementation project(':metadata')

View File

@ -52,9 +52,7 @@
<allow pkg="org.apache.kafka.server"/> <allow pkg="org.apache.kafka.server"/>
<allow pkg="org.apache.kafka.storage"/> <allow pkg="org.apache.kafka.storage"/>
<allow pkg="org.apache.kafka.clients"/> <allow pkg="org.apache.kafka.clients"/>
<allow class="org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage"/> <allow pkg="org.apache.kafka.coordinator.common.runtime"/>
<allow class="org.apache.kafka.coordinator.common.runtime.KRaftCoordinatorMetadataImage"/>
<allow class="org.apache.kafka.coordinator.common.runtime.HdrHistogram"/>
<allow pkg="org.apache.kafka.coordinator.group"/> <allow pkg="org.apache.kafka.coordinator.group"/>
<allow pkg="org.apache.kafka.image"/> <allow pkg="org.apache.kafka.image"/>
<allow pkg="org.apache.kafka.metadata"/> <allow pkg="org.apache.kafka.metadata"/>

View File

@ -107,6 +107,7 @@
<allow pkg="org.apache.kafka.common.config" /> <allow pkg="org.apache.kafka.common.config" />
<allow pkg="org.apache.kafka.common.message" /> <allow pkg="org.apache.kafka.common.message" />
<allow pkg="org.apache.kafka.common.metadata" /> <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.protocol" />
<allow pkg="org.apache.kafka.common.quota" /> <allow pkg="org.apache.kafka.common.quota" />
<allow pkg="org.apache.kafka.common.record" /> <allow pkg="org.apache.kafka.common.record" />

View File

@ -27,7 +27,9 @@ import java.util.Collection;
import java.util.List; import java.util.List;
import java.util.Locale; import java.util.Locale;
import java.util.Map; import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.BiConsumer; import java.util.function.BiConsumer;
import static org.apache.kafka.clients.ClientsTestUtils.consumeAndVerifyRecords; import static org.apache.kafka.clients.ClientsTestUtils.consumeAndVerifyRecords;
@ -166,6 +168,80 @@ public class PlaintextConsumerCallbackTest {
testGetPositionOfNewlyAssignedPartitionOnPartitionsAssignedCallback(CLASSIC); 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 @ClusterTest
public void testAsyncConsumerGetPositionOfNewlyAssignedPartitionOnPartitionsAssignedCallback() throws InterruptedException { public void testAsyncConsumerGetPositionOfNewlyAssignedPartitionOnPartitionsAssignedCallback() throws InterruptedException {
testGetPositionOfNewlyAssignedPartitionOnPartitionsAssignedCallback(CONSUMER); testGetPositionOfNewlyAssignedPartitionOnPartitionsAssignedCallback(CONSUMER);
@ -284,4 +360,12 @@ public class PlaintextConsumerCallbackTest {
ENABLE_AUTO_COMMIT_CONFIG, "false" 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"
));
}
} }

View File

@ -22,7 +22,7 @@ import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Collection; import java.util.Collection;
/** /**
* Options for the {@link Admin#deleteShareGroups(Collection <String>, DeleteShareGroupsOptions)} call. * Options for the {@link Admin#deleteShareGroups(Collection, DeleteShareGroupsOptions)} call.
* <p> * <p>
* The API of this class is evolving, see {@link Admin} for details. * The API of this class is evolving, see {@link Admin} for details.
*/ */

View File

@ -25,7 +25,7 @@ import java.util.HashMap;
import java.util.Map; 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> * <p></p>
* The API of this class is evolving, see {@link Admin} for details. * The API of this class is evolving, see {@link Admin} for details.
*/ */

View File

@ -21,7 +21,7 @@ import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Collection; import java.util.Collection;
/** /**
* Options for the {@link Admin#deleteStreamsGroups(Collection<String>, DeleteStreamsGroupsOptions)} call. * Options for the {@link Admin#deleteStreamsGroups(Collection, DeleteStreamsGroupsOptions)} call.
* <p> * <p>
* The API of this class is evolving, see {@link Admin} for details. * The API of this class is evolving, see {@link Admin} for details.
*/ */

View File

@ -21,6 +21,7 @@ import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Collection; import java.util.Collection;
import java.util.Map;
/** /**
* Specification of streams group offsets to list using {@link Admin#listStreamsGroupOffsets(Map, ListStreamsGroupOffsetsOptions)}. * Specification of streams group offsets to list using {@link Admin#listStreamsGroupOffsets(Map, ListStreamsGroupOffsetsOptions)}.

View File

@ -33,14 +33,16 @@ public class RecordsToDelete {
/** /**
* Delete all the records before the given {@code offset} * 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) { public static RecordsToDelete beforeOffset(long offset) {
return new RecordsToDelete(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() { public long beforeOffset() {
return offset; return offset;

View File

@ -122,8 +122,8 @@ public class DeleteShareGroupOffsetsHandler extends AdminApiHandler.Batched<Coor
if (topic.errorCode() != Errors.NONE.code()) { if (topic.errorCode() != Errors.NONE.code()) {
final Errors topicError = Errors.forCode(topic.errorCode()); final Errors topicError = Errors.forCode(topic.errorCode());
final String topicErrorMessage = topic.errorMessage(); final String topicErrorMessage = topic.errorMessage();
log.debug("DeleteShareGroupOffsets request for group id {} and topic {} failed and returned error {}." + topicErrorMessage, log.debug("DeleteShareGroupOffsets request for group id {} and topic {} failed and returned error {}. {}",
groupId.idValue, topic.topicName(), topicError); groupId.idValue, topic.topicName(), topicError, topicErrorMessage);
} }
topicResults.put( topicResults.put(
topic.topicName(), topic.topicName(),
@ -147,14 +147,14 @@ public class DeleteShareGroupOffsetsHandler extends AdminApiHandler.Batched<Coor
case REBALANCE_IN_PROGRESS: case REBALANCE_IN_PROGRESS:
// If the coordinator is in the middle of loading, then we just need to retry // 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" + 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; break;
case COORDINATOR_NOT_AVAILABLE: case COORDINATOR_NOT_AVAILABLE:
case NOT_COORDINATOR: case NOT_COORDINATOR:
// If the coordinator is unavailable or there was a coordinator change, then we unmap // If the coordinator is unavailable or there was a coordinator change, then we unmap
// the key so that we retry the `FindCoordinator` request // 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, log.debug("DeleteShareGroupOffsets request for group id {} returned error {}. Will rediscover the coordinator and retry. {}",
groupId.idValue, error); groupId.idValue, error, errorMessage);
groupsToUnmap.add(groupId); groupsToUnmap.add(groupId);
break; break;
case INVALID_GROUP_ID: case INVALID_GROUP_ID:
@ -164,11 +164,11 @@ public class DeleteShareGroupOffsetsHandler extends AdminApiHandler.Batched<Coor
case UNKNOWN_SERVER_ERROR: case UNKNOWN_SERVER_ERROR:
case KAFKA_STORAGE_ERROR: case KAFKA_STORAGE_ERROR:
case GROUP_AUTHORIZATION_FAILED: 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)); failed.put(groupId, error.exception(errorMessage));
break; break;
default: 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)); failed.put(groupId, error.exception(errorMessage));
} }
} }

View File

@ -202,7 +202,7 @@ public final class ListOffsetsHandler extends Batched<TopicPartition, ListOffset
public Map<TopicPartition, Throwable> handleUnsupportedVersionException( public Map<TopicPartition, Throwable> handleUnsupportedVersionException(
int brokerId, UnsupportedVersionException exception, Set<TopicPartition> keys 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<>(); Map<TopicPartition, Throwable> maxTimestampPartitions = new HashMap<>();
for (TopicPartition topicPartition : keys) { for (TopicPartition topicPartition : keys) {
Long offsetTimestamp = offsetTimestampsByPartition.get(topicPartition); Long offsetTimestamp = offsetTimestampsByPartition.get(topicPartition);

View File

@ -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 * 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 * 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 * 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. * other consumer member taking over that partition and triggering its {@link #onPartitionsAssigned(Collection) onPartitionsAssigned} callback to load the state.
* <p> * <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. * 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. * 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. * 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)}) * The consumer may need to give up some partitions (thus this callback executed) under the following scenarios:
* or is unsubscribing ({@link KafkaConsumer#unsubscribe()}). * <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 * It is recommended that offsets should be committed in this callback to either Kafka or a
* custom offset store to prevent duplicate data. * custom offset store to prevent duplicate data.
* <p> * <p>
* In eager rebalancing, it will always be called at the start of a rebalance and after the consumer stops fetching data. * This callback is always called before re-assigning the partitions.
* 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. * 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}. * For examples on usage of this API, see Usage Examples section of {@link KafkaConsumer KafkaConsumer}.
* <p> * <p>
* It is common for the revocation callback to use the consumer instance in order to commit offsets. It is possible * 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 * 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. * 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 * @param partitions The list of partitions that were assigned to the consumer and now need to be revoked. This will
* include all currently assigned partitions, i.e. there may still be some partitions left) * 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.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} * @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 * 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 * partition re-assignment. This method will be called after the partition re-assignment completes (even if no new
* consumer starts fetching data, and only as the result of a {@link Consumer#poll(java.time.Duration) poll(long)} call. * 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> * <p>
* It is guaranteed that under normal conditions all the processes in a consumer group will execute their * 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 #onPartitionsRevoked(Collection)} callback before any instance executes this onPartitionsAssigned callback.
* {@link #onPartitionsAssigned(Collection)} callback. During exceptional scenarios, partitions may be migrated * During exceptional scenarios, partitions may be migrated
* without the old owner being notified (i.e. their {@link #onPartitionsRevoked(Collection)} callback not triggered), * 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 * and later when the old owner consumer realized this event, the {@link #onPartitionsLost(Collection)} callback
* will be triggered by the consumer then. * 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 * 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. * 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 * @param partitions Partitions that have been added to the assignment as a result of the rebalance.
* NOT be included, i.e. this list will only include newly added partitions) * 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.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} * @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. * 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 * @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 * to other consumers. With both, the Classic and Consumer protocols, this will always include
* previously assigned partitions, but this may change in future protocols (ie there would still * all partitions that were previously assigned to the consumer.
* be some partitions left)
* @throws org.apache.kafka.common.errors.WakeupException If raised from a nested call to {@link KafkaConsumer} * @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} * @throws org.apache.kafka.common.errors.InterruptException If raised from a nested call to {@link KafkaConsumer}
*/ */

View File

@ -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()}. * If the given list of topics is empty, it is treated the same as {@link #unsubscribe()}.
* *
* <p> * <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: * group and will trigger a rebalance operation if any one of the following events are triggered:
* <ul> * <ul>
* <li>Number of partitions change for any of the subscribed topics * <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 * <li>A new member is added to the consumer group
* </ul> * </ul>
* <p> * <p>
* When any of these events are triggered, the provided listener will be invoked first to indicate that * When any of these events are triggered, the provided listener will be invoked in this way:
* the consumer's assignment has been revoked, and then again when the new assignment has been received. * <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 * Note that rebalances will only occur during an active call to {@link #poll(Duration)}, so callbacks will
* also only be invoked during that time. * also only be invoked during that time.
* *

View File

@ -988,7 +988,7 @@ public abstract class AbstractMembershipManager<R extends AbstractResponse> impl
String reason = rejoinedWhileReconciliationInProgress ? String reason = rejoinedWhileReconciliationInProgress ?
"the member has re-joined the group" : "the member has re-joined the group" :
"the member already transitioned out of the reconciling state into " + state; "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(); markReconciliationCompleted();
} }
return shouldAbort; return shouldAbort;

View File

@ -965,8 +965,8 @@ public abstract class AbstractStickyAssignor extends AbstractPartitionAssignor {
super(partitionsPerTopic, rackInfo, currentAssignment); super(partitionsPerTopic, rackInfo, currentAssignment);
this.subscriptions = subscriptions; this.subscriptions = subscriptions;
topic2AllPotentialConsumers = new HashMap<>(partitionsPerTopic.keySet().size()); topic2AllPotentialConsumers = new HashMap<>(partitionsPerTopic.size());
consumer2AllPotentialTopics = new HashMap<>(subscriptions.keySet().size()); consumer2AllPotentialTopics = new HashMap<>(subscriptions.size());
// initialize topic2AllPotentialConsumers and consumer2AllPotentialTopics // initialize topic2AllPotentialConsumers and consumer2AllPotentialTopics
partitionsPerTopic.keySet().forEach( partitionsPerTopic.keySet().forEach(

View File

@ -187,25 +187,6 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
*/ */
private class BackgroundEventProcessor implements EventProcessor<BackgroundEvent> { 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 @Override
public void process(final BackgroundEvent event) { public void process(final BackgroundEvent event) {
switch (event.type()) { switch (event.type()) {
@ -278,44 +259,26 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
private StreamsOnTasksRevokedCallbackCompletedEvent invokeOnTasksRevokedCallback(final Set<StreamsRebalanceData.TaskId> activeTasksToRevoke, private StreamsOnTasksRevokedCallbackCompletedEvent invokeOnTasksRevokedCallback(final Set<StreamsRebalanceData.TaskId> activeTasksToRevoke,
final CompletableFuture<Void> future) { 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")); final Optional<KafkaException> error = exceptionFromCallback.map(e -> ConsumerUtils.maybeWrapAsKafkaException(e, "Task revocation callback throws an error"));
return new StreamsOnTasksRevokedCallbackCompletedEvent(future, error); return new StreamsOnTasksRevokedCallbackCompletedEvent(future, error);
} }
private StreamsOnTasksAssignedCallbackCompletedEvent invokeOnTasksAssignedCallback(final StreamsRebalanceData.Assignment assignment, private StreamsOnTasksAssignedCallbackCompletedEvent invokeOnTasksAssignedCallback(final StreamsRebalanceData.Assignment assignment,
final CompletableFuture<Void> future) { final CompletableFuture<Void> future) {
final Optional<KafkaException> error; final Optional<Exception> exceptionFromCallback = Optional.ofNullable(streamsRebalanceListenerInvoker().invokeTasksAssigned(assignment));
final Optional<Exception> exceptionFromCallback = streamsRebalanceListener().onTasksAssigned(assignment); final Optional<KafkaException> error = exceptionFromCallback.map(e -> ConsumerUtils.maybeWrapAsKafkaException(e, "Task assignment callback throws an error"));
if (exceptionFromCallback.isPresent()) {
error = Optional.of(ConsumerUtils.maybeWrapAsKafkaException(exceptionFromCallback.get(), "Task assignment callback throws an error"));
} else {
error = Optional.empty();
streamsRebalanceData().setReconciledAssignment(assignment);
}
return new StreamsOnTasksAssignedCallbackCompletedEvent(future, error); return new StreamsOnTasksAssignedCallbackCompletedEvent(future, error);
} }
private StreamsOnAllTasksLostCallbackCompletedEvent invokeOnAllTasksLostCallback(final CompletableFuture<Void> future) { private StreamsOnAllTasksLostCallbackCompletedEvent invokeOnAllTasksLostCallback(final CompletableFuture<Void> future) {
final Optional<KafkaException> error; final Optional<Exception> exceptionFromCallback = Optional.ofNullable(streamsRebalanceListenerInvoker().invokeAllTasksLost());
final Optional<Exception> exceptionFromCallback = streamsRebalanceListener().onAllTasksLost(); final Optional<KafkaException> error = exceptionFromCallback.map(e -> ConsumerUtils.maybeWrapAsKafkaException(e, "All tasks lost callback throws an error"));
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);
}
return new StreamsOnAllTasksLostCallbackCompletedEvent(future, error); return new StreamsOnAllTasksLostCallbackCompletedEvent(future, error);
} }
private StreamsRebalanceData streamsRebalanceData() { private StreamsRebalanceListenerInvoker streamsRebalanceListenerInvoker() {
return streamsRebalanceData.orElseThrow( return streamsRebalanceListenerInvoker.orElseThrow(
() -> new IllegalStateException("Background event processor was not created to be used with Streams " +
"rebalance protocol events"));
}
private StreamsRebalanceListener streamsRebalanceListener() {
return streamsRebalanceListener.orElseThrow(
() -> new IllegalStateException("Background event processor was not created to be used with Streams " + () -> new IllegalStateException("Background event processor was not created to be used with Streams " +
"rebalance protocol events")); "rebalance protocol events"));
} }
@ -365,6 +328,7 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
private final WakeupTrigger wakeupTrigger = new WakeupTrigger(); private final WakeupTrigger wakeupTrigger = new WakeupTrigger();
private final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker; private final OffsetCommitCallbackInvoker offsetCommitCallbackInvoker;
private final ConsumerRebalanceListenerInvoker rebalanceListenerInvoker; private final ConsumerRebalanceListenerInvoker rebalanceListenerInvoker;
private final Optional<StreamsRebalanceListenerInvoker> streamsRebalanceListenerInvoker;
// Last triggered async commit future. Used to wait until all previous async commits are completed. // 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. // 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; private CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> lastPendingAsyncCommit = null;
@ -520,7 +484,9 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
time, time,
new RebalanceCallbackMetricsManager(metrics) 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); this.backgroundEventReaper = backgroundEventReaperFactory.build(logContext);
// The FetchCollector is only used on the application thread. // 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.time = time;
this.backgroundEventQueue = backgroundEventQueue; this.backgroundEventQueue = backgroundEventQueue;
this.rebalanceListenerInvoker = rebalanceListenerInvoker; this.rebalanceListenerInvoker = rebalanceListenerInvoker;
this.streamsRebalanceListenerInvoker = Optional.empty();
this.backgroundEventProcessor = new BackgroundEventProcessor(); this.backgroundEventProcessor = new BackgroundEventProcessor();
this.backgroundEventReaper = backgroundEventReaper; this.backgroundEventReaper = backgroundEventReaper;
this.metrics = metrics; this.metrics = metrics;
@ -706,6 +673,7 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
networkClientDelegateSupplier, networkClientDelegateSupplier,
requestManagersSupplier, requestManagersSupplier,
asyncConsumerMetrics); asyncConsumerMetrics);
this.streamsRebalanceListenerInvoker = Optional.empty();
this.backgroundEventProcessor = new BackgroundEventProcessor(); this.backgroundEventProcessor = new BackgroundEventProcessor();
this.backgroundEventReaper = new CompletableEventReaper(logContext); this.backgroundEventReaper = new CompletableEventReaper(logContext);
} }
@ -1493,7 +1461,7 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
() -> autoCommitOnClose(closeTimer), firstException); () -> autoCommitOnClose(closeTimer), firstException);
swallow(log, Level.ERROR, "Failed to stop finding coordinator", swallow(log, Level.ERROR, "Failed to stop finding coordinator",
this::stopFindCoordinatorOnClose, firstException); this::stopFindCoordinatorOnClose, firstException);
swallow(log, Level.ERROR, "Failed to release group assignment", swallow(log, Level.ERROR, "Failed to run rebalance callbacks",
this::runRebalanceCallbacksOnClose, firstException); this::runRebalanceCallbacksOnClose, firstException);
swallow(log, Level.ERROR, "Failed to leave group while closing consumer", swallow(log, Level.ERROR, "Failed to leave group while closing consumer",
() -> leaveGroupOnClose(closeTimer, membershipOperation), firstException); () -> leaveGroupOnClose(closeTimer, membershipOperation), firstException);
@ -1543,11 +1511,23 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
} }
private void runRebalanceCallbacksOnClose() { private void runRebalanceCallbacksOnClose() {
if (groupMetadata.get().isEmpty() || rebalanceListenerInvoker == null) if (groupMetadata.get().isEmpty())
return; return;
int memberEpoch = groupMetadata.get().get().generationId(); 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(); Set<TopicPartition> assignedPartitions = groupAssignmentSnapshot.get();
if (assignedPartitions.isEmpty()) if (assignedPartitions.isEmpty())
@ -1557,12 +1537,13 @@ public class AsyncKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
SortedSet<TopicPartition> droppedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); SortedSet<TopicPartition> droppedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR);
droppedPartitions.addAll(assignedPartitions); droppedPartitions.addAll(assignedPartitions);
final Exception error; if (memberEpoch > 0) {
if (memberEpoch > 0)
error = rebalanceListenerInvoker.invokePartitionsRevoked(droppedPartitions); error = rebalanceListenerInvoker.invokePartitionsRevoked(droppedPartitions);
else } else {
error = rebalanceListenerInvoker.invokePartitionsLost(droppedPartitions); error = rebalanceListenerInvoker.invokePartitionsLost(droppedPartitions);
}
}
if (error != null) if (error != null)
throw ConsumerUtils.maybeWrapAsKafkaException(error); 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) { 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()); subscribeInternal(topics, Optional.empty());
backgroundEventProcessor.setStreamsRebalanceListener(streamsRebalanceListener);
} }
@Override @Override

View File

@ -541,7 +541,7 @@ public class CommitRequestManager implements RequestManager, MemberStateListener
boolean inflightRemoved = pendingRequests.inflightOffsetFetches.remove(fetchRequest); boolean inflightRemoved = pendingRequests.inflightOffsetFetches.remove(fetchRequest);
if (!inflightRemoved) { if (!inflightRemoved) {
log.warn("A duplicated, inflight, request was identified, but unable to find it in the " + log.warn("A duplicated, inflight, request was identified, but unable to find it in the " +
"outbound buffer:" + fetchRequest); "outbound buffer: {}", fetchRequest);
} }
if (error == null) { if (error == null) {
maybeUpdateLastSeenEpochIfNewer(res); maybeUpdateLastSeenEpochIfNewer(res);

View File

@ -163,7 +163,7 @@ public class ConsumerMembershipManager extends AbstractMembershipManager<Consume
logContext, logContext,
backgroundEventHandler, backgroundEventHandler,
time, time,
new ConsumerRebalanceMetricsManager(metrics), new ConsumerRebalanceMetricsManager(metrics, subscriptions),
autoCommitEnabled); autoCommitEnabled);
} }

View File

@ -287,7 +287,7 @@ public class StreamsGroupHeartbeatRequestManager implements RequestManager {
private final HeartbeatMetricsManager metricsManager; 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 * Timer for tracking the time since the last consumer poll. If the timer expires, the consumer will stop

View File

@ -296,7 +296,7 @@ public class StreamsMembershipManager implements RequestManager {
this.backgroundEventHandler = backgroundEventHandler; this.backgroundEventHandler = backgroundEventHandler;
this.streamsRebalanceData = streamsRebalanceData; this.streamsRebalanceData = streamsRebalanceData;
this.subscriptionState = subscriptionState; this.subscriptionState = subscriptionState;
metricsManager = new ConsumerRebalanceMetricsManager(metrics); metricsManager = new ConsumerRebalanceMetricsManager(metrics, subscriptionState);
this.time = time; this.time = time;
} }
@ -1010,8 +1010,8 @@ public class StreamsMembershipManager implements RequestManager {
return; return;
} }
if (reconciliationInProgress) { if (reconciliationInProgress) {
log.trace("Ignoring reconciliation attempt. Another reconciliation is already in progress. Assignment " + log.trace("Ignoring reconciliation attempt. Another reconciliation is already in progress. Assignment {}" +
targetAssignment + " will be handled in the next reconciliation loop."); " will be handled in the next reconciliation loop.", targetAssignment);
return; return;
} }
@ -1131,12 +1131,12 @@ public class StreamsMembershipManager implements RequestManager {
); );
final SortedSet<TopicPartition> partitionsToAssign = topicPartitionsForActiveTasks(activeTasksToAssign); final SortedSet<TopicPartition> partitionsToAssign = topicPartitionsForActiveTasks(activeTasksToAssign);
final SortedSet<TopicPartition> partitionsToAssigneNotPreviouslyOwned = final SortedSet<TopicPartition> partitionsToAssignNotPreviouslyOwned =
partitionsToAssignNotPreviouslyOwned(partitionsToAssign, topicPartitionsForActiveTasks(ownedActiveTasks)); partitionsToAssignNotPreviouslyOwned(partitionsToAssign, topicPartitionsForActiveTasks(ownedActiveTasks));
subscriptionState.assignFromSubscribedAwaitingCallback( subscriptionState.assignFromSubscribedAwaitingCallback(
partitionsToAssign, partitionsToAssign,
partitionsToAssigneNotPreviouslyOwned partitionsToAssignNotPreviouslyOwned
); );
notifyAssignmentChange(partitionsToAssign); notifyAssignmentChange(partitionsToAssign);
@ -1152,10 +1152,10 @@ public class StreamsMembershipManager implements RequestManager {
if (callbackError == null) { if (callbackError == null) {
subscriptionState.enablePartitionsAwaitingCallback(partitionsToAssign); subscriptionState.enablePartitionsAwaitingCallback(partitionsToAssign);
} else { } else {
if (!partitionsToAssigneNotPreviouslyOwned.isEmpty()) { if (!partitionsToAssignNotPreviouslyOwned.isEmpty()) {
log.warn("Leaving newly assigned partitions {} marked as non-fetchable and not " + log.warn("Leaving newly assigned partitions {} marked as non-fetchable and not " +
"requiring initializing positions after onTasksAssigned callback failed.", "requiring initializing positions after onTasksAssigned callback failed.",
partitionsToAssigneNotPreviouslyOwned, callbackError); partitionsToAssignNotPreviouslyOwned, callbackError);
} }
} }
}); });
@ -1205,9 +1205,9 @@ public class StreamsMembershipManager implements RequestManager {
Stream.concat( Stream.concat(
streamsRebalanceData.subtopologies().get(task.subtopologyId()).sourceTopics().stream(), streamsRebalanceData.subtopologies().get(task.subtopologyId()).sourceTopics().stream(),
streamsRebalanceData.subtopologies().get(task.subtopologyId()).repartitionSourceTopics().keySet().stream() streamsRebalanceData.subtopologies().get(task.subtopologyId()).repartitionSourceTopics().keySet().stream()
).forEach(topic -> { ).forEach(topic ->
topicPartitions.add(new TopicPartition(topic, task.partitionId())); topicPartitions.add(new TopicPartition(topic, task.partitionId()))
}) )
); );
return topicPartitions; return topicPartitions;
} }
@ -1223,7 +1223,7 @@ public class StreamsMembershipManager implements RequestManager {
String reason = rejoinedWhileReconciliationInProgress ? String reason = rejoinedWhileReconciliationInProgress ?
"the member has re-joined the group" : "the member has re-joined the group" :
"the member already transitioned out of the reconciling state into " + state; "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(); markReconciliationCompleted();
} }
return shouldAbort; return shouldAbort;

View File

@ -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;
}
}

View File

@ -477,7 +477,7 @@ public class SubscriptionState {
* Provides the number of assigned partitions in a thread safe manner. * Provides the number of assigned partitions in a thread safe manner.
* @return the number of assigned partitions. * @return the number of assigned partitions.
*/ */
synchronized int numAssignedPartitions() { public synchronized int numAssignedPartitions() {
return this.assignment.size(); return this.assignment.size();
} }

View File

@ -16,6 +16,8 @@
*/ */
package org.apache.kafka.clients.consumer.internals.metrics; 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.MetricName;
import org.apache.kafka.common.metrics.Measurable; import org.apache.kafka.common.metrics.Measurable;
import org.apache.kafka.common.metrics.Metrics; 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.Rate;
import org.apache.kafka.common.metrics.stats.WindowedCount; import org.apache.kafka.common.metrics.stats.WindowedCount;
import java.util.Collection;
import java.util.concurrent.TimeUnit; 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.CONSUMER_METRIC_GROUP_PREFIX;
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.COORDINATOR_METRICS_SUFFIX; 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 lastRebalanceSecondsAgo;
public final MetricName failedRebalanceTotal; public final MetricName failedRebalanceTotal;
public final MetricName failedRebalanceRate; public final MetricName failedRebalanceRate;
public final MetricName assignedPartitionsCount;
private long lastRebalanceEndMs = -1L; private long lastRebalanceEndMs = -1L;
private long lastRebalanceStartMs = -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); super(CONSUMER_METRIC_GROUP_PREFIX + COORDINATOR_METRICS_SUFFIX);
this.metrics = metrics;
rebalanceLatencyAvg = createMetric(metrics, "rebalance-latency-avg", rebalanceLatencyAvg = createMetric(metrics, "rebalance-latency-avg",
"The average time in ms taken for a group to complete a rebalance"); "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"); "The total number of failed rebalance events");
failedRebalanceRate = createMetric(metrics, "failed-rebalance-rate-per-hour", failedRebalanceRate = createMetric(metrics, "failed-rebalance-rate-per-hour",
"The number of failed rebalance events 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 = metrics.sensor("rebalance-latency");
successfulRebalanceSensor.add(rebalanceLatencyAvg, new Avg()); successfulRebalanceSensor.add(rebalanceLatencyAvg, new Avg());
@ -106,4 +116,15 @@ public final class ConsumerRebalanceMetricsManager extends RebalanceMetricsManag
public boolean rebalanceStarted() { public boolean rebalanceStarted() {
return lastRebalanceStartMs > lastRebalanceEndMs; 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);
}
} }

View File

@ -553,7 +553,7 @@ public final class Metrics implements Closeable {
try { try {
reporter.metricRemoval(metric); reporter.metricRemoval(metric);
} catch (Exception e) { } 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); log.trace("Removed metric named {}", metricName);
@ -596,7 +596,7 @@ public final class Metrics implements Closeable {
try { try {
reporter.metricChange(metric); reporter.metricChange(metric);
} catch (Exception e) { } 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); log.trace("Registered metric named {}", metricName);
@ -688,7 +688,7 @@ public final class Metrics implements Closeable {
log.info("Closing reporter {}", reporter.getClass().getName()); log.info("Closing reporter {}", reporter.getClass().getName());
reporter.close(); reporter.close();
} catch (Exception e) { } 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"); log.info("Metrics reporters closed");

View File

@ -40,7 +40,7 @@ public final class ByteBufferUnmapper {
private static final RuntimeException UNMAP_NOT_SUPPORTED_EXCEPTION; private static final RuntimeException UNMAP_NOT_SUPPORTED_EXCEPTION;
static { static {
Object unmap = null; MethodHandle unmap = null;
RuntimeException exception = null; RuntimeException exception = null;
try { try {
unmap = lookupUnmapMethodHandle(); unmap = lookupUnmapMethodHandle();
@ -48,7 +48,7 @@ public final class ByteBufferUnmapper {
exception = e; exception = e;
} }
if (unmap != null) { if (unmap != null) {
UNMAP = (MethodHandle) unmap; UNMAP = unmap;
UNMAP_NOT_SUPPORTED_EXCEPTION = null; UNMAP_NOT_SUPPORTED_EXCEPTION = null;
} else { } else {
UNMAP = null; UNMAP = null;

View File

@ -75,7 +75,8 @@ public class ConfigUtils {
} else if (value instanceof String) { } else if (value instanceof String) {
return Boolean.parseBoolean((String) value); return Boolean.parseBoolean((String) value);
} else { } 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; return defaultValue;
} }
} }

View File

@ -69,7 +69,7 @@ public class LoggingSignalHandler {
for (String signal : SIGNALS) { for (String signal : SIGNALS) {
register(signal, jvmSignalHandlers); 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) { private Object createSignalHandler(final Map<String, Object> jvmSignalHandlers) {

View File

@ -33,7 +33,7 @@ import java.util.Map;
public class SecurityUtils { 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, ResourceType> NAME_TO_RESOURCE_TYPES;
private static final Map<String, AclOperation> NAME_TO_OPERATIONS; private static final Map<String, AclOperation> NAME_TO_OPERATIONS;

View File

@ -857,7 +857,7 @@ public final class Utils {
public static void delete(final File rootFile) throws IOException { public static void delete(final File rootFile) throws IOException {
if (rootFile == null) if (rootFile == null)
return; return;
Files.walkFileTree(rootFile.toPath(), new SimpleFileVisitor<Path>() { Files.walkFileTree(rootFile.toPath(), new SimpleFileVisitor<>() {
@Override @Override
public FileVisitResult visitFileFailed(Path path, IOException exc) throws IOException { public FileVisitResult visitFileFailed(Path path, IOException exc) throws IOException {
if (exc instanceof NoSuchFileException) { if (exc instanceof NoSuchFileException) {
@ -1403,7 +1403,7 @@ public final class Utils {
* @return new Collector<Map.Entry<K, V>, M, M> * @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) { 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 @Override
public Supplier<M> supplier() { public Supplier<M> supplier() {
return mapSupplier; return mapSupplier;

View File

@ -33,7 +33,7 @@
{ "name": "PartitionIndex", "type": "int32", "versions": "0+", { "name": "PartitionIndex", "type": "int32", "versions": "0+",
"about": "The partition index." }, "about": "The partition index." },
{ "name": "Offset", "type": "int64", "versions": "0+", { "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+", { "name": "TimeoutMs", "type": "int32", "versions": "0+",

View File

@ -155,6 +155,8 @@ import javax.management.ObjectName;
import static java.util.Collections.singletonList; 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.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.requests.FetchMetadata.INVALID_SESSION_ID;
import static org.apache.kafka.common.utils.Utils.propsToMap; import static org.apache.kafka.common.utils.Utils.propsToMap;
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; 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 @ParameterizedTest
@EnumSource(GroupProtocol.class) @EnumSource(GroupProtocol.class)
public void testUnsubscribingCustomMetricsWithSameNameDoesntAffectConsumerMetrics(GroupProtocol groupProtocol) { public void testUnsubscribingCustomMetricsWithSameNameDoesntAffectConsumerMetrics(GroupProtocol groupProtocol) {

View File

@ -2221,6 +2221,73 @@ public class AsyncKafkaConsumerTest {
}).when(applicationEventHandler).add(ArgumentMatchers.isA(CommitEvent.class)); }).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() { private void markReconcileAndAutoCommitCompleteForPollEvent() {
doAnswer(invocation -> { doAnswer(invocation -> {
PollEvent event = invocation.getArgument(0); PollEvent event = invocation.getArgument(0);

View File

@ -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.AbstractMembershipManager.TOPIC_PARTITION_COMPARATOR;
import static org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer.invokeRebalanceCallbacks; 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.requests.ConsumerGroupHeartbeatRequest.LEAVE_GROUP_MEMBER_EPOCH;
import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkEntry;
import static org.apache.kafka.common.utils.Utils.mkMap; import static org.apache.kafka.common.utils.Utils.mkMap;
@ -125,7 +127,7 @@ public class ConsumerMembershipManagerTest {
time = new MockTime(0); time = new MockTime(0);
backgroundEventHandler = new BackgroundEventHandler(backgroundEventQueue, time, mock(AsyncConsumerMetrics.class)); backgroundEventHandler = new BackgroundEventHandler(backgroundEventQueue, time, mock(AsyncConsumerMetrics.class));
metrics = new Metrics(time); metrics = new Metrics(time);
rebalanceMetricsManager = new ConsumerRebalanceMetricsManager(metrics); rebalanceMetricsManager = new ConsumerRebalanceMetricsManager(metrics, subscriptionState);
when(commitRequestManager.maybeAutoCommitSyncBeforeRebalance(anyLong())).thenReturn(CompletableFuture.completedFuture(null)); when(commitRequestManager.maybeAutoCommitSyncBeforeRebalance(anyLong())).thenReturn(CompletableFuture.completedFuture(null));
} }
@ -181,6 +183,15 @@ public class ConsumerMembershipManagerTest {
assertEquals(Optional.of("rack1"), membershipManager.rackId()); 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 @Test
public void testMembershipManagerInitSupportsEmptyGroupInstanceId() { public void testMembershipManagerInitSupportsEmptyGroupInstanceId() {
createMembershipManagerJoiningGroup(); createMembershipManagerJoiningGroup();

View File

@ -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.assertFalse;
import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertInstanceOf;
import static org.junit.jupiter.api.Assertions.assertNotEquals; 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.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.any;
@ -131,6 +132,15 @@ public class StreamsMembershipManagerTest {
verifyInStateUnsubscribed(membershipManager); 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 @Test
public void testUnexpectedErrorInHeartbeatResponse() { public void testUnexpectedErrorInHeartbeatResponse() {
final String errorMessage = "Nobody expects the Spanish Inquisition!"; final String errorMessage = "Nobody expects the Spanish Inquisition!";

View File

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

View File

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

View File

@ -318,7 +318,7 @@ public class KafkaFutureTest {
awaitAndAssertResult(future, 21, null); awaitAndAssertResult(future, 21, null);
Throwable cause = awaitAndAssertFailure(dependantFuture, CompletionException.class, "java.lang.RuntimeException: We require more vespene gas"); Throwable cause = awaitAndAssertFailure(dependantFuture, CompletionException.class, "java.lang.RuntimeException: We require more vespene gas");
assertInstanceOf(RuntimeException.class, cause.getCause()); assertInstanceOf(RuntimeException.class, cause.getCause());
assertEquals(cause.getCause().getMessage(), "We require more vespene gas"); assertEquals("We require more vespene gas", cause.getCause().getMessage());
} }
@Test @Test

View File

@ -35,8 +35,8 @@ public class UuidTest {
public void testSignificantBits() { public void testSignificantBits() {
Uuid id = new Uuid(34L, 98L); Uuid id = new Uuid(34L, 98L);
assertEquals(id.getMostSignificantBits(), 34L); assertEquals(34L, id.getMostSignificantBits());
assertEquals(id.getLeastSignificantBits(), 98L); assertEquals(98L, id.getLeastSignificantBits());
} }
@Test @Test
@ -74,15 +74,15 @@ public class UuidTest {
String zeroIdString = Uuid.ZERO_UUID.toString(); 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) @RepeatedTest(value = 100, name = RepeatedTest.LONG_DISPLAY_NAME)
public void testRandomUuid() { public void testRandomUuid() {
Uuid randomID = Uuid.randomUuid(); Uuid randomID = Uuid.randomUuid();
assertNotEquals(randomID, Uuid.ZERO_UUID); assertNotEquals(Uuid.ZERO_UUID, randomID);
assertNotEquals(randomID, Uuid.METADATA_TOPIC_ID); assertNotEquals(Uuid.METADATA_TOPIC_ID, randomID);
assertFalse(randomID.toString().startsWith("-")); assertFalse(randomID.toString().startsWith("-"));
} }

View File

@ -379,8 +379,8 @@ public class AbstractConfigTest {
Properties props = new Properties(); Properties props = new Properties();
props.put("config.providers", "file"); props.put("config.providers", "file");
TestIndirectConfigResolution config = new TestIndirectConfigResolution(props); TestIndirectConfigResolution config = new TestIndirectConfigResolution(props);
assertEquals(config.originals().get("config.providers"), "file"); assertEquals("file", config.originals().get("config.providers"));
assertEquals(config.originals(Collections.singletonMap("config.providers", "file2")).get("config.providers"), "file2"); assertEquals("file2", config.originals(Collections.singletonMap("config.providers", "file2")).get("config.providers"));
} }
@Test @Test

View File

@ -62,10 +62,10 @@ public class ChannelBuildersTest {
assertNull(configs.get("listener.name.listener1.gssapi.sasl.kerberos.service.name")); assertNull(configs.get("listener.name.listener1.gssapi.sasl.kerberos.service.name"));
assertFalse(securityConfig.unused().contains("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")); 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")); assertFalse(securityConfig.unused().contains("sasl.kerberos.service.name"));
assertNull(configs.get("listener.name.listener1.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")); assertNull(configs.get("plain.sasl.server.callback.handler.class"));
assertFalse(securityConfig.unused().contains("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")); 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")); assertFalse(securityConfig.unused().contains("custom.config2.key"));
// test configs without listener prefix // test configs without listener prefix
securityConfig = new TestSecurityConfig(props); securityConfig = new TestSecurityConfig(props);
configs = ChannelBuilders.channelBuilderConfigs(securityConfig, null); 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")); assertFalse(securityConfig.unused().contains("listener.name.listener1.gssapi.sasl.kerberos.service.name"));
assertNull(configs.get("gssapi.sasl.kerberos.service.name")); assertNull(configs.get("gssapi.sasl.kerberos.service.name"));
assertFalse(securityConfig.unused().contains("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")); assertFalse(securityConfig.unused().contains("listener.name.listener1.sasl.kerberos.service.name"));
assertNull(configs.get("sasl.kerberos.service.name")); assertNull(configs.get("sasl.kerberos.service.name"));
assertFalse(securityConfig.unused().contains("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")); 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")); 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")); assertFalse(securityConfig.unused().contains("custom.config2.key"));
} }

View File

@ -53,7 +53,7 @@ public class ErrorsTest {
public void testExceptionsAreNotGeneric() { public void testExceptionsAreNotGeneric() {
for (Errors error : Errors.values()) { for (Errors error : Errors.values()) {
if (error != Errors.NONE) 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");
} }
} }

View File

@ -197,7 +197,7 @@ public class FileRecordsTest {
* Test a simple append and read. * Test a simple append and read.
*/ */
@Test @Test
public void testRead() throws IOException { public void testRead() {
FileRecords read = fileRecords.slice(0, fileRecords.sizeInBytes()); FileRecords read = fileRecords.slice(0, fileRecords.sizeInBytes());
assertEquals(fileRecords.sizeInBytes(), read.sizeInBytes()); assertEquals(fileRecords.sizeInBytes(), read.sizeInBytes());
TestUtils.checkEquals(fileRecords.batches(), read.batches()); 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 that the message set iterator obeys start and end slicing
*/ */
@Test @Test
public void testIteratorWithLimits() throws IOException { public void testIteratorWithLimits() {
RecordBatch batch = batches(fileRecords).get(1); RecordBatch batch = batches(fileRecords).get(1);
int start = fileRecords.searchForOffsetFromPosition(1, 0).position; int start = fileRecords.searchForOffsetFromPosition(1, 0).position;
int size = batch.sizeInBytes(); int size = batch.sizeInBytes();

View File

@ -46,8 +46,8 @@ public class ReplicaSelectorTest {
ReplicaSelector selector = new RackAwareReplicaSelector(); ReplicaSelector selector = new RackAwareReplicaSelector();
Optional<ReplicaView> selected = selector.select(tp, metadata("rack-b"), partitionView); Optional<ReplicaView> selected = selector.select(tp, metadata("rack-b"), partitionView);
assertOptional(selected, replicaInfo -> { assertOptional(selected, replicaInfo -> {
assertEquals(replicaInfo.endpoint().rack(), "rack-b", "Expect replica to be in rack-b"); assertEquals("rack-b", replicaInfo.endpoint().rack(), "Expect replica to be in rack-b");
assertEquals(replicaInfo.endpoint().id(), 3, "Expected replica 3 since it is more caught-up"); assertEquals(3, replicaInfo.endpoint().id(), "Expected replica 3 since it is more caught-up");
}); });
selected = selector.select(tp, metadata("not-a-rack"), partitionView); selected = selector.select(tp, metadata("not-a-rack"), partitionView);
@ -57,7 +57,7 @@ public class ReplicaSelectorTest {
selected = selector.select(tp, metadata("rack-a"), partitionView); selected = selector.select(tp, metadata("rack-a"), partitionView);
assertOptional(selected, replicaInfo -> { 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"); assertEquals(replicaInfo, leader, "Expect the leader since it's in rack-a");
}); });

View File

@ -50,6 +50,20 @@ import java.util.function.Function;
*/ */
public class CoordinatorLoaderImpl<T> implements CoordinatorLoader<T> { 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 static final Logger LOG = LoggerFactory.getLogger(CoordinatorLoaderImpl.class);
private final Time time; private final Time time;
@ -57,6 +71,7 @@ public class CoordinatorLoaderImpl<T> implements CoordinatorLoader<T> {
private final Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier; private final Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier;
private final Deserializer<T> deserializer; private final Deserializer<T> deserializer;
private final int loadBufferSize; private final int loadBufferSize;
private final long commitIntervalOffsets;
private final AtomicBoolean isRunning = new AtomicBoolean(true); private final AtomicBoolean isRunning = new AtomicBoolean(true);
private final KafkaScheduler scheduler = new KafkaScheduler(1); 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<UnifiedLog>> partitionLogSupplier,
Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier, Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier,
Deserializer<T> deserializer, Deserializer<T> deserializer,
int loadBufferSize int loadBufferSize,
long commitIntervalOffsets
) { ) {
this.time = time; this.time = time;
this.partitionLogSupplier = partitionLogSupplier; this.partitionLogSupplier = partitionLogSupplier;
this.partitionLogEndOffsetSupplier = partitionLogEndOffsetSupplier; this.partitionLogEndOffsetSupplier = partitionLogEndOffsetSupplier;
this.deserializer = deserializer; this.deserializer = deserializer;
this.loadBufferSize = loadBufferSize; this.loadBufferSize = loadBufferSize;
this.commitIntervalOffsets = commitIntervalOffsets;
this.scheduler.startup(); this.scheduler.startup();
} }
@ -121,7 +138,7 @@ public class CoordinatorLoaderImpl<T> implements CoordinatorLoader<T> {
long currentOffset = log.logStartOffset(); long currentOffset = log.logStartOffset();
LoadStats stats = new LoadStats(); LoadStats stats = new LoadStats();
long previousHighWatermark = -1L; long lastCommittedOffset = -1L;
while (shouldFetchNextBatch(currentOffset, logEndOffset(tp), stats.readAtLeastOneRecord)) { while (shouldFetchNextBatch(currentOffset, logEndOffset(tp), stats.readAtLeastOneRecord)) {
FetchDataInfo fetchDataInfo = log.read(currentOffset, loadBufferSize, FetchIsolation.LOG_END, true); FetchDataInfo fetchDataInfo = log.read(currentOffset, loadBufferSize, FetchIsolation.LOG_END, true);
@ -133,17 +150,16 @@ public class CoordinatorLoaderImpl<T> implements CoordinatorLoader<T> {
buffer = memoryRecords.buffer(); 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; currentOffset = replayResult.nextOffset;
previousHighWatermark = replayResult.highWatermark; lastCommittedOffset = replayResult.lastCommittedOffset;
} }
long endTimeMs = time.milliseconds(); long endTimeMs = time.milliseconds();
if (logEndOffset(tp) == -1L) { if (logEndOffset(tp) == -1L) {
future.completeExceptionally(new NotLeaderOrFollowerException( 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()) { } else if (isRunning.get()) {
future.complete(new LoadSummary(startTimeMs, endTimeMs, schedulerQueueTimeMs, stats.numRecords, stats.numBytes)); future.complete(new LoadSummary(startTimeMs, endTimeMs, schedulerQueueTimeMs, stats.numRecords, stats.numBytes));
} else { } else {
@ -208,9 +224,8 @@ public class CoordinatorLoaderImpl<T> implements CoordinatorLoader<T> {
CoordinatorPlayback<T> coordinator, CoordinatorPlayback<T> coordinator,
LoadStats loadStats, LoadStats loadStats,
long currentOffset, long currentOffset,
long previousHighWatermark long lastCommittedOffset
) { ) {
for (MutableRecordBatch batch : memoryRecords.batches()) { for (MutableRecordBatch batch : memoryRecords.batches()) {
if (batch.isControlBatch()) { if (batch.isControlBatch()) {
for (Record record : batch) { for (Record record : batch) {
@ -288,14 +303,18 @@ public class CoordinatorLoaderImpl<T> implements CoordinatorLoader<T> {
if (currentOffset >= currentHighWatermark) { if (currentOffset >= currentHighWatermark) {
coordinator.updateLastWrittenOffset(currentOffset); coordinator.updateLastWrittenOffset(currentOffset);
if (currentHighWatermark > previousHighWatermark) { if (currentHighWatermark > lastCommittedOffset) {
coordinator.updateLastCommittedOffset(currentHighWatermark); coordinator.updateLastCommittedOffset(currentHighWatermark);
previousHighWatermark = currentHighWatermark; lastCommittedOffset = currentHighWatermark;
} }
} else if (currentOffset - lastCommittedOffset >= commitIntervalOffsets) {
coordinator.updateLastWrittenOffset(currentOffset);
coordinator.updateLastCommittedOffset(currentOffset);
lastCommittedOffset = currentOffset;
} }
} }
loadStats.numBytes += memoryRecords.sizeInBytes(); 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 @Override
public String toString() { public String toString() {
return "LoadStats{" + return "LoadStats(" +
"numRecords=" + numRecords + "numRecords=" + numRecords +
", numBytes=" + numBytes + ", numBytes=" + numBytes +
", readAtLeastOneRecord=" + readAtLeastOneRecord + ", readAtLeastOneRecord=" + readAtLeastOneRecord +
'}'; ')';
} }
} }
private record ReplayResult(long nextOffset, long highWatermark) { private record ReplayResult(long nextOffset, long lastCommittedOffset) { }
}
} }

View File

@ -61,7 +61,7 @@ public class SnapshottableCoordinator<S extends CoordinatorShard<U>, U> implemen
*/ */
private long lastCommittedOffset; private long lastCommittedOffset;
SnapshottableCoordinator( public SnapshottableCoordinator(
LogContext logContext, LogContext logContext,
SnapshotRegistry snapshotRegistry, SnapshotRegistry snapshotRegistry,
S coordinator, S coordinator,

View File

@ -91,7 +91,8 @@ class CoordinatorLoaderImplTest {
partitionLogSupplier, partitionLogSupplier,
partitionLogEndOffsetSupplier, partitionLogEndOffsetSupplier,
serde, serde,
1000 1000,
CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS
)) { )) {
assertFutureThrows(NotLeaderOrFollowerException.class, loader.load(tp, coordinator)); assertFutureThrows(NotLeaderOrFollowerException.class, loader.load(tp, coordinator));
} }
@ -110,7 +111,8 @@ class CoordinatorLoaderImplTest {
partitionLogSupplier, partitionLogSupplier,
partitionLogEndOffsetSupplier, partitionLogEndOffsetSupplier,
serde, serde,
1000 1000,
CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS
)) { )) {
loader.close(); loader.close();
assertFutureThrows(RuntimeException.class, loader.load(tp, coordinator)); assertFutureThrows(RuntimeException.class, loader.load(tp, coordinator));
@ -131,7 +133,8 @@ class CoordinatorLoaderImplTest {
partitionLogSupplier, partitionLogSupplier,
partitionLogEndOffsetSupplier, partitionLogEndOffsetSupplier,
serde, serde,
1000 1000,
CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS
)) { )) {
when(log.logStartOffset()).thenReturn(0L); when(log.logStartOffset()).thenReturn(0L);
when(log.highWatermark()).thenReturn(0L); when(log.highWatermark()).thenReturn(0L);
@ -217,7 +220,8 @@ class CoordinatorLoaderImplTest {
partitionLogSupplier, partitionLogSupplier,
partitionLogEndOffsetSupplier, partitionLogEndOffsetSupplier,
serde, serde,
1000 1000,
CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS
)) { )) {
when(log.logStartOffset()).thenReturn(0L); when(log.logStartOffset()).thenReturn(0L);
@ -262,7 +266,8 @@ class CoordinatorLoaderImplTest {
partitionLogSupplier, partitionLogSupplier,
partitionLogEndOffsetSupplier, partitionLogEndOffsetSupplier,
serde, serde,
1000 1000,
CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS
)) { )) {
when(log.logStartOffset()).thenReturn(0L); when(log.logStartOffset()).thenReturn(0L);
@ -298,7 +303,8 @@ class CoordinatorLoaderImplTest {
partitionLogSupplier, partitionLogSupplier,
partitionLogEndOffsetSupplier, partitionLogEndOffsetSupplier,
serde, serde,
1000 1000,
CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS
)) { )) {
when(log.logStartOffset()).thenReturn(0L); when(log.logStartOffset()).thenReturn(0L);
@ -337,7 +343,8 @@ class CoordinatorLoaderImplTest {
partitionLogSupplier, partitionLogSupplier,
partitionLogEndOffsetSupplier, partitionLogEndOffsetSupplier,
serde, serde,
1000 1000,
CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS
)) { )) {
when(log.logStartOffset()).thenReturn(0L); when(log.logStartOffset()).thenReturn(0L);
@ -365,7 +372,8 @@ class CoordinatorLoaderImplTest {
partitionLogSupplier, partitionLogSupplier,
partitionLogEndOffsetSupplier, partitionLogEndOffsetSupplier,
serde, serde,
1000 1000,
CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS
)) { )) {
long startTimeMs = time.milliseconds(); long startTimeMs = time.milliseconds();
when(log.logStartOffset()).thenReturn(0L); when(log.logStartOffset()).thenReturn(0L);
@ -412,7 +420,8 @@ class CoordinatorLoaderImplTest {
partitionLogSupplier, partitionLogSupplier,
partitionLogEndOffsetSupplier, partitionLogEndOffsetSupplier,
serde, serde,
1000 1000,
CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS
)) { )) {
when(log.logStartOffset()).thenReturn(0L); when(log.logStartOffset()).thenReturn(0L);
when(log.highWatermark()).thenReturn(0L, 0L, 2L); when(log.highWatermark()).thenReturn(0L, 0L, 2L);
@ -475,7 +484,8 @@ class CoordinatorLoaderImplTest {
partitionLogSupplier, partitionLogSupplier,
partitionLogEndOffsetSupplier, partitionLogEndOffsetSupplier,
serde, serde,
1000 1000,
CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS
)) { )) {
when(log.logStartOffset()).thenReturn(0L); when(log.logStartOffset()).thenReturn(0L);
when(log.highWatermark()).thenReturn(0L); when(log.highWatermark()).thenReturn(0L);
@ -501,7 +511,8 @@ class CoordinatorLoaderImplTest {
partitionLogSupplier, partitionLogSupplier,
partitionLogEndOffsetSupplier, partitionLogEndOffsetSupplier,
serde, serde,
1000 1000,
CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS
)) { )) {
when(log.logStartOffset()).thenReturn(0L); when(log.logStartOffset()).thenReturn(0L);
when(log.highWatermark()).thenReturn(5L, 7L, 7L); 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 @Test
void testPartitionGoesOfflineDuringLoad() throws Exception { void testPartitionGoesOfflineDuringLoad() throws Exception {
TopicPartition tp = new TopicPartition("foo", 0); TopicPartition tp = new TopicPartition("foo", 0);
@ -565,7 +649,8 @@ class CoordinatorLoaderImplTest {
partitionLogSupplier, partitionLogSupplier,
partitionLogEndOffsetSupplier, partitionLogEndOffsetSupplier,
serde, serde,
1000 1000,
CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS
)) { )) {
when(log.logStartOffset()).thenReturn(0L); when(log.logStartOffset()).thenReturn(0L);
when(log.highWatermark()).thenReturn(0L); when(log.highWatermark()).thenReturn(0L);

View File

@ -20,16 +20,13 @@ import java.io.File
import java.net.InetSocketAddress import java.net.InetSocketAddress
import java.nio.file.Files import java.nio.file.Files
import java.nio.file.Paths 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.concurrent.CompletableFuture
import java.util.{Map => JMap}
import java.util.{Collection => JCollection}
import kafka.server.KafkaConfig import kafka.server.KafkaConfig
import kafka.utils.CoreUtils import kafka.utils.CoreUtils
import kafka.utils.Logging import kafka.utils.Logging
import org.apache.kafka.clients.{ApiVersions, ManualMetadataUpdater, MetadataRecoveryStrategy, NetworkClient} import org.apache.kafka.clients.{ApiVersions, ManualMetadataUpdater, MetadataRecoveryStrategy, NetworkClient}
import org.apache.kafka.common.KafkaException import org.apache.kafka.common.KafkaException
import org.apache.kafka.common.Node
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.Uuid import org.apache.kafka.common.Uuid
import org.apache.kafka.common.metrics.Metrics 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.JaasContext
import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.utils.{LogContext, Time, Utils} 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.ProcessRole
import org.apache.kafka.server.common.Feature import org.apache.kafka.server.common.Feature
import org.apache.kafka.server.common.serialization.RecordSerde 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 org.apache.kafka.storage.internals.log.{LogManager, UnifiedLog}
import scala.jdk.CollectionConverters._ import scala.jdk.CollectionConverters._
import scala.jdk.OptionConverters._
object KafkaRaftManager { object KafkaRaftManager {
private def createLogDirectory(logDir: File, logDirName: String): File = { 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]( class KafkaRaftManager[T](
clusterId: String, clusterId: String,
config: KafkaConfig, config: KafkaConfig,
@ -178,12 +151,6 @@ class KafkaRaftManager[T](
CoreUtils.swallow(dataDirLock.foreach(_.destroy()), this) CoreUtils.swallow(dataDirLock.foreach(_.destroy()), this)
} }
override def register(
listener: RaftClient.Listener[T]
): Unit = {
client.register(listener)
}
override def handleRequest( override def handleRequest(
context: RequestContext, context: RequestContext,
header: RequestHeader, header: RequestHeader,
@ -292,13 +259,5 @@ class KafkaRaftManager[T](
(controllerListenerName, networkClient) (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 override def recordSerde: RecordSerde[T] = serde
} }

View File

@ -18,6 +18,7 @@
package kafka.server package kafka.server
import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.ConcurrentHashMap
import java.util.concurrent.locks.ReentrantLock
import java.util.{Collections, Properties} import java.util.{Collections, Properties}
import kafka.coordinator.transaction.TransactionCoordinator import kafka.coordinator.transaction.TransactionCoordinator
import kafka.utils.Logging 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.coordinator.transaction.TransactionLogConfig
import org.apache.kafka.server.common.{ControllerRequestCompletionHandler, NodeToControllerChannelManager} import org.apache.kafka.server.common.{ControllerRequestCompletionHandler, NodeToControllerChannelManager}
import org.apache.kafka.server.quota.ControllerMutationQuota import org.apache.kafka.server.quota.ControllerMutationQuota
import org.apache.kafka.common.utils.Time
import scala.collection.{Map, Seq, Set, mutable} import scala.collection.{Map, Seq, Set, mutable}
import scala.jdk.CollectionConverters._ import scala.jdk.CollectionConverters._
@ -50,21 +52,96 @@ trait AutoTopicCreationManager {
def createStreamsInternalTopics( def createStreamsInternalTopics(
topics: Map[String, CreatableTopic], topics: Map[String, CreatableTopic],
requestContext: RequestContext requestContext: RequestContext,
timeoutMs: Long
): Unit ): 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( class DefaultAutoTopicCreationManager(
config: KafkaConfig, config: KafkaConfig,
channelManager: NodeToControllerChannelManager, channelManager: NodeToControllerChannelManager,
groupCoordinator: GroupCoordinator, groupCoordinator: GroupCoordinator,
txnCoordinator: TransactionCoordinator, txnCoordinator: TransactionCoordinator,
shareCoordinator: ShareCoordinator shareCoordinator: ShareCoordinator,
time: Time,
topicErrorCacheCapacity: Int = 1000
) extends AutoTopicCreationManager with Logging { ) extends AutoTopicCreationManager with Logging {
private val inflightTopics = Collections.newSetFromMap(new ConcurrentHashMap[String, java.lang.Boolean]()) 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. * Initiate auto topic creation for the given topics.
* *
@ -93,13 +170,21 @@ class DefaultAutoTopicCreationManager(
override def createStreamsInternalTopics( override def createStreamsInternalTopics(
topics: Map[String, CreatableTopic], topics: Map[String, CreatableTopic],
requestContext: RequestContext requestContext: RequestContext,
timeoutMs: Long
): Unit = { ): Unit = {
if (topics.nonEmpty) { 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( private def sendCreateTopicRequest(
creatableTopics: Map[String, CreatableTopic], creatableTopics: Map[String, CreatableTopic],
requestContext: Option[RequestContext] requestContext: Option[RequestContext]
@ -264,4 +349,101 @@ class DefaultAutoTopicCreationManager(
(creatableTopics, uncreatableTopics) (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()
}
} }

View File

@ -387,7 +387,7 @@ class BrokerServer(
autoTopicCreationManager = new DefaultAutoTopicCreationManager( autoTopicCreationManager = new DefaultAutoTopicCreationManager(
config, clientToControllerChannelManager, groupCoordinator, config, clientToControllerChannelManager, groupCoordinator,
transactionCoordinator, shareCoordinator) transactionCoordinator, shareCoordinator, time)
dynamicConfigHandlers = Map[ConfigType, ConfigHandler]( dynamicConfigHandlers = Map[ConfigType, ConfigHandler](
ConfigType.TOPIC -> new TopicConfigHandler(replicaManager, config, quotaManagers), ConfigType.TOPIC -> new TopicConfigHandler(replicaManager, config, quotaManagers),
@ -613,7 +613,8 @@ class BrokerServer(
tp => replicaManager.getLog(tp).toJava, tp => replicaManager.getLog(tp).toJava,
tp => replicaManager.getLogEndOffset(tp).map(Long.box).toJava, tp => replicaManager.getLogEndOffset(tp).map(Long.box).toJava,
serde, serde,
config.groupCoordinatorConfig.offsetsLoadBufferSize config.groupCoordinatorConfig.offsetsLoadBufferSize,
CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS
) )
val writer = new CoordinatorPartitionWriter( val writer = new CoordinatorPartitionWriter(
replicaManager replicaManager
@ -644,7 +645,8 @@ class BrokerServer(
tp => replicaManager.getLog(tp).toJava, tp => replicaManager.getLog(tp).toJava,
tp => replicaManager.getLogEndOffset(tp).map(Long.box).toJava, tp => replicaManager.getLogEndOffset(tp).map(Long.box).toJava,
serde, serde,
config.shareCoordinatorConfig.shareCoordinatorLoadBufferSize() config.shareCoordinatorConfig.shareCoordinatorLoadBufferSize(),
CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS
) )
val writer = new CoordinatorPartitionWriter( val writer = new CoordinatorPartitionWriter(
replicaManager replicaManager
@ -780,6 +782,9 @@ class BrokerServer(
if (shareCoordinator != null) if (shareCoordinator != null)
CoreUtils.swallow(shareCoordinator.shutdown(), this) CoreUtils.swallow(shareCoordinator.shutdown(), this)
if (autoTopicCreationManager != null)
CoreUtils.swallow(autoTopicCreationManager.close(), this)
if (assignmentsManager != null) if (assignmentsManager != null)
CoreUtils.swallow(assignmentsManager.close(), this) CoreUtils.swallow(assignmentsManager.close(), this)

View File

@ -24,7 +24,6 @@ import java.util.Map.Entry
import java.util.concurrent.CompletableFuture import java.util.concurrent.CompletableFuture
import java.util.function.Consumer import java.util.function.Consumer
import kafka.network.RequestChannel import kafka.network.RequestChannel
import kafka.raft.RaftManager
import kafka.server.QuotaFactory.QuotaManagers import kafka.server.QuotaFactory.QuotaManagers
import kafka.server.logger.RuntimeLoggerManager import kafka.server.logger.RuntimeLoggerManager
import kafka.server.metadata.KRaftMetadataCache 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.metadata.{BrokerHeartbeatReply, BrokerRegistrationReply}
import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.security.auth.KafkaPrincipal
import org.apache.kafka.common.security.auth.SecurityProtocol 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.{ApiVersionManager, DelegationTokenManager, ProcessRole}
import org.apache.kafka.server.authorizer.Authorizer import org.apache.kafka.server.authorizer.Authorizer
import org.apache.kafka.server.common.{ApiMessageAndVersion, RequestLocal} import org.apache.kafka.server.common.{ApiMessageAndVersion, RequestLocal}
@ -1070,7 +1070,7 @@ class ControllerApis(
EndpointType.CONTROLLER, EndpointType.CONTROLLER,
clusterId, clusterId,
() => registrationsPublisher.describeClusterControllers(request.context.listenerName()), () => registrationsPublisher.describeClusterControllers(request.context.listenerName()),
() => raftManager.leaderAndEpoch.leaderId().orElse(-1) () => raftManager.client.leaderAndEpoch.leaderId().orElse(-1)
) )
requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
new DescribeClusterResponse(response.setThrottleTimeMs(requestThrottleMs))) new DescribeClusterResponse(response.setThrottleTimeMs(requestThrottleMs)))

View File

@ -2812,10 +2812,35 @@ class KafkaApis(val requestChannel: RequestChannel,
) )
} }
} else { } 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)) requestHelper.sendMaybeThrottle(request, new StreamsGroupHeartbeatResponse(responseData))
} }
} }

View File

@ -17,7 +17,6 @@
package kafka.server package kafka.server
import kafka.raft.RaftManager
import kafka.utils.Logging import kafka.utils.Logging
import org.apache.kafka.clients._ import org.apache.kafka.clients._
import org.apache.kafka.common.metrics.Metrics 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.security.auth.SecurityProtocol
import org.apache.kafka.common.utils.{LogContext, Time} import org.apache.kafka.common.utils.{LogContext, Time}
import org.apache.kafka.common.{Node, Reconfigurable} 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.common.{ApiMessageAndVersion, ControllerRequestCompletionHandler, NodeToControllerChannelManager}
import org.apache.kafka.server.util.{InterBrokerSendThread, RequestAndCompletionHandler} import org.apache.kafka.server.util.{InterBrokerSendThread, RequestAndCompletionHandler}
@ -37,7 +37,7 @@ import java.util.concurrent.LinkedBlockingDeque
import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.atomic.AtomicReference
import scala.collection.Seq import scala.collection.Seq
import scala.jdk.CollectionConverters._ import scala.jdk.CollectionConverters._
import scala.jdk.OptionConverters.{RichOption, RichOptionalInt} import scala.jdk.OptionConverters.{RichOption, RichOptional, RichOptionalInt}
case class ControllerInformation( case class ControllerInformation(
node: Option[Node], node: Option[Node],
@ -79,10 +79,10 @@ class RaftControllerNodeProvider(
val saslMechanism: String val saslMechanism: String
) extends ControllerNodeProvider with Logging { ) 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 = override def getControllerInfo(): ControllerInformation =
ControllerInformation(raftManager.leaderAndEpoch.leaderId.toScala.flatMap(idToNode), ControllerInformation(raftManager.client.leaderAndEpoch.leaderId.toScala.flatMap(idToNode),
listenerName, securityProtocol, saslMechanism) listenerName, securityProtocol, saslMechanism)
} }

View File

@ -342,7 +342,7 @@ class SharedServer(
throw new RuntimeException("Unable to install metadata publishers.", t) throw new RuntimeException("Unable to install metadata publishers.", t)
} }
} }
_raftManager.register(loader) _raftManager.client.register(loader)
debug("Completed SharedServer startup.") debug("Completed SharedServer startup.")
started = true started = true
} catch { } catch {

View File

@ -18,7 +18,6 @@
package kafka.tools package kafka.tools
import kafka.network.RequestChannel import kafka.network.RequestChannel
import kafka.raft.RaftManager
import kafka.server.ApiRequestHandler import kafka.server.ApiRequestHandler
import kafka.utils.Logging import kafka.utils.Logging
import org.apache.kafka.common.internals.FatalExitError 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.protocol.{ApiKeys, ApiMessage}
import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, BeginQuorumEpochResponse, EndQuorumEpochResponse, FetchResponse, FetchSnapshotResponse, VoteResponse} import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, BeginQuorumEpochResponse, EndQuorumEpochResponse, FetchResponse, FetchSnapshotResponse, VoteResponse}
import org.apache.kafka.common.utils.Time import org.apache.kafka.common.utils.Time
import org.apache.kafka.raft.RaftManager
import org.apache.kafka.server.ApiVersionManager import org.apache.kafka.server.ApiVersionManager
import org.apache.kafka.server.common.RequestLocal import org.apache.kafka.server.common.RequestLocal

View File

@ -22,7 +22,7 @@ import java.util.concurrent.atomic.{AtomicInteger, AtomicLong}
import java.util.concurrent.{CompletableFuture, CountDownLatch, LinkedBlockingDeque, TimeUnit} import java.util.concurrent.{CompletableFuture, CountDownLatch, LinkedBlockingDeque, TimeUnit}
import joptsimple.{OptionException, OptionSpec} import joptsimple.{OptionException, OptionSpec}
import kafka.network.SocketServer import kafka.network.SocketServer
import kafka.raft.{DefaultExternalKRaftMetrics, KafkaRaftManager, RaftManager} import kafka.raft.{DefaultExternalKRaftMetrics, KafkaRaftManager}
import kafka.server.{KafkaConfig, KafkaRequestHandlerPool} import kafka.server.{KafkaConfig, KafkaRequestHandlerPool}
import kafka.utils.{CoreUtils, Logging} import kafka.utils.{CoreUtils, Logging}
import org.apache.kafka.common.message.ApiMessageType.ListenerType 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.utils.{Exit, Time, Utils}
import org.apache.kafka.common.{TopicPartition, Uuid, protocol} import org.apache.kafka.common.{TopicPartition, Uuid, protocol}
import org.apache.kafka.raft.errors.NotLeaderException 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.security.CredentialProvider
import org.apache.kafka.server.SimpleApiVersionManager import org.apache.kafka.server.SimpleApiVersionManager
import org.apache.kafka.server.common.{FinalizedFeatures, MetadataVersion} import org.apache.kafka.server.common.{FinalizedFeatures, MetadataVersion}
@ -180,7 +180,7 @@ class TestRaftServer(
private var claimedEpoch: Option[Int] = None private var claimedEpoch: Option[Int] = None
raftManager.register(this) raftManager.client.register(this)
override def handleLeaderChange(newLeaderAndEpoch: LeaderAndEpoch): Unit = { override def handleLeaderChange(newLeaderAndEpoch: LeaderAndEpoch): Unit = {
if (newLeaderAndEpoch.isLeader(config.nodeId)) { if (newLeaderAndEpoch.isLeader(config.nodeId)) {

View File

@ -30,6 +30,7 @@ import org.apache.kafka.common.Uuid
import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.utils.Time import org.apache.kafka.common.utils.Time
import org.apache.kafka.common.utils.Utils
import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.raft.{Endpoints, MetadataLogConfig, QuorumConfig} import org.apache.kafka.raft.{Endpoints, MetadataLogConfig, QuorumConfig}
import org.apache.kafka.server.ProcessRole import org.apache.kafka.server.ProcessRole
@ -126,6 +127,7 @@ class RaftManagerTest {
val logDir = TestUtils.tempDir() val logDir = TestUtils.tempDir()
val nodeId = 1 val nodeId = 1
try {
val raftManager = createRaftManager( val raftManager = createRaftManager(
new TopicPartition("__raft_id_test", 0), new TopicPartition("__raft_id_test", 0),
createConfig( createConfig(
@ -137,6 +139,9 @@ class RaftManagerTest {
) )
assertEquals(nodeId, raftManager.client.nodeId.getAsInt) assertEquals(nodeId, raftManager.client.nodeId.getAsInt)
raftManager.shutdown() raftManager.shutdown()
} finally {
Utils.delete(logDir)
}
} }
@ParameterizedTest @ParameterizedTest
@ -155,6 +160,7 @@ class RaftManagerTest {
} }
val nodeId = 1 val nodeId = 1
try {
val raftManager = createRaftManager( val raftManager = createRaftManager(
new TopicPartition("__raft_id_test", 0), new TopicPartition("__raft_id_test", 0),
createConfig( createConfig(
@ -171,6 +177,10 @@ class RaftManagerTest {
raftManager.shutdown() raftManager.shutdown()
assertFalse(fileLocked(lockPath)) assertFalse(fileLocked(lockPath))
} finally {
logDir.foreach(p => Utils.delete(p.toFile))
metadataDir.foreach(p => Utils.delete(p.toFile))
}
} }
@Test @Test
@ -179,6 +189,7 @@ class RaftManagerTest {
val metadataDir = Some(TestUtils.tempDir().toPath) val metadataDir = Some(TestUtils.tempDir().toPath)
val nodeId = 1 val nodeId = 1
try {
val raftManager = createRaftManager( val raftManager = createRaftManager(
new TopicPartition("__raft_id_test", 0), new TopicPartition("__raft_id_test", 0),
createConfig( createConfig(
@ -195,6 +206,10 @@ class RaftManagerTest {
raftManager.shutdown() raftManager.shutdown()
assertFalse(fileLocked(lockPath)) assertFalse(fileLocked(lockPath))
} finally {
logDir.foreach(p => Utils.delete(p.toFile))
metadataDir.foreach(p => Utils.delete(p.toFile))
}
} }
def createMetadataLog(config: KafkaConfig): Unit = { def createMetadataLog(config: KafkaConfig): Unit = {

View File

@ -34,7 +34,7 @@ class AllocateProducerIdsRequestTest(cluster: ClusterInstance) {
def testAllocateProducersIdSentToController(): Unit = { def testAllocateProducersIdSentToController(): Unit = {
val sourceBroker = cluster.brokers.values().stream().findFirst().get().asInstanceOf[BrokerServer] 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() val controllerServer = cluster.controllers.values().stream()
.filter(_.config.nodeId == controllerId) .filter(_.config.nodeId == controllerId)
.findFirst() .findFirst()
@ -50,7 +50,7 @@ class AllocateProducerIdsRequestTest(cluster: ClusterInstance) {
def testAllocateProducersIdSentToNonController(): Unit = { def testAllocateProducersIdSentToNonController(): Unit = {
val sourceBroker = cluster.brokers.values().stream().findFirst().get().asInstanceOf[BrokerServer] 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() val controllerServer = cluster.controllers().values().stream()
.filter(_.config.nodeId != controllerId) .filter(_.config.nodeId != controllerId)
.findFirst() .findFirst()

View File

@ -35,6 +35,7 @@ import org.apache.kafka.common.protocol.{ApiKeys, ByteBufferAccessor, Errors}
import org.apache.kafka.common.requests._ import org.apache.kafka.common.requests._
import org.apache.kafka.common.security.auth.{KafkaPrincipal, KafkaPrincipalSerde, SecurityProtocol} import org.apache.kafka.common.security.auth.{KafkaPrincipal, KafkaPrincipalSerde, SecurityProtocol}
import org.apache.kafka.common.utils.{SecurityUtils, Utils} 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.group.{GroupCoordinator, GroupCoordinatorConfig}
import org.apache.kafka.coordinator.share.{ShareCoordinator, ShareCoordinatorConfig} import org.apache.kafka.coordinator.share.{ShareCoordinator, ShareCoordinatorConfig}
import org.apache.kafka.metadata.MetadataCache 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.Assertions.{assertEquals, assertThrows, assertTrue}
import org.junit.jupiter.api.{BeforeEach, Test} import org.junit.jupiter.api.{BeforeEach, Test}
import org.mockito.ArgumentMatchers.any import org.mockito.ArgumentMatchers.any
import org.mockito.Mockito.never
import org.mockito.{ArgumentCaptor, ArgumentMatchers, Mockito} import org.mockito.{ArgumentCaptor, ArgumentMatchers, Mockito}
import org.mockito.Mockito.never
import scala.collection.{Map, Seq} import scala.collection.{Map, Seq}
class AutoTopicCreationManagerTest { class AutoTopicCreationManagerTest {
private val requestTimeout = 100 private val requestTimeout = 100
private val testCacheCapacity = 3
private var config: KafkaConfig = _ private var config: KafkaConfig = _
private val metadataCache = Mockito.mock(classOf[MetadataCache]) private val metadataCache = Mockito.mock(classOf[MetadataCache])
private val brokerToController = Mockito.mock(classOf[NodeToControllerChannelManager]) private val brokerToController = Mockito.mock(classOf[NodeToControllerChannelManager])
@ -60,6 +62,7 @@ class AutoTopicCreationManagerTest {
private val transactionCoordinator = Mockito.mock(classOf[TransactionCoordinator]) private val transactionCoordinator = Mockito.mock(classOf[TransactionCoordinator])
private val shareCoordinator = Mockito.mock(classOf[ShareCoordinator]) private val shareCoordinator = Mockito.mock(classOf[ShareCoordinator])
private var autoTopicCreationManager: AutoTopicCreationManager = _ private var autoTopicCreationManager: AutoTopicCreationManager = _
private val mockTime = new MockTime(0L, 0L)
private val internalTopicPartitions = 2 private val internalTopicPartitions = 2
private val internalTopicReplicationFactor: Short = 2 private val internalTopicReplicationFactor: Short = 2
@ -76,6 +79,8 @@ class AutoTopicCreationManagerTest {
props.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, internalTopicReplicationFactor.toString) props.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, internalTopicReplicationFactor.toString)
props.setProperty(TransactionLogConfig.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, internalTopicReplicationFactor.toString) props.setProperty(TransactionLogConfig.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, internalTopicReplicationFactor.toString)
props.setProperty(ShareCoordinatorConfig.STATE_TOPIC_NUM_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) config = KafkaConfig.fromProps(props)
val aliveBrokers = util.List.of(new Node(0, "host0", 0), new Node(1, "host1", 1)) val aliveBrokers = util.List.of(new Node(0, "host0", 0), new Node(1, "host1", 1))
@ -115,7 +120,9 @@ class AutoTopicCreationManagerTest {
brokerToController, brokerToController,
groupCoordinator, groupCoordinator,
transactionCoordinator, transactionCoordinator,
shareCoordinator) shareCoordinator,
mockTime,
topicErrorCacheCapacity = testCacheCapacity)
val topicsCollection = new CreateTopicsRequestData.CreatableTopicCollection val topicsCollection = new CreateTopicsRequestData.CreatableTopicCollection
topicsCollection.add(getNewTopic(topicName, numPartitions, replicationFactor)) topicsCollection.add(getNewTopic(topicName, numPartitions, replicationFactor))
@ -231,9 +238,11 @@ class AutoTopicCreationManagerTest {
brokerToController, brokerToController,
groupCoordinator, groupCoordinator,
transactionCoordinator, 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]]) val argumentCaptor = ArgumentCaptor.forClass(classOf[AbstractRequest.Builder[_ <: AbstractRequest]])
Mockito.verify(brokerToController).sendRequest( Mockito.verify(brokerToController).sendRequest(
@ -267,9 +276,11 @@ class AutoTopicCreationManagerTest {
brokerToController, brokerToController,
groupCoordinator, groupCoordinator,
transactionCoordinator, transactionCoordinator,
shareCoordinator) shareCoordinator,
mockTime,
topicErrorCacheCapacity = testCacheCapacity)
autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext) autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext, config.groupCoordinatorConfig.streamsGroupHeartbeatIntervalMs() * 2)
Mockito.verify(brokerToController, never()).sendRequest( Mockito.verify(brokerToController, never()).sendRequest(
any(classOf[AbstractRequest.Builder[_ <: AbstractRequest]]), any(classOf[AbstractRequest.Builder[_ <: AbstractRequest]]),
@ -288,9 +299,11 @@ class AutoTopicCreationManagerTest {
brokerToController, brokerToController,
groupCoordinator, groupCoordinator,
transactionCoordinator, 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]]) val argumentCaptor = ArgumentCaptor.forClass(classOf[AbstractRequest.Builder[_ <: AbstractRequest]])
Mockito.verify(brokerToController).sendRequest( Mockito.verify(brokerToController).sendRequest(
@ -319,7 +332,9 @@ class AutoTopicCreationManagerTest {
brokerToController, brokerToController,
groupCoordinator, groupCoordinator,
transactionCoordinator, transactionCoordinator,
shareCoordinator) shareCoordinator,
mockTime,
topicErrorCacheCapacity = testCacheCapacity)
val createTopicApiVersion = new ApiVersionsResponseData.ApiVersion() val createTopicApiVersion = new ApiVersionsResponseData.ApiVersion()
.setApiKey(ApiKeys.CREATE_TOPICS.id) .setApiKey(ApiKeys.CREATE_TOPICS.id)
@ -356,4 +371,217 @@ class AutoTopicCreationManagerTest {
.setNumPartitions(numPartitions) .setNumPartitions(numPartitions)
.setReplicationFactor(replicationFactor) .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")
}
} }

View File

@ -18,7 +18,6 @@
package kafka.server package kafka.server
import kafka.network.RequestChannel import kafka.network.RequestChannel
import kafka.raft.RaftManager
import kafka.server.QuotaFactory.QuotaManagers import kafka.server.QuotaFactory.QuotaManagers
import kafka.server.metadata.KRaftMetadataCache import kafka.server.metadata.KRaftMetadataCache
import org.apache.kafka.clients.admin.AlterConfigOp 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.SocketServerConfigs
import org.apache.kafka.network.metrics.RequestChannelMetrics import org.apache.kafka.network.metrics.RequestChannelMetrics
import org.apache.kafka.network.Session 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.SimpleApiVersionManager
import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult, Authorizer} import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult, Authorizer}
import org.apache.kafka.server.common.{ApiMessageAndVersion, FinalizedFeatures, KRaftVersion, MetadataVersion, ProducerIdsBlock, RequestLocal} import org.apache.kafka.server.common.{ApiMessageAndVersion, FinalizedFeatures, KRaftVersion, MetadataVersion, ProducerIdsBlock, RequestLocal}

View File

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

View File

@ -168,7 +168,8 @@ class KafkaApisTest extends Logging {
authorizer: Option[Authorizer] = None, authorizer: Option[Authorizer] = None,
configRepository: ConfigRepository = new MockConfigRepository(), configRepository: ConfigRepository = new MockConfigRepository(),
overrideProperties: Map[String, String] = Map.empty, overrideProperties: Map[String, String] = Map.empty,
featureVersions: Seq[FeatureVersion] = Seq.empty featureVersions: Seq[FeatureVersion] = Seq.empty,
autoTopicCreationManager: Option[AutoTopicCreationManager] = None
): KafkaApis = { ): KafkaApis = {
val properties = TestUtils.createBrokerConfig(brokerId) val properties = TestUtils.createBrokerConfig(brokerId)
@ -194,7 +195,7 @@ class KafkaApisTest extends Logging {
groupCoordinator = groupCoordinator, groupCoordinator = groupCoordinator,
txnCoordinator = txnCoordinator, txnCoordinator = txnCoordinator,
shareCoordinator = shareCoordinator, shareCoordinator = shareCoordinator,
autoTopicCreationManager = autoTopicCreationManager, autoTopicCreationManager = autoTopicCreationManager.getOrElse(this.autoTopicCreationManager),
brokerId = brokerId, brokerId = brokerId,
config = config, config = config,
configRepository = configRepository, configRepository = configRepository,
@ -10887,7 +10888,7 @@ class KafkaApisTest extends Logging {
future.complete(new StreamsGroupHeartbeatResult(streamsGroupHeartbeatResponse, missingTopics.asJava)) future.complete(new StreamsGroupHeartbeatResult(streamsGroupHeartbeatResponse, missingTopics.asJava))
val response = verifyNoThrottling[StreamsGroupHeartbeatResponse](requestChannelRequest) val response = verifyNoThrottling[StreamsGroupHeartbeatResponse](requestChannelRequest)
assertEquals(streamsGroupHeartbeatResponse, response.data) assertEquals(streamsGroupHeartbeatResponse, response.data)
verify(autoTopicCreationManager).createStreamsInternalTopics(missingTopics, requestChannelRequest.context) verify(autoTopicCreationManager).createStreamsInternalTopics(any(), any(), anyLong())
} }
@Test @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 @ParameterizedTest
@ValueSource(booleans = Array(true, false)) @ValueSource(booleans = Array(true, false))
def testConsumerGroupDescribe(includeAuthorizedOperations: Boolean): Unit = { def testConsumerGroupDescribe(includeAuthorizedOperations: Boolean): Unit = {

View File

@ -293,7 +293,7 @@ class LeaderEpochIntegrationTest extends QuorumTestHarness with Logging {
} }
private def waitUntilQuorumLeaderElected(controllerServer: ControllerServer, timeout: Long = JTestUtils.DEFAULT_MAX_WAIT_MS): Int = { 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")) leaderAndEpoch.leaderId().orElseThrow(() => new AssertionError(s"Quorum Controller leader not elected after $timeout ms"))
} }

View File

@ -147,7 +147,7 @@ Single Node
- To produce messages using client scripts (Ensure that java version >= 17): - To produce messages using client scripts (Ensure that java version >= 17):
``` ```
# Run from root of the repo # 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: - File Input:
- Here ssl configs are provided via 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): - To produce messages using client scripts (Ensure that java version >= 17):
``` ```
# Run from root of the repo # 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 Multi Node Cluster
@ -219,7 +219,7 @@ Multi Node Cluster
- To produce messages using client scripts (Ensure that java version >= 17): - To produce messages using client scripts (Ensure that java version >= 17):
``` ```
# Run from root of the repo # 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: - Isolated:
- Examples are present in `docker-compose-files/cluster/isolated` directory. - 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): - To produce messages using client scripts (Ensure that java version >= 17):
``` ```
# Run from root of the repo # 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. - 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.

View File

@ -65,7 +65,7 @@ class DockerSanityTest(unittest.TestCase):
subprocess.run(["bash", "-c", " ".join(command)]) subprocess.run(["bash", "-c", " ".join(command)])
def consume_message(self, topic, consumer_config): 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) command.extend(consumer_config)
message = subprocess.check_output(["bash", "-c", " ".join(command)]) message = subprocess.check_output(["bash", "-c", " ".join(command)])
return message.decode("utf-8").strip() return message.decode("utf-8").strip()
@ -93,9 +93,9 @@ class DockerSanityTest(unittest.TestCase):
errors.append(constants.BROKER_METRICS_ERROR_PREFIX + str(e)) errors.append(constants.BROKER_METRICS_ERROR_PREFIX + str(e))
return errors 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") 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) message = self.consume_message(constants.BROKER_METRICS_TEST_TOPIC, consumer_config)
try: try:
self.assertEqual(message, "key:message") self.assertEqual(message, "key:message")
@ -129,13 +129,13 @@ class DockerSanityTest(unittest.TestCase):
return errors return errors
producer_config = ["--bootstrap-server", ssl_broker_port, 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") self.produce_message(topic, producer_config, "key", "message")
consumer_config = [ consumer_config = [
"--bootstrap-server", ssl_broker_port, "--bootstrap-server", ssl_broker_port,
"--property", "auto.offset.reset=earliest", "--command-property", "auto.offset.reset=earliest",
"--consumer.config", f"{self.FIXTURES_DIR}/{constants.SSL_CLIENT_CONFIG}", "--command-config", f"{self.FIXTURES_DIR}/{constants.SSL_CLIENT_CONFIG}",
] ]
message = self.consume_message(topic, consumer_config) message = self.consume_message(topic, consumer_config)
try: try:
@ -155,7 +155,7 @@ class DockerSanityTest(unittest.TestCase):
errors.append(constants.BROKER_RESTART_ERROR_PREFIX + str(e)) errors.append(constants.BROKER_RESTART_ERROR_PREFIX + str(e))
return errors 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") self.produce_message(constants.BROKER_RESTART_TEST_TOPIC, producer_config, "key", "message")
print("Stopping Container") print("Stopping Container")
@ -163,7 +163,7 @@ class DockerSanityTest(unittest.TestCase):
print("Resuming Container") print("Resuming Container")
self.resume_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) message = self.consume_message(constants.BROKER_RESTART_TEST_TOPIC, consumer_config)
try: try:
self.assertEqual(message, "key:message") self.assertEqual(message, "key:message")

View File

@ -510,8 +510,8 @@ ssl.key.password=test1234</code></pre>
</ol> </ol>
<br> <br>
Examples using console-producer and console-consumer: 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 <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 --consumer.config client-ssl.properties</code></pre> $ bin/kafka-console-consumer.sh --bootstrap-server localhost:9093 --topic test --command-config client-ssl.properties</code></pre>
</li> </li>
</ol> </ol>
<h3 class="anchor-heading"><a id="security_sasl" class="anchor-link"></a><a href="#security_sasl">7.4 Authentication using SASL</a></h3> <h3 class="anchor-heading"><a id="security_sasl" class="anchor-link"></a><a href="#security_sasl">7.4 Authentication using SASL</a></h3>

View File

@ -142,6 +142,11 @@
</li> </li>
</ul> </ul>
</li> </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> </ul>
<h4><a id="upgrade_4_1_0" href="#upgrade_4_1_0">Upgrading to 4.1.0</a></h4> <h4><a id="upgrade_4_1_0" href="#upgrade_4_1_0">Upgrading to 4.1.0</a></h4>

View File

@ -124,11 +124,11 @@ public class CoordinatorRecordTypeGenerator implements TypeClassGenerator {
for (Map.Entry<Short, CoordinatorRecord> entry : records.entrySet()) { for (Map.Entry<Short, CoordinatorRecord> entry : records.entrySet()) {
MessageSpec key = entry.getValue().key; MessageSpec key = entry.getValue().key;
if (key == null) { 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; MessageSpec value = entry.getValue().value;
if (value == null) { 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()); String name = cleanName(key.name());
numProcessed++; numProcessed++;

View File

@ -813,10 +813,10 @@ public class GroupMetadataManager {
} }
if (group == null) { if (group == null) {
return new ConsumerGroup(snapshotRegistry, groupId, metrics); return new ConsumerGroup(snapshotRegistry, groupId);
} else if (createIfNotExists && maybeDeleteEmptyClassicGroup(group, records)) { } else if (createIfNotExists && maybeDeleteEmptyClassicGroup(group, records)) {
log.info("[GroupId {}] Converted the empty classic group to a consumer group.", groupId); 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 { } else {
if (group.type() == CONSUMER) { if (group.type() == CONSUMER) {
return (ConsumerGroup) group; return (ConsumerGroup) group;
@ -851,10 +851,10 @@ public class GroupMetadataManager {
Group group = groups.get(groupId); Group group = groups.get(groupId);
if (group == null) { if (group == null) {
return new StreamsGroup(logContext, snapshotRegistry, groupId, metrics); return new StreamsGroup(logContext, snapshotRegistry, groupId);
} else if (maybeDeleteEmptyClassicGroup(group, records)) { } else if (maybeDeleteEmptyClassicGroup(group, records)) {
log.info("[GroupId {}] Converted the empty classic group to a streams group.", groupId); 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 { } else {
return castToStreamsGroup(group); return castToStreamsGroup(group);
} }
@ -980,7 +980,7 @@ public class GroupMetadataManager {
} }
if (group == null) { if (group == null) {
ConsumerGroup consumerGroup = new ConsumerGroup(snapshotRegistry, groupId, metrics); ConsumerGroup consumerGroup = new ConsumerGroup(snapshotRegistry, groupId);
groups.put(groupId, consumerGroup); groups.put(groupId, consumerGroup);
return consumerGroup; return consumerGroup;
} else if (group.type() == CONSUMER) { } 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 // 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, // 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. // 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); groups.put(groupId, consumerGroup);
return consumerGroup; return consumerGroup;
} else { } else {
@ -1023,7 +1023,7 @@ public class GroupMetadataManager {
} }
if (group == null) { if (group == null) {
StreamsGroup streamsGroup = new StreamsGroup(logContext, snapshotRegistry, groupId, metrics); StreamsGroup streamsGroup = new StreamsGroup(logContext, snapshotRegistry, groupId);
groups.put(groupId, streamsGroup); groups.put(groupId, streamsGroup);
return streamsGroup; return streamsGroup;
} else if (group.type() == STREAMS) { } else if (group.type() == STREAMS) {
@ -1364,7 +1364,6 @@ public class GroupMetadataManager {
try { try {
consumerGroup = ConsumerGroup.fromClassicGroup( consumerGroup = ConsumerGroup.fromClassicGroup(
snapshotRegistry, snapshotRegistry,
metrics,
classicGroup, classicGroup,
topicHashCache, topicHashCache,
metadataImage metadataImage

View File

@ -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.api.assignor.SubscriptionType;
import org.apache.kafka.coordinator.group.classic.ClassicGroup; import org.apache.kafka.coordinator.group.classic.ClassicGroup;
import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataValue; 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.Assignment;
import org.apache.kafka.coordinator.group.modern.MemberState; import org.apache.kafka.coordinator.group.modern.MemberState;
import org.apache.kafka.coordinator.group.modern.ModernGroup; import org.apache.kafka.coordinator.group.modern.ModernGroup;
@ -119,11 +118,6 @@ public class ConsumerGroup extends ModernGroup<ConsumerGroupMember> {
*/ */
private final TimelineHashMap<String, Integer> serverAssignors; private final TimelineHashMap<String, Integer> serverAssignors;
/**
* The coordinator metrics.
*/
private final GroupCoordinatorMetricsShard metrics;
/** /**
* The number of members that use the classic protocol. * The number of members that use the classic protocol.
*/ */
@ -155,14 +149,12 @@ public class ConsumerGroup extends ModernGroup<ConsumerGroupMember> {
public ConsumerGroup( public ConsumerGroup(
SnapshotRegistry snapshotRegistry, SnapshotRegistry snapshotRegistry,
String groupId, String groupId
GroupCoordinatorMetricsShard metrics
) { ) {
super(snapshotRegistry, groupId); super(snapshotRegistry, groupId);
this.state = new TimelineObject<>(snapshotRegistry, EMPTY); this.state = new TimelineObject<>(snapshotRegistry, EMPTY);
this.staticMembers = new TimelineHashMap<>(snapshotRegistry, 0); this.staticMembers = new TimelineHashMap<>(snapshotRegistry, 0);
this.serverAssignors = new TimelineHashMap<>(snapshotRegistry, 0); this.serverAssignors = new TimelineHashMap<>(snapshotRegistry, 0);
this.metrics = Objects.requireNonNull(metrics);
this.numClassicProtocolMembers = new TimelineInteger(snapshotRegistry); this.numClassicProtocolMembers = new TimelineInteger(snapshotRegistry);
this.classicProtocolMembersSupportedProtocols = new TimelineHashMap<>(snapshotRegistry, 0); this.classicProtocolMembersSupportedProtocols = new TimelineHashMap<>(snapshotRegistry, 0);
this.currentPartitionEpoch = 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. * Create a new consumer group according to the given classic group.
* *
* @param snapshotRegistry The SnapshotRegistry. * @param snapshotRegistry The SnapshotRegistry.
* @param metrics The GroupCoordinatorMetricsShard.
* @param classicGroup The converted classic group. * @param classicGroup The converted classic group.
* @param topicHashCache The cache for topic hashes. * @param topicHashCache The cache for topic hashes.
* @param metadataImage The current metadata image for the Kafka cluster. * @param metadataImage The current metadata image for the Kafka cluster.
@ -1141,13 +1132,12 @@ public class ConsumerGroup extends ModernGroup<ConsumerGroupMember> {
*/ */
public static ConsumerGroup fromClassicGroup( public static ConsumerGroup fromClassicGroup(
SnapshotRegistry snapshotRegistry, SnapshotRegistry snapshotRegistry,
GroupCoordinatorMetricsShard metrics,
ClassicGroup classicGroup, ClassicGroup classicGroup,
Map<String, Long> topicHashCache, Map<String, Long> topicHashCache,
CoordinatorMetadataImage metadataImage CoordinatorMetadataImage metadataImage
) { ) {
String groupId = classicGroup.groupId(); String groupId = classicGroup.groupId();
ConsumerGroup consumerGroup = new ConsumerGroup(snapshotRegistry, groupId, metrics); ConsumerGroup consumerGroup = new ConsumerGroup(snapshotRegistry, groupId);
consumerGroup.setGroupEpoch(classicGroup.generationId()); consumerGroup.setGroupEpoch(classicGroup.generationId());
consumerGroup.setTargetAssignmentEpoch(classicGroup.generationId()); consumerGroup.setTargetAssignmentEpoch(classicGroup.generationId());

View File

@ -31,7 +31,6 @@ import org.apache.kafka.coordinator.group.Group;
import org.apache.kafka.coordinator.group.OffsetExpirationCondition; import org.apache.kafka.coordinator.group.OffsetExpirationCondition;
import org.apache.kafka.coordinator.group.OffsetExpirationConditionImpl; import org.apache.kafka.coordinator.group.OffsetExpirationConditionImpl;
import org.apache.kafka.coordinator.group.Utils; 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.ConfiguredSubtopology;
import org.apache.kafka.coordinator.group.streams.topics.ConfiguredTopology; import org.apache.kafka.coordinator.group.streams.topics.ConfiguredTopology;
import org.apache.kafka.timeline.SnapshotRegistry; 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>>> currentStandbyTaskToProcessIds;
private final TimelineHashMap<String, TimelineHashMap<Integer, Set<String>>> currentWarmupTaskToProcessIds; private final TimelineHashMap<String, TimelineHashMap<Integer, Set<String>>> currentWarmupTaskToProcessIds;
/**
* The coordinator metrics.
*/
private final GroupCoordinatorMetricsShard metrics;
/** /**
* The Streams topology. * The Streams topology.
*/ */
@ -220,8 +214,7 @@ public class StreamsGroup implements Group {
public StreamsGroup( public StreamsGroup(
LogContext logContext, LogContext logContext,
SnapshotRegistry snapshotRegistry, SnapshotRegistry snapshotRegistry,
String groupId, String groupId
GroupCoordinatorMetricsShard metrics
) { ) {
this.log = logContext.logger(StreamsGroup.class); this.log = logContext.logger(StreamsGroup.class);
this.logContext = logContext; this.logContext = logContext;
@ -238,7 +231,6 @@ public class StreamsGroup implements Group {
this.currentActiveTaskToProcessId = new TimelineHashMap<>(snapshotRegistry, 0); this.currentActiveTaskToProcessId = new TimelineHashMap<>(snapshotRegistry, 0);
this.currentStandbyTaskToProcessIds = new TimelineHashMap<>(snapshotRegistry, 0); this.currentStandbyTaskToProcessIds = new TimelineHashMap<>(snapshotRegistry, 0);
this.currentWarmupTaskToProcessIds = new TimelineHashMap<>(snapshotRegistry, 0); this.currentWarmupTaskToProcessIds = new TimelineHashMap<>(snapshotRegistry, 0);
this.metrics = Objects.requireNonNull(metrics);
this.topology = new TimelineObject<>(snapshotRegistry, Optional.empty()); this.topology = new TimelineObject<>(snapshotRegistry, Optional.empty());
this.configuredTopology = new TimelineObject<>(snapshotRegistry, Optional.empty()); this.configuredTopology = new TimelineObject<>(snapshotRegistry, Optional.empty());
} }

View File

@ -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.StreamsGroupTargetAssignmentMetadataValue;
import org.apache.kafka.coordinator.group.generated.StreamsGroupTopologyKey; import org.apache.kafka.coordinator.group.generated.StreamsGroupTopologyKey;
import org.apache.kafka.coordinator.group.generated.StreamsGroupTopologyValue; 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.consumer.ConsumerGroup;
import org.apache.kafka.coordinator.group.modern.share.ShareGroup; import org.apache.kafka.coordinator.group.modern.share.ShareGroup;
import org.apache.kafka.coordinator.group.streams.StreamsGroupHeartbeatResult; import org.apache.kafka.coordinator.group.streams.StreamsGroupHeartbeatResult;
@ -1384,10 +1383,9 @@ public class GroupCoordinatorShardTest {
ArgumentCaptor<List<CoordinatorRecord>> recordsCapture = ArgumentCaptor.forClass(List.class); ArgumentCaptor<List<CoordinatorRecord>> recordsCapture = ArgumentCaptor.forClass(List.class);
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
GroupCoordinatorMetricsShard metricsShard = mock(GroupCoordinatorMetricsShard.class);
ConsumerGroup group1 = new ConsumerGroup(snapshotRegistry, "group-id", metricsShard); ConsumerGroup group1 = new ConsumerGroup(snapshotRegistry, "group-id");
ConsumerGroup group2 = new ConsumerGroup(snapshotRegistry, "other-group-id", metricsShard); ConsumerGroup group2 = new ConsumerGroup(snapshotRegistry, "other-group-id");
when(groupMetadataManager.groupIds()).thenReturn(Set.of("group-id", "other-group-id")); when(groupMetadataManager.groupIds()).thenReturn(Set.of("group-id", "other-group-id"));
when(groupMetadataManager.group("group-id")).thenReturn(group1); when(groupMetadataManager.group("group-id")).thenReturn(group1);

View File

@ -45,7 +45,6 @@ import org.apache.kafka.coordinator.group.OffsetAndMetadata;
import org.apache.kafka.coordinator.group.OffsetExpirationCondition; import org.apache.kafka.coordinator.group.OffsetExpirationCondition;
import org.apache.kafka.coordinator.group.OffsetExpirationConditionImpl; import org.apache.kafka.coordinator.group.OffsetExpirationConditionImpl;
import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataValue; 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.Assignment;
import org.apache.kafka.coordinator.group.modern.MemberState; import org.apache.kafka.coordinator.group.modern.MemberState;
import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroup; 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.assertNull;
import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.mockito.Mockito.mock;
public class ClassicGroupTest { public class ClassicGroupTest {
private final String protocolType = "consumer"; private final String protocolType = "consumer";
@ -1383,8 +1381,7 @@ public class ClassicGroupTest {
ConsumerGroup consumerGroup = new ConsumerGroup( ConsumerGroup consumerGroup = new ConsumerGroup(
new SnapshotRegistry(logContext), new SnapshotRegistry(logContext),
groupId, groupId
mock(GroupCoordinatorMetricsShard.class)
); );
consumerGroup.setGroupEpoch(10); consumerGroup.setGroupEpoch(10);
consumerGroup.setTargetAssignmentEpoch(10); consumerGroup.setTargetAssignmentEpoch(10);
@ -1536,8 +1533,7 @@ public class ClassicGroupTest {
ConsumerGroup consumerGroup = new ConsumerGroup( ConsumerGroup consumerGroup = new ConsumerGroup(
new SnapshotRegistry(logContext), new SnapshotRegistry(logContext),
groupId, groupId
mock(GroupCoordinatorMetricsShard.class)
); );
consumerGroup.setGroupEpoch(10); consumerGroup.setGroupEpoch(10);
consumerGroup.setTargetAssignmentEpoch(10); consumerGroup.setTargetAssignmentEpoch(10);

View File

@ -25,7 +25,6 @@ import org.apache.kafka.common.errors.IllegalGenerationException;
import org.apache.kafka.common.errors.StaleMemberEpochException; import org.apache.kafka.common.errors.StaleMemberEpochException;
import org.apache.kafka.common.errors.UnknownMemberIdException; import org.apache.kafka.common.errors.UnknownMemberIdException;
import org.apache.kafka.common.errors.UnsupportedVersionException; 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.ConsumerGroupDescribeResponseData;
import org.apache.kafka.common.message.JoinGroupRequestData; import org.apache.kafka.common.message.JoinGroupRequestData;
import org.apache.kafka.common.protocol.ApiKeys; 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.ClassicGroup;
import org.apache.kafka.coordinator.group.classic.ClassicGroupMember; import org.apache.kafka.coordinator.group.classic.ClassicGroupMember;
import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataValue; 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.Assignment;
import org.apache.kafka.coordinator.group.modern.MemberState; import org.apache.kafka.coordinator.group.modern.MemberState;
import org.apache.kafka.coordinator.group.modern.ModernGroup; 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.assertNull;
import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.mockito.Mockito.mock;
public class ConsumerGroupTest { public class ConsumerGroupTest {
@ -91,8 +88,7 @@ public class ConsumerGroupTest {
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
return new ConsumerGroup( return new ConsumerGroup(
snapshotRegistry, snapshotRegistry,
groupId, groupId
mock(GroupCoordinatorMetricsShard.class)
); );
} }
@ -700,8 +696,7 @@ public class ConsumerGroupTest {
@Test @Test
public void testUpdateInvertedAssignment() { public void testUpdateInvertedAssignment() {
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
GroupCoordinatorMetricsShard metricsShard = mock(GroupCoordinatorMetricsShard.class); ConsumerGroup consumerGroup = new ConsumerGroup(snapshotRegistry, "test-group");
ConsumerGroup consumerGroup = new ConsumerGroup(snapshotRegistry, "test-group", metricsShard);
Uuid topicId = Uuid.randomUuid(); Uuid topicId = Uuid.randomUuid();
String memberId1 = "member1"; String memberId1 = "member1";
String memberId2 = "member2"; String memberId2 = "member2";
@ -916,12 +911,7 @@ public class ConsumerGroupTest {
@Test @Test
public void testAsListedGroup() { public void testAsListedGroup() {
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
GroupCoordinatorMetricsShard metricsShard = new GroupCoordinatorMetricsShard( ConsumerGroup group = new ConsumerGroup(snapshotRegistry, "group-foo");
snapshotRegistry,
Map.of(),
new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 0)
);
ConsumerGroup group = new ConsumerGroup(snapshotRegistry, "group-foo", metricsShard);
snapshotRegistry.idempotentCreateSnapshot(0); snapshotRegistry.idempotentCreateSnapshot(0);
assertEquals(ConsumerGroup.ConsumerGroupState.EMPTY.toString(), group.stateAsString(0)); assertEquals(ConsumerGroup.ConsumerGroupState.EMPTY.toString(), group.stateAsString(0));
group.updateMember(new ConsumerGroupMember.Builder("member1") group.updateMember(new ConsumerGroupMember.Builder("member1")
@ -937,8 +927,7 @@ public class ConsumerGroupTest {
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
ConsumerGroup group = new ConsumerGroup( ConsumerGroup group = new ConsumerGroup(
snapshotRegistry, snapshotRegistry,
"group-foo", "group-foo"
mock(GroupCoordinatorMetricsShard.class)
); );
// Simulate a call from the admin client without member id and member epoch. // Simulate a call from the admin client without member id and member epoch.
@ -997,7 +986,7 @@ public class ConsumerGroupTest {
long commitTimestamp = 20000L; long commitTimestamp = 20000L;
long offsetsRetentionMs = 10000L; long offsetsRetentionMs = 10000L;
OffsetAndMetadata offsetAndMetadata = new OffsetAndMetadata(15000L, OptionalInt.empty(), "", commitTimestamp, OptionalLong.empty(), Uuid.ZERO_UUID); 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(); Optional<OffsetExpirationCondition> offsetExpirationCondition = group.offsetExpirationCondition();
assertTrue(offsetExpirationCondition.isPresent()); assertTrue(offsetExpirationCondition.isPresent());
@ -1034,7 +1023,7 @@ public class ConsumerGroupTest {
@Test @Test
public void testAsDescribedGroup() { public void testAsDescribedGroup() {
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); 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); snapshotRegistry.idempotentCreateSnapshot(0);
assertEquals(ConsumerGroup.ConsumerGroupState.EMPTY.toString(), group.stateAsString(0)); assertEquals(ConsumerGroup.ConsumerGroupState.EMPTY.toString(), group.stateAsString(0));
@ -1071,12 +1060,7 @@ public class ConsumerGroupTest {
@Test @Test
public void testIsInStatesCaseInsensitive() { public void testIsInStatesCaseInsensitive() {
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
GroupCoordinatorMetricsShard metricsShard = new GroupCoordinatorMetricsShard( ConsumerGroup group = new ConsumerGroup(snapshotRegistry, "group-foo");
snapshotRegistry,
Map.of(),
new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 0)
);
ConsumerGroup group = new ConsumerGroup(snapshotRegistry, "group-foo", metricsShard);
snapshotRegistry.idempotentCreateSnapshot(0); snapshotRegistry.idempotentCreateSnapshot(0);
assertTrue(group.isInStates(Set.of("empty"), 0)); assertTrue(group.isInStates(Set.of("empty"), 0));
assertFalse(group.isInStates(Set.of("Empty"), 0)); assertFalse(group.isInStates(Set.of("Empty"), 0));
@ -1307,7 +1291,6 @@ public class ConsumerGroupTest {
ConsumerGroup consumerGroup = ConsumerGroup.fromClassicGroup( ConsumerGroup consumerGroup = ConsumerGroup.fromClassicGroup(
new SnapshotRegistry(logContext), new SnapshotRegistry(logContext),
mock(GroupCoordinatorMetricsShard.class),
classicGroup, classicGroup,
new HashMap<>(), new HashMap<>(),
metadataImage metadataImage
@ -1315,8 +1298,7 @@ public class ConsumerGroupTest {
ConsumerGroup expectedConsumerGroup = new ConsumerGroup( ConsumerGroup expectedConsumerGroup = new ConsumerGroup(
new SnapshotRegistry(logContext), new SnapshotRegistry(logContext),
groupId, groupId
mock(GroupCoordinatorMetricsShard.class)
); );
expectedConsumerGroup.setGroupEpoch(10); expectedConsumerGroup.setGroupEpoch(10);
expectedConsumerGroup.setTargetAssignmentEpoch(10); expectedConsumerGroup.setTargetAssignmentEpoch(10);

View File

@ -16,7 +16,6 @@
*/ */
package org.apache.kafka.coordinator.group.streams; package org.apache.kafka.coordinator.group.streams;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.Uuid; import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.errors.GroupNotEmptyException; import org.apache.kafka.common.errors.GroupNotEmptyException;
import org.apache.kafka.common.errors.StaleMemberEpochException; 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.StreamsGroupTargetAssignmentMetadataKey;
import org.apache.kafka.coordinator.group.generated.StreamsGroupTopologyKey; import org.apache.kafka.coordinator.group.generated.StreamsGroupTopologyKey;
import org.apache.kafka.coordinator.group.generated.StreamsGroupTopologyValue; 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.StreamsGroup.StreamsGroupState;
import org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.TaskRole; import org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.TaskRole;
import org.apache.kafka.coordinator.group.streams.topics.ConfiguredTopology; import org.apache.kafka.coordinator.group.streams.topics.ConfiguredTopology;
@ -90,8 +88,7 @@ public class StreamsGroupTest {
return new StreamsGroup( return new StreamsGroup(
LOG_CONTEXT, LOG_CONTEXT,
snapshotRegistry, snapshotRegistry,
groupId, groupId
mock(GroupCoordinatorMetricsShard.class)
); );
} }
@ -693,8 +690,7 @@ public class StreamsGroupTest {
StreamsGroup group = new StreamsGroup( StreamsGroup group = new StreamsGroup(
LOG_CONTEXT, LOG_CONTEXT,
snapshotRegistry, snapshotRegistry,
"group-foo", "group-foo"
mock(GroupCoordinatorMetricsShard.class)
); );
group.setGroupEpoch(1); group.setGroupEpoch(1);
group.setTopology(new StreamsTopology(1, Map.of())); group.setTopology(new StreamsTopology(1, Map.of()));
@ -719,8 +715,7 @@ public class StreamsGroupTest {
StreamsGroup group = new StreamsGroup( StreamsGroup group = new StreamsGroup(
LOG_CONTEXT, LOG_CONTEXT,
snapshotRegistry, snapshotRegistry,
"group-foo", "group-foo"
mock(GroupCoordinatorMetricsShard.class)
); );
// Simulate a call from the admin client without member ID and member epoch. // Simulate a call from the admin client without member ID and member epoch.
@ -790,7 +785,7 @@ public class StreamsGroupTest {
long commitTimestamp = 20000L; long commitTimestamp = 20000L;
long offsetsRetentionMs = 10000L; long offsetsRetentionMs = 10000L;
OffsetAndMetadata offsetAndMetadata = new OffsetAndMetadata(15000L, OptionalInt.empty(), "", commitTimestamp, OptionalLong.empty(), Uuid.ZERO_UUID); 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(); Optional<OffsetExpirationCondition> offsetExpirationCondition = group.offsetExpirationCondition();
assertTrue(offsetExpirationCondition.isPresent()); assertTrue(offsetExpirationCondition.isPresent());
@ -803,7 +798,7 @@ public class StreamsGroupTest {
@Test @Test
public void testAsDescribedGroup() { public void testAsDescribedGroup() {
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); 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); snapshotRegistry.idempotentCreateSnapshot(0);
assertEquals(StreamsGroup.StreamsGroupState.EMPTY.toString(), group.stateAsString(0)); assertEquals(StreamsGroup.StreamsGroupState.EMPTY.toString(), group.stateAsString(0));
@ -887,12 +882,7 @@ public class StreamsGroupTest {
@Test @Test
public void testIsInStatesCaseInsensitiveAndUnderscored() { public void testIsInStatesCaseInsensitiveAndUnderscored() {
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(LOG_CONTEXT); SnapshotRegistry snapshotRegistry = new SnapshotRegistry(LOG_CONTEXT);
GroupCoordinatorMetricsShard metricsShard = new GroupCoordinatorMetricsShard( StreamsGroup group = new StreamsGroup(LOG_CONTEXT, snapshotRegistry, "group-foo");
snapshotRegistry,
Map.of(),
new TopicPartition("__consumer_offsets", 0)
);
StreamsGroup group = new StreamsGroup(LOG_CONTEXT, snapshotRegistry, "group-foo", metricsShard);
snapshotRegistry.idempotentCreateSnapshot(0); snapshotRegistry.idempotentCreateSnapshot(0);
assertTrue(group.isInStates(Set.of("empty"), 0)); assertTrue(group.isInStates(Set.of("empty"), 0));
assertFalse(group.isInStates(Set.of("Empty"), 0)); assertFalse(group.isInStates(Set.of("Empty"), 0));
@ -911,8 +901,7 @@ public class StreamsGroupTest {
StreamsGroup streamsGroup = new StreamsGroup( StreamsGroup streamsGroup = new StreamsGroup(
LOG_CONTEXT, LOG_CONTEXT,
snapshotRegistry, snapshotRegistry,
"group-foo", "group-foo"
mock(GroupCoordinatorMetricsShard.class)
); );
MetadataImage metadataImage = new MetadataImageBuilder() MetadataImage metadataImage = new MetadataImageBuilder()
@ -933,8 +922,7 @@ public class StreamsGroupTest {
StreamsGroup streamsGroup = new StreamsGroup( StreamsGroup streamsGroup = new StreamsGroup(
LOG_CONTEXT, LOG_CONTEXT,
snapshotRegistry, snapshotRegistry,
"test-group", "test-group"
mock(GroupCoordinatorMetricsShard.class)
); );
streamsGroup.updateMember(new StreamsGroupMember.Builder("member1") streamsGroup.updateMember(new StreamsGroupMember.Builder("member1")
.setMemberEpoch(1) .setMemberEpoch(1)
@ -961,8 +949,7 @@ public class StreamsGroupTest {
public void testIsSubscribedToTopic() { public void testIsSubscribedToTopic() {
LogContext logContext = new LogContext(); LogContext logContext = new LogContext();
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(logContext); SnapshotRegistry snapshotRegistry = new SnapshotRegistry(logContext);
GroupCoordinatorMetricsShard metricsShard = mock(GroupCoordinatorMetricsShard.class); StreamsGroup streamsGroup = new StreamsGroup(logContext, snapshotRegistry, "test-group");
StreamsGroup streamsGroup = new StreamsGroup(logContext, snapshotRegistry, "test-group", metricsShard);
assertFalse(streamsGroup.isSubscribedToTopic("test-topic1")); assertFalse(streamsGroup.isSubscribedToTopic("test-topic1"));
assertFalse(streamsGroup.isSubscribedToTopic("test-topic2")); assertFalse(streamsGroup.isSubscribedToTopic("test-topic2"));
@ -1008,8 +995,7 @@ public class StreamsGroupTest {
String memberId2 = "test-member-id2"; String memberId2 = "test-member-id2";
LogContext logContext = new LogContext(); LogContext logContext = new LogContext();
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(logContext); SnapshotRegistry snapshotRegistry = new SnapshotRegistry(logContext);
GroupCoordinatorMetricsShard metricsShard = mock(GroupCoordinatorMetricsShard.class); StreamsGroup streamsGroup = new StreamsGroup(logContext, snapshotRegistry, "test-group");
StreamsGroup streamsGroup = new StreamsGroup(logContext, snapshotRegistry, "test-group", metricsShard);
streamsGroup.updateMember(streamsGroup.getOrCreateDefaultMember(memberId1)); streamsGroup.updateMember(streamsGroup.getOrCreateDefaultMember(memberId1));
streamsGroup.updateMember(streamsGroup.getOrCreateDefaultMember(memberId2)); streamsGroup.updateMember(streamsGroup.getOrCreateDefaultMember(memberId2));

View File

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

View File

@ -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;
}
}

View File

@ -17,6 +17,8 @@
package org.apache.kafka.controller; 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.protocol.ObjectSerializationCache;
import org.apache.kafka.common.utils.BufferSupplier; import org.apache.kafka.common.utils.BufferSupplier;
import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.LogContext;
@ -758,6 +760,11 @@ public final class MockRaftClient implements RaftClient<ApiMessageAndVersion>, A
return OptionalInt.of(nodeId); return OptionalInt.of(nodeId);
} }
@Override
public Optional<Node> voterNode(int id, ListenerName listenerName) {
return Optional.empty();
}
public List<RaftClient.Listener<ApiMessageAndVersion>> listeners() { public List<RaftClient.Listener<ApiMessageAndVersion>> listeners() {
final CompletableFuture<List<RaftClient.Listener<ApiMessageAndVersion>>> future = new CompletableFuture<>(); final CompletableFuture<List<RaftClient.Listener<ApiMessageAndVersion>>> future = new CompletableFuture<>();
eventQueue.append(() -> eventQueue.append(() ->

View File

@ -17,6 +17,8 @@
package org.apache.kafka.image.publisher; 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.common.utils.MockTime;
import org.apache.kafka.image.FakeSnapshotWriter; import org.apache.kafka.image.FakeSnapshotWriter;
import org.apache.kafka.image.MetadataImageTest; import org.apache.kafka.image.MetadataImageTest;
@ -73,6 +75,11 @@ public class SnapshotEmitterTest {
return OptionalInt.empty(); return OptionalInt.empty();
} }
@Override
public Optional<Node> voterNode(int id, ListenerName listenerName) {
return Optional.empty();
}
@Override @Override
public long prepareAppend(int epoch, List<ApiMessageAndVersion> records) { public long prepareAppend(int epoch, List<ApiMessageAndVersion> records) {
return 0; return 0;

View File

@ -3793,6 +3793,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
} }
} }
@Override
public Optional<Node> voterNode(int id, ListenerName listenerName) { public Optional<Node> voterNode(int id, ListenerName listenerName) {
return partitionState.lastVoterSet().voterNode(id, listenerName); return partitionState.lastVoterSet().voterNode(id, listenerName);
} }

View File

@ -16,7 +16,9 @@
*/ */
package org.apache.kafka.raft; package org.apache.kafka.raft;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.errors.ApiException; 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.BufferAllocationException;
import org.apache.kafka.raft.errors.NotLeaderException; import org.apache.kafka.raft.errors.NotLeaderException;
import org.apache.kafka.server.common.KRaftVersion; import org.apache.kafka.server.common.KRaftVersion;
@ -129,6 +131,15 @@ public interface RaftClient<T> extends AutoCloseable {
*/ */
OptionalInt nodeId(); 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. * Prepare a list of records to be appended to the log.
* *

View File

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

View File

@ -57,11 +57,12 @@ import org.apache.kafka.test.TestUtils;
import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.TestInfo;
import org.junit.jupiter.api.Timeout; 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.io.IOException;
import java.lang.reflect.Field; import java.lang.reflect.Field;
@ -76,6 +77,7 @@ import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future; import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException; import java.util.concurrent.TimeoutException;
import java.util.stream.Stream;
import static java.util.Collections.singleton; import static java.util.Collections.singleton;
import static org.apache.kafka.streams.query.StateQueryRequest.inStore; 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); public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(NUM_BROKERS);
private KafkaStreams kafkaStreams; private KafkaStreams kafkaStreams;
private String groupProtocol;
@BeforeAll @BeforeAll
public static void before() public static void before()
@ -149,8 +152,8 @@ public class IQv2IntegrationTest {
)); ));
} }
@BeforeEach private void setup(final String groupProtocol, final TestInfo testInfo) {
public void beforeTest(final TestInfo testInfo) { this.groupProtocol = groupProtocol;
final StreamsBuilder builder = new StreamsBuilder(); final StreamsBuilder builder = new StreamsBuilder();
builder.table( builder.table(
@ -159,7 +162,6 @@ public class IQv2IntegrationTest {
Materialized.as(STORE_NAME) Materialized.as(STORE_NAME)
); );
final String safeTestName = safeUniqueTestName(testInfo); final String safeTestName = safeUniqueTestName(testInfo);
kafkaStreams = new KafkaStreams(builder.build(), streamsConfiguration(safeTestName)); kafkaStreams = new KafkaStreams(builder.build(), streamsConfiguration(safeTestName));
kafkaStreams.cleanUp(); kafkaStreams.cleanUp();
@ -167,17 +169,21 @@ public class IQv2IntegrationTest {
@AfterEach @AfterEach
public void afterTest() { public void afterTest() {
if (kafkaStreams != null) {
kafkaStreams.close(Duration.ofSeconds(60)); kafkaStreams.close(Duration.ofSeconds(60));
kafkaStreams.cleanUp(); kafkaStreams.cleanUp();
} }
}
@AfterAll @AfterAll
public static void after() { public static void after() {
CLUSTER.stop(); CLUSTER.stop();
} }
@Test @ParameterizedTest(name = "{1}")
public void shouldFailUnknownStore() { @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 KeyQuery<Integer, ValueAndTimestamp<Integer>> query = KeyQuery.withKey(1);
final StateQueryRequest<ValueAndTimestamp<Integer>> request = final StateQueryRequest<ValueAndTimestamp<Integer>> request =
inStore("unknown-store").withQuery(query); inStore("unknown-store").withQuery(query);
@ -185,8 +191,10 @@ public class IQv2IntegrationTest {
assertThrows(UnknownStateStoreException.class, () -> kafkaStreams.query(request)); assertThrows(UnknownStateStoreException.class, () -> kafkaStreams.query(request));
} }
@Test @ParameterizedTest(name = "{1}")
public void shouldFailNotStarted() { @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 KeyQuery<Integer, ValueAndTimestamp<Integer>> query = KeyQuery.withKey(1);
final StateQueryRequest<ValueAndTimestamp<Integer>> request = final StateQueryRequest<ValueAndTimestamp<Integer>> request =
inStore(STORE_NAME).withQuery(query); inStore(STORE_NAME).withQuery(query);
@ -194,8 +202,10 @@ public class IQv2IntegrationTest {
assertThrows(StreamsNotStartedException.class, () -> kafkaStreams.query(request)); assertThrows(StreamsNotStartedException.class, () -> kafkaStreams.query(request));
} }
@Test @ParameterizedTest(name = "{1}")
public void shouldFailStopped() { @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 KeyQuery<Integer, ValueAndTimestamp<Integer>> query = KeyQuery.withKey(1);
final StateQueryRequest<ValueAndTimestamp<Integer>> request = final StateQueryRequest<ValueAndTimestamp<Integer>> request =
inStore(STORE_NAME).withQuery(query); inStore(STORE_NAME).withQuery(query);
@ -205,9 +215,11 @@ public class IQv2IntegrationTest {
assertThrows(StreamsStoppedException.class, () -> kafkaStreams.query(request)); assertThrows(StreamsStoppedException.class, () -> kafkaStreams.query(request));
} }
@Test @ParameterizedTest(name = "{1}")
public void shouldRejectNonRunningActive() @MethodSource("groupProtocolParameters")
public void shouldRejectNonRunningActive(final String groupProtocol, final String testName, final TestInfo testInfo)
throws NoSuchFieldException, IllegalAccessException { throws NoSuchFieldException, IllegalAccessException {
setup(groupProtocol, testInfo);
final KeyQuery<Integer, ValueAndTimestamp<Integer>> query = KeyQuery.withKey(1); final KeyQuery<Integer, ValueAndTimestamp<Integer>> query = KeyQuery.withKey(1);
final StateQueryRequest<ValueAndTimestamp<Integer>> request = final StateQueryRequest<ValueAndTimestamp<Integer>> request =
inStore(STORE_NAME).withQuery(query).requireActive(); inStore(STORE_NAME).withQuery(query).requireActive();
@ -261,8 +273,10 @@ public class IQv2IntegrationTest {
} }
} }
@Test @ParameterizedTest(name = "{1}")
public void shouldFetchFromPartition() { @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 KeyQuery<Integer, ValueAndTimestamp<Integer>> query = KeyQuery.withKey(1);
final int partition = 1; final int partition = 1;
final Set<Integer> partitions = singleton(partition); final Set<Integer> partitions = singleton(partition);
@ -276,8 +290,10 @@ public class IQv2IntegrationTest {
assertThat(result.getPartitionResults().keySet(), equalTo(partitions)); assertThat(result.getPartitionResults().keySet(), equalTo(partitions));
} }
@Test @ParameterizedTest(name = "{1}")
public void shouldFetchExplicitlyFromAllPartitions() { @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 KeyQuery<Integer, ValueAndTimestamp<Integer>> query = KeyQuery.withKey(1);
final Set<Integer> partitions = Set.of(0, 1); final Set<Integer> partitions = Set.of(0, 1);
final StateQueryRequest<ValueAndTimestamp<Integer>> request = final StateQueryRequest<ValueAndTimestamp<Integer>> request =
@ -290,8 +306,10 @@ public class IQv2IntegrationTest {
assertThat(result.getPartitionResults().keySet(), equalTo(partitions)); assertThat(result.getPartitionResults().keySet(), equalTo(partitions));
} }
@Test @ParameterizedTest(name = "{1}")
public void shouldNotRequireQueryHandler(final TestInfo testInfo) { @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 KeyQuery<Integer, ValueAndTimestamp<Integer>> query = KeyQuery.withKey(1);
final int partition = 1; final int partition = 1;
final Set<Integer> partitions = singleton(partition); final Set<Integer> partitions = singleton(partition);
@ -423,8 +441,11 @@ public class IQv2IntegrationTest {
); );
// Discard the basic streams and replace with test-specific topology // Discard the basic streams and replace with test-specific topology
if (kafkaStreams != null) {
kafkaStreams.close(); kafkaStreams.close();
}
final String safeTestName = safeUniqueTestName(testInfo); final String safeTestName = safeUniqueTestName(testInfo);
this.groupProtocol = groupProtocol;
kafkaStreams = new KafkaStreams(builder.build(), streamsConfiguration(safeTestName)); kafkaStreams = new KafkaStreams(builder.build(), streamsConfiguration(safeTestName));
kafkaStreams.cleanUp(); kafkaStreams.cleanUp();
@ -446,7 +467,7 @@ public class IQv2IntegrationTest {
private Properties streamsConfiguration(final String safeTestName) { private Properties streamsConfiguration(final String safeTestName) {
final Properties config = new Properties(); final Properties config = new Properties();
config.put(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG, StreamsConfig.OPTIMIZE); 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.APPLICATION_SERVER_CONFIG, "localhost:" + (++port));
config.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); config.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
config.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); 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(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 1000);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L); config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
config.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1); config.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1);
config.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, groupProtocol);
return config; return config;
} }
private static Stream<Arguments> groupProtocolParameters() {
return Stream.of(
Arguments.of("classic", "CLASSIC protocol"),
Arguments.of("streams", "STREAMS protocol")
);
}
} }

View File

@ -360,7 +360,9 @@ public class IQv2StoreIntegrationTest {
for (final boolean logEnabled : Arrays.asList(true, false)) { for (final boolean logEnabled : Arrays.asList(true, false)) {
for (final StoresToTest toTest : StoresToTest.values()) { for (final StoresToTest toTest : StoresToTest.values()) {
for (final String kind : Arrays.asList("DSL", "PAPI")) { 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 StoreSupplier<?> supplier = storeToTest.supplier();
final Properties streamsConfig = streamsConfiguration( final Properties streamsConfig = streamsConfiguration(
cache, cache,
log, log,
storeToTest.name(), storeToTest.name(),
kind kind,
groupProtocol
); );
final StreamsBuilder builder = new StreamsBuilder(); final StreamsBuilder builder = new StreamsBuilder();
@ -765,8 +768,8 @@ public class IQv2StoreIntegrationTest {
@ParameterizedTest @ParameterizedTest
@MethodSource("data") @MethodSource("data")
public void verifyStore(final boolean cache, final boolean log, final StoresToTest storeToTest, final String kind) { public void verifyStore(final boolean cache, final boolean log, final StoresToTest storeToTest, final String kind, final String groupProtocol) {
setup(cache, log, storeToTest, kind); setup(cache, log, storeToTest, kind, groupProtocol);
try { try {
if (storeToTest.global()) { if (storeToTest.global()) {
// See KAFKA-13523 // See KAFKA-13523
@ -2030,10 +2033,10 @@ public class IQv2StoreIntegrationTest {
} }
private static Properties streamsConfiguration(final boolean cache, final boolean log, 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 = final String safeTestName =
IQv2StoreIntegrationTest.class.getName() + "-" + cache + "-" + log + "-" + supplier IQv2StoreIntegrationTest.class.getName() + "-" + cache + "-" + log + "-" + supplier
+ "-" + kind + "-" + RANDOM.nextInt(); + "-" + kind + "-" + groupProtocol + "-" + RANDOM.nextInt();
final Properties config = new Properties(); final Properties config = new Properties();
config.put(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG, StreamsConfig.OPTIMIZE); config.put(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG, StreamsConfig.OPTIMIZE);
config.put(StreamsConfig.APPLICATION_ID_CONFIG, "app-" + safeTestName); 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(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 1000);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L); config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
config.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1); config.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1);
config.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, groupProtocol);
return config; return config;
} }
} }

View File

@ -47,7 +47,10 @@ import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Tag; 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.Duration;
import java.time.Instant; import java.time.Instant;
@ -57,7 +60,9 @@ import java.util.Map;
import java.util.Map.Entry; import java.util.Map.Entry;
import java.util.Optional; import java.util.Optional;
import java.util.Properties; 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.CoreMatchers.nullValue;
import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.empty; 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 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 RECORD_NUMBER = RECORD_VALUES.length;
private static final int LAST_INDEX = RECORD_NUMBER - 1; 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"))); public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(NUM_BROKERS, Utils.mkProperties(Collections.singletonMap("auto.create.topics.enable", "true")));
private KafkaStreams kafkaStreams; private KafkaStreams kafkaStreams;
private String groupProtocol;
@BeforeAll @BeforeAll
public static void before() throws Exception { public static void beforeAll() throws Exception {
CLUSTER.start(); 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(); final Properties producerProps = new Properties();
producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, IntegerSerializer.class); 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[2], RECORD_KEY, RECORD_VALUES[2])).get();
producer.send(new ProducerRecord<>(INPUT_TOPIC_NAME, 0, RECORD_TIMESTAMPS[3], RECORD_KEY, RECORD_VALUES[3])).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 private void setup(final String groupProtocol, final TestInfo testInfo) {
public void beforeTest() { this.groupProtocol = groupProtocol;
final StreamsBuilder builder = new StreamsBuilder(); final StreamsBuilder builder = new StreamsBuilder();
builder.table(INPUT_TOPIC_NAME, builder.table(INPUT_TOPIC_NAME,
Materialized.as(Stores.persistentVersionedKeyValueStore(STORE_NAME, HISTORY_RETENTION, SEGMENT_INTERVAL))); Materialized.as(Stores.persistentVersionedKeyValueStore(STORE_NAME, HISTORY_RETENTION, SEGMENT_INTERVAL)));
final Properties configs = new Properties(); 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.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
configs.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.IntegerSerde.class.getName()); 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.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.IntegerSerde.class.getName());
configs.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, groupProtocol);
kafkaStreams = IntegrationTestUtils.getStartedStreams(configs, builder, true); kafkaStreams = IntegrationTestUtils.getStartedStreams(configs, builder, true);
} }
@ -132,8 +148,19 @@ public class IQv2VersionedStoreIntegrationTest {
CLUSTER.stop(); CLUSTER.stop();
} }
@Test private static Stream<Arguments> groupProtocolParameters() {
public void verifyStore() { 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 */ /* Test Versioned Key Queries */
// retrieve the latest value // retrieve the latest value
shouldHandleVersionedKeyQuery(Optional.empty(), RECORD_VALUES[3], RECORD_TIMESTAMPS[3], Optional.empty()); shouldHandleVersionedKeyQuery(Optional.empty(), RECORD_VALUES[3], RECORD_TIMESTAMPS[3], Optional.empty());
@ -255,7 +282,10 @@ public class IQv2VersionedStoreIntegrationTest {
private void shouldHandleRaceCondition() { private void shouldHandleRaceCondition() {
final MultiVersionedKeyQuery<Integer, Integer> query = defineQuery(RECORD_KEY, Optional.empty(), Optional.empty(), ResultOrder.ANY); 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 // verify results in two steps
for (final Entry<Integer, QueryResult<VersionedRecordIterator<Integer>>> partitionResultsEntry : partitionResults.entrySet()) { for (final Entry<Integer, QueryResult<VersionedRecordIterator<Integer>>> partitionResultsEntry : partitionResults.entrySet()) {
@ -327,14 +357,14 @@ public class IQv2VersionedStoreIntegrationTest {
return query; return query;
} }
private static Map<Integer, QueryResult<VersionedRecordIterator<Integer>>> sendRequestAndReceiveResults(final MultiVersionedKeyQuery<Integer, Integer> query, final KafkaStreams kafkaStreams) { 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(INPUT_POSITION)); final StateQueryRequest<VersionedRecordIterator<Integer>> request = StateQueryRequest.inStore(STORE_NAME).withQuery(query).withPositionBound(PositionBound.at(inputPosition));
final StateQueryResult<VersionedRecordIterator<Integer>> result = IntegrationTestUtils.iqv2WaitForResult(kafkaStreams, request); final StateQueryResult<VersionedRecordIterator<Integer>> result = IntegrationTestUtils.iqv2WaitForResult(kafkaStreams, request);
return result.getPartitionResults(); return result.getPartitionResults();
} }
private static QueryResult<VersionedRecord<Integer>> sendRequestAndReceiveResults(final VersionedKeyQuery<Integer, Integer> query, final KafkaStreams kafkaStreams) { 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(INPUT_POSITION)); final StateQueryRequest<VersionedRecord<Integer>> request = StateQueryRequest.inStore(STORE_NAME).withQuery(query).withPositionBound(PositionBound.at(inputPosition));
final StateQueryResult<VersionedRecord<Integer>> result = IntegrationTestUtils.iqv2WaitForResult(kafkaStreams, request); final StateQueryResult<VersionedRecord<Integer>> result = IntegrationTestUtils.iqv2WaitForResult(kafkaStreams, request);
return result.getOnlyPartitionResult(); 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]). * 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] // update the record value at RECORD_TIMESTAMPS[0]
final Properties producerProps = new Properties(); final Properties producerProps = new Properties();
producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
@ -361,8 +391,9 @@ public class IQv2VersionedStoreIntegrationTest {
try (final KafkaProducer<Integer, Integer> producer = new KafkaProducer<>(producerProps)) { try (final KafkaProducer<Integer, Integer> producer = new KafkaProducer<>(producerProps)) {
producer.send(new ProducerRecord<>(INPUT_TOPIC_NAME, 0, RECORD_TIMESTAMPS[0], RECORD_KEY, 999999)); 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 // make sure that the new value is picked up by the store
final Properties consumerProps = new Properties(); final Properties consumerProps = new Properties();

View File

@ -1122,15 +1122,15 @@ public class KStreamAggregationIntegrationTest {
final String[] args = new String[] { final String[] args = new String[] {
"--bootstrap-server", CLUSTER.bootstrapServers(), "--bootstrap-server", CLUSTER.bootstrapServers(),
"--from-beginning", "--from-beginning",
"--property", "print.key=true", "--formatter-property", "print.key=true",
"--property", "print.timestamp=" + printTimestamp, "--formatter-property", "print.timestamp=" + printTimestamp,
"--topic", outputTopic, "--topic", outputTopic,
"--max-messages", String.valueOf(numMessages), "--max-messages", String.valueOf(numMessages),
"--property", "key.deserializer=" + keyDeserializer.getClass().getName(), "--formatter-property", "key.deserializer=" + keyDeserializer.getClass().getName(),
"--property", "value.deserializer=" + valueDeserializer.getClass().getName(), "--formatter-property", "value.deserializer=" + valueDeserializer.getClass().getName(),
"--property", "key.separator=" + keySeparator, "--formatter-property", "key.separator=" + keySeparator,
"--property", "key.deserializer." + TimeWindowedDeserializer.WINDOWED_INNER_DESERIALIZER_CLASS + "=" + Serdes.serdeFrom(innerClass).getClass().getName(), "--formatter-property", "key.deserializer." + TimeWindowedDeserializer.WINDOWED_INNER_DESERIALIZER_CLASS + "=" + Serdes.serdeFrom(innerClass).getClass().getName(),
"--property", "key.deserializer.window.size.ms=500", "--formatter-property", "key.deserializer.window.size.ms=500",
}; };
ConsoleConsumer.run(new ConsoleConsumerOptions(args)); ConsoleConsumer.run(new ConsoleConsumerOptions(args));

View File

@ -89,6 +89,7 @@ public class DefaultStreamsRebalanceListener implements StreamsRebalanceListener
taskManager.handleAssignment(activeTasksWithPartitions, standbyTasksWithPartitions); taskManager.handleAssignment(activeTasksWithPartitions, standbyTasksWithPartitions);
streamThread.setState(StreamThread.State.PARTITIONS_ASSIGNED); streamThread.setState(StreamThread.State.PARTITIONS_ASSIGNED);
taskManager.handleRebalanceComplete(); taskManager.handleRebalanceComplete();
streamsRebalanceData.setReconciledAssignment(assignment);
} catch (final Exception exception) { } catch (final Exception exception) {
return Optional.of(exception); return Optional.of(exception);
} }
@ -99,6 +100,7 @@ public class DefaultStreamsRebalanceListener implements StreamsRebalanceListener
public Optional<Exception> onAllTasksLost() { public Optional<Exception> onAllTasksLost() {
try { try {
taskManager.handleLostAll(); taskManager.handleLostAll();
streamsRebalanceData.setReconciledAssignment(StreamsRebalanceData.Assignment.EMPTY);
} catch (final Exception exception) { } catch (final Exception exception) {
return Optional.of(exception); return Optional.of(exception);
} }

View File

@ -118,10 +118,8 @@ public class DefaultStreamsRebalanceListenerTest {
@Test @Test
void testOnTasksAssigned() { void testOnTasksAssigned() {
createRebalanceListenerWithRebalanceData(new StreamsRebalanceData( final StreamsRebalanceData streamsRebalanceData = mock(StreamsRebalanceData.class);
UUID.randomUUID(), when(streamsRebalanceData.subtopologies()).thenReturn(Map.of(
Optional.empty(),
Map.of(
"1", "1",
new StreamsRebalanceData.Subtopology( new StreamsRebalanceData.Subtopology(
Set.of("source1"), Set.of("source1"),
@ -146,21 +144,20 @@ public class DefaultStreamsRebalanceListenerTest {
Map.of(), Map.of(),
Set.of() Set.of()
) )
),
Map.of()
)); ));
createRebalanceListenerWithRebalanceData(streamsRebalanceData);
final Optional<Exception> result = defaultStreamsRebalanceListener.onTasksAssigned( final StreamsRebalanceData.Assignment assignment = new StreamsRebalanceData.Assignment(
new StreamsRebalanceData.Assignment(
Set.of(new StreamsRebalanceData.TaskId("1", 0)), Set.of(new StreamsRebalanceData.TaskId("1", 0)),
Set.of(new StreamsRebalanceData.TaskId("2", 0)), Set.of(new StreamsRebalanceData.TaskId("2", 0)),
Set.of(new StreamsRebalanceData.TaskId("3", 0)) Set.of(new StreamsRebalanceData.TaskId("3", 0))
)
); );
final Optional<Exception> result = defaultStreamsRebalanceListener.onTasksAssigned(assignment);
assertTrue(result.isEmpty()); assertTrue(result.isEmpty());
final InOrder inOrder = inOrder(taskManager, streamThread); final InOrder inOrder = inOrder(taskManager, streamThread, streamsRebalanceData);
inOrder.verify(taskManager).handleAssignment( inOrder.verify(taskManager).handleAssignment(
Map.of(new TaskId(1, 0), Set.of(new TopicPartition("source1", 0), new TopicPartition("repartition1", 0))), Map.of(new TaskId(1, 0), Set.of(new TopicPartition("source1", 0), new TopicPartition("repartition1", 0))),
Map.of( Map.of(
@ -170,6 +167,7 @@ public class DefaultStreamsRebalanceListenerTest {
); );
inOrder.verify(streamThread).setState(StreamThread.State.PARTITIONS_ASSIGNED); inOrder.verify(streamThread).setState(StreamThread.State.PARTITIONS_ASSIGNED);
inOrder.verify(taskManager).handleRebalanceComplete(); inOrder.verify(taskManager).handleRebalanceComplete();
inOrder.verify(streamsRebalanceData).setReconciledAssignment(assignment);
} }
@Test @Test
@ -177,21 +175,32 @@ public class DefaultStreamsRebalanceListenerTest {
final Exception exception = new RuntimeException("sample exception"); final Exception exception = new RuntimeException("sample exception");
doThrow(exception).when(taskManager).handleAssignment(any(), any()); doThrow(exception).when(taskManager).handleAssignment(any(), any());
createRebalanceListenerWithRebalanceData(new StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of())); final StreamsRebalanceData streamsRebalanceData = mock(StreamsRebalanceData.class);
final Optional<Exception> result = defaultStreamsRebalanceListener.onTasksAssigned(new StreamsRebalanceData.Assignment(Set.of(), Set.of(), Set.of())); when(streamsRebalanceData.subtopologies()).thenReturn(Map.of());
assertTrue(defaultStreamsRebalanceListener.onAllTasksLost().isEmpty()); createRebalanceListenerWithRebalanceData(streamsRebalanceData);
final Optional<Exception> result = defaultStreamsRebalanceListener.onTasksAssigned(
new StreamsRebalanceData.Assignment(Set.of(), Set.of(), Set.of())
);
assertTrue(result.isPresent()); assertTrue(result.isPresent());
assertEquals(exception, result.get()); assertEquals(exception, result.get());
verify(taskManager).handleLostAll(); verify(taskManager).handleAssignment(any(), any());
verify(streamThread, never()).setState(StreamThread.State.PARTITIONS_ASSIGNED); verify(streamThread, never()).setState(StreamThread.State.PARTITIONS_ASSIGNED);
verify(taskManager, never()).handleRebalanceComplete(); verify(taskManager, never()).handleRebalanceComplete();
verify(streamsRebalanceData, never()).setReconciledAssignment(any());
} }
@Test @Test
void testOnAllTasksLost() { 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()); 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 @Test
@ -199,10 +208,13 @@ public class DefaultStreamsRebalanceListenerTest {
final Exception exception = new RuntimeException("sample exception"); final Exception exception = new RuntimeException("sample exception");
doThrow(exception).when(taskManager).handleLostAll(); 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(); final Optional<Exception> result = defaultStreamsRebalanceListener.onAllTasksLost();
assertTrue(result.isPresent()); assertTrue(result.isPresent());
assertEquals(exception, result.get()); assertEquals(exception, result.get());
verify(taskManager).handleLostAll(); verify(taskManager).handleLostAll();
verify(streamsRebalanceData, never()).setReconciledAssignment(any());
} }
} }

View File

@ -152,6 +152,7 @@ public class TaskManagerTest {
private final TopicPartition t1p1 = new TopicPartition(topic1, 1); private final TopicPartition t1p1 = new TopicPartition(topic1, 1);
private final TopicPartition t2p2 = new TopicPartition(topic2, 1); private final TopicPartition t2p2 = new TopicPartition(topic2, 1);
private final TopicPartition t1p1changelog = new TopicPartition("changelog", 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> taskId01Partitions = Set.of(t1p1);
private final Set<TopicPartition> taskId01ChangelogPartitions = Set.of(t1p1changelog); private final Set<TopicPartition> taskId01ChangelogPartitions = Set.of(t1p1changelog);
private final Map<TaskId, Set<TopicPartition>> taskId01Assignment = singletonMap(taskId01, taskId01Partitions); 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); 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) { private TaskManager setUpTaskManager(final ProcessingMode processingMode, final boolean stateUpdaterEnabled) {
return setUpTaskManager(processingMode, null, stateUpdaterEnabled, false); return setUpTaskManager(processingMode, null, stateUpdaterEnabled, false);
} }
@ -249,52 +254,6 @@ public class TaskManagerTest {
return taskManager; 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 @Test
public void shouldLockAllTasksOnCorruptionWithProcessingThreads() { public void shouldLockAllTasksOnCorruptionWithProcessingThreads() {
@ -1853,14 +1812,20 @@ public class TaskManagerTest {
} }
@Test @Test
public void shouldReportLatestOffsetAsOffsetSumForRunningTask() throws Exception { public void shouldComputeOffsetSumForRunningStatefulTask() {
final Map<TopicPartition, Long> changelogOffsets = mkMap( final StreamTask runningStatefulTask = statefulTask(taskId00, taskId00ChangelogPartitions)
mkEntry(new TopicPartition("changelog", 0), Task.LATEST_OFFSET), .inState(State.RUNNING).build();
mkEntry(new TopicPartition("changelog", 1), Task.LATEST_OFFSET) final long changelogOffsetOfRunningTask = Task.LATEST_OFFSET;
); when(runningStatefulTask.changelogOffsets())
final Map<TaskId, Long> expectedOffsetSums = mkMap(mkEntry(taskId00, Task.LATEST_OFFSET)); .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 @Test
@ -1911,14 +1876,14 @@ public class TaskManagerTest {
} }
@Test @Test
public void shouldComputeOffsetSumForRunningStatefulTaskAndRestoringTaskWithStateUpdater() { public void shouldComputeOffsetSumForRunningStatefulTaskAndRestoringTask() {
final StreamTask runningStatefulTask = statefulTask(taskId00, taskId00ChangelogPartitions) final StreamTask runningStatefulTask = statefulTask(taskId00, taskId00ChangelogPartitions)
.inState(State.RUNNING).build(); .inState(State.RUNNING).build();
final StreamTask restoringStatefulTask = statefulTask(taskId01, taskId01ChangelogPartitions) final StreamTask restoringStatefulTask = statefulTask(taskId01, taskId01ChangelogPartitions)
.inState(State.RESTORING).build(); .inState(State.RESTORING).build();
final StandbyTask restoringStandbyTask = standbyTask(taskId02, taskId02ChangelogPartitions) final StandbyTask restoringStandbyTask = standbyTask(taskId02, taskId02ChangelogPartitions)
.inState(State.RUNNING).build(); .inState(State.RUNNING).build();
final long changelogOffsetOfRunningTask = 42L; final long changelogOffsetOfRunningTask = Task.LATEST_OFFSET;
final long changelogOffsetOfRestoringStatefulTask = 24L; final long changelogOffsetOfRestoringStatefulTask = 24L;
final long changelogOffsetOfRestoringStandbyTask = 84L; final long changelogOffsetOfRestoringStandbyTask = 84L;
when(runningStatefulTask.changelogOffsets()) when(runningStatefulTask.changelogOffsets())
@ -1943,14 +1908,26 @@ public class TaskManagerTest {
} }
@Test @Test
public void shouldSkipUnknownOffsetsWhenComputingOffsetSum() throws Exception { public void shouldSkipUnknownOffsetsWhenComputingOffsetSum() {
final Map<TopicPartition, Long> changelogOffsets = mkMap( final StreamTask restoringStatefulTask = statefulTask(taskId01, taskId01ChangelogPartitions)
mkEntry(new TopicPartition("changelog", 0), OffsetCheckpoint.OFFSET_UNKNOWN), .inState(State.RESTORING).build();
mkEntry(new TopicPartition("changelog", 1), 10L) final long changelogOffsetOfRestoringStandbyTask = 84L;
); when(restoringStatefulTask.changelogOffsets())
final Map<TaskId, Long> expectedOffsetSums = mkMap(mkEntry(taskId00, 10L)); .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, private void computeOffsetSumAndVerify(final Map<TopicPartition, Long> changelogOffsets,

View File

@ -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 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 kafka_opts_override Override parameters of the KAFKA_OPTS environment variable
client_prop_file_override Override client.properties file used by the consumer 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 []), JmxMixin.__init__(self, num_nodes=num_nodes, jmx_object_names=jmx_object_names, jmx_attributes=(jmx_attributes or []),
root=ConsoleConsumer.PERSISTENT_ROOT) 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_LOG4J_OPTS=\"%(log4j_param)s%(log4j_config)s\"; " \
"export KAFKA_OPTS=%(kafka_opts)s; " \ "export KAFKA_OPTS=%(kafka_opts)s; " \
"%(console_consumer)s " \ "%(console_consumer)s " \
"--topic %(topic)s " \ "--topic %(topic)s " % args
"--consumer.config %(config_file)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 += " --bootstrap-server %(broker_list)s" % args
cmd += " --isolation-level %s" % self.isolation_level 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 # This will be added in the properties file instead
cmd += " --timeout-ms %s" % self.consumer_timeout_ms cmd += " --timeout-ms %s" % self.consumer_timeout_ms
formatter_property_arg = "--formatter-property" if version.supports_formatter_property() else "--property"
if self.print_timestamp: if self.print_timestamp:
cmd += " --property print.timestamp=true" cmd += " %s print.timestamp=true" % formatter_property_arg
if self.print_key: if self.print_key:
cmd += " --property print.key=true" cmd += " %s print.key=true" % formatter_property_arg
if self.print_partition: if self.print_partition:
cmd += " --property print.partition=true" cmd += " %s print.partition=true" % formatter_property_arg
# LoggingMessageFormatter was introduced after 0.9 # LoggingMessageFormatter was introduced after 0.9
if node.version > LATEST_3_7: if node.version > LATEST_3_7:
@ -194,9 +198,10 @@ class ConsoleConsumer(KafkaPathResolverMixin, JmxMixin, BackgroundThreadService)
if self.enable_systest_events: if self.enable_systest_events:
cmd += " --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: if self.consumer_properties is not None:
for k, v in self.consumer_properties.items(): 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 cmd += " 2>> %(stderr)s | tee -a %(stdout)s &" % args
return cmd return cmd

View File

@ -21,7 +21,7 @@ from ducktape.utils.util import wait_until
from kafkatest.directory_layout.kafka_path import KafkaPathResolverMixin from kafkatest.directory_layout.kafka_path import KafkaPathResolverMixin
from kafkatest.services.monitor.jmx import JmxMixin 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 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 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 kafka_opts_override Override parameters of the KAFKA_OPTS environment variable
client_prop_file_override Override client.properties file used by the consumer 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 []), JmxMixin.__init__(self, num_nodes=num_nodes, jmx_object_names=jmx_object_names, jmx_attributes=(jmx_attributes or []),
root=ConsoleShareConsumer.PERSISTENT_ROOT) 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_LOG4J_OPTS=\"%(log4j_param)s%(log4j_config)s\"; " \
"export KAFKA_OPTS=%(kafka_opts)s; " \ "export KAFKA_OPTS=%(kafka_opts)s; " \
"%(console_share_consumer)s " \ "%(console_share_consumer)s " \
"--topic %(topic)s " \ "--topic %(topic)s " % args
"--consumer-config %(config_file)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 += " --bootstrap-server %(broker_list)s" % args
if self.share_consumer_timeout_ms is not None: if self.share_consumer_timeout_ms is not None:
# This will be added in the properties file instead # This will be added in the properties file instead
cmd += " --timeout-ms %s" % self.share_consumer_timeout_ms 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: if self.print_timestamp:
cmd += " --property print.timestamp=true" cmd += " %s print.timestamp=true" % formatter_property_arg
if self.print_key: if self.print_key:
cmd += " --property print.key=true" cmd += " %s print.key=true" % formatter_property_arg
if self.print_partition: 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" cmd += " --formatter org.apache.kafka.tools.consumer.LoggingMessageFormatter"
if self.enable_systest_events: if self.enable_systest_events:
cmd += " --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: if self.share_consumer_properties is not None:
for k, v in self.share_consumer_properties.items(): 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 cmd += " 2>> %(stderr)s | tee -a %(stdout)s &" % args
return cmd return cmd

View File

@ -104,6 +104,20 @@ class KafkaVersion(LooseVersion):
# - For older versions, continue using --producer.config or --consumer.config # - For older versions, continue using --producer.config or --consumer.config
return self >= V_4_2_0 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): def get_version(node=None):
"""Return the version attached to the given node. """Return the version attached to the given node.
Default to DEV_BRANCH if node or node.version is undefined (aka None) Default to DEV_BRANCH if node or node.version is undefined (aka None)

View File

@ -131,8 +131,12 @@ public class ConsoleProducer {
private final OptionSpec<Integer> socketBufferSizeOpt; private final OptionSpec<Integer> socketBufferSizeOpt;
private final OptionSpec<String> propertyOpt; private final OptionSpec<String> propertyOpt;
private final OptionSpec<String> readerConfigOpt; private final OptionSpec<String> readerConfigOpt;
@Deprecated(since = "4.2", forRemoval = true)
private final OptionSpec<String> producerPropertyOpt; private final OptionSpec<String> producerPropertyOpt;
private OptionSpec<String> commandPropertyOpt;
@Deprecated(since = "4.2", forRemoval = true)
private final OptionSpec<String> producerConfigOpt; private final OptionSpec<String> producerConfigOpt;
private OptionSpec<String> commandConfigOpt;
public ConsoleProducerOptions(String[] args) { public ConsoleProducerOptions(String[] args) {
super(args); super(args);
@ -250,11 +254,20 @@ public class ConsoleProducer {
.withRequiredArg() .withRequiredArg()
.describedAs("config file") .describedAs("config file")
.ofType(String.class); .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() .withRequiredArg()
.describedAs("producer_prop") .describedAs("producer_prop")
.ofType(String.class); .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() .withRequiredArg()
.describedAs("config file") .describedAs("config file")
.ofType(String.class); .ofType(String.class);
@ -273,6 +286,23 @@ public class ConsoleProducer {
CommandLineUtils.checkRequiredArgs(parser, options, topicOpt); 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 { try {
ToolsUtils.validateBootstrapServer(options.valueOf(bootstrapServerOpt)); ToolsUtils.validateBootstrapServer(options.valueOf(bootstrapServerOpt));
} catch (IllegalArgumentException e) { } catch (IllegalArgumentException e) {
@ -314,11 +344,11 @@ public class ConsoleProducer {
Properties producerProps() throws IOException { Properties producerProps() throws IOException {
Properties props = new Properties(); Properties props = new Properties();
if (options.has(producerConfigOpt)) { if (options.has(commandConfigOpt)) {
props.putAll(loadProps(options.valueOf(producerConfigOpt))); 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(BOOTSTRAP_SERVERS_CONFIG, options.valueOf(bootstrapServerOpt));
props.put(COMPRESSION_TYPE_CONFIG, compressionCodec()); props.put(COMPRESSION_TYPE_CONFIG, compressionCodec());

View File

@ -48,7 +48,9 @@ public final class ConsoleConsumerOptions extends CommandDefaultOptions {
private final OptionSpec<Integer> partitionIdOpt; private final OptionSpec<Integer> partitionIdOpt;
private final OptionSpec<String> offsetOpt; private final OptionSpec<String> offsetOpt;
private final OptionSpec<String> messageFormatterOpt; 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<String> messageFormatterConfigOpt;
private final OptionSpec<?> resetBeginningOpt; private final OptionSpec<?> resetBeginningOpt;
private final OptionSpec<Integer> maxMessagesOpt; private final OptionSpec<Integer> maxMessagesOpt;
@ -66,9 +68,10 @@ public final class ConsoleConsumerOptions extends CommandDefaultOptions {
private final long timeoutMs; private final long timeoutMs;
private final MessageFormatter formatter; private final MessageFormatter formatter;
@SuppressWarnings("deprecation")
public ConsoleConsumerOptions(String[] args) throws IOException { public ConsoleConsumerOptions(String[] args) throws IOException {
super(args); super(args);
topicOpt = parser.accepts("topic", "The topic to consume on.") topicOpt = parser.accepts("topic", "The topic to consume from.")
.withRequiredArg() .withRequiredArg()
.describedAs("topic") .describedAs("topic")
.ofType(String.class); .ofType(String.class);
@ -87,11 +90,23 @@ public final class ConsoleConsumerOptions extends CommandDefaultOptions {
.describedAs("consume offset") .describedAs("consume offset")
.ofType(String.class) .ofType(String.class)
.defaultsTo("latest"); .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() .withRequiredArg()
.describedAs("consumer_prop") .describedAs("consumer_prop")
.ofType(String.class); .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() .withRequiredArg()
.describedAs("config file") .describedAs("config file")
.ofType(String.class); .ofType(String.class);
@ -100,7 +115,28 @@ public final class ConsoleConsumerOptions extends CommandDefaultOptions {
.describedAs("class") .describedAs("class")
.ofType(String.class) .ofType(String.class)
.defaultsTo(DefaultMessageFormatter.class.getName()); .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" + "The properties to initialize the message formatter. Default properties include: \n" +
" print.timestamp=true|false\n" + " print.timestamp=true|false\n" +
" print.key=true|false\n" + " print.key=true|false\n" +
@ -140,11 +176,11 @@ public final class ConsoleConsumerOptions extends CommandDefaultOptions {
.withRequiredArg() .withRequiredArg()
.describedAs("server to connect to") .describedAs("server to connect to")
.ofType(String.class); .ofType(String.class);
keyDeserializerOpt = parser.accepts("key-deserializer") keyDeserializerOpt = parser.accepts("key-deserializer", "The name of the class to use for deserializing keys.")
.withRequiredArg() .withRequiredArg()
.describedAs("deserializer for key") .describedAs("deserializer for keys")
.ofType(String.class); .ofType(String.class);
valueDeserializerOpt = parser.accepts("value-deserializer") valueDeserializerOpt = parser.accepts("value-deserializer", "The name of the class to use for deserializing values.")
.withRequiredArg() .withRequiredArg()
.describedAs("deserializer for values") .describedAs("deserializer for values")
.ofType(String.class); .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."); CommandLineUtils.maybePrintHelpOrVersion(this, "This tool helps to read data from Kafka topics and outputs it to standard output.");
checkRequiredArgs(); 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) if (options.has(consumerPropertyOpt)) {
? Utils.loadProps(options.valueOf(consumerConfigOpt)) 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(); : new Properties();
Properties extraConsumerProps = CommandLineUtils.parseKeyValueArgs(options.valuesOf(consumerPropertyOpt)); Properties extraConsumerProps = CommandLineUtils.parseKeyValueArgs(options.valuesOf(commandPropertyOpt));
Set<String> groupIdsProvided = checkConsumerGroup(consumerPropsFromFile, extraConsumerProps); Set<String> groupIdsProvided = checkConsumerGroup(consumerPropsFromFile, extraConsumerProps);
consumerProps = buildConsumerProps(consumerPropsFromFile, extraConsumerProps, groupIdsProvided); consumerProps = buildConsumerProps(consumerPropsFromFile, extraConsumerProps, groupIdsProvided);
offset = parseOffset(); offset = parseOffset();
@ -323,6 +373,13 @@ public final class ConsoleConsumerOptions extends CommandDefaultOptions {
Class<?> messageFormatterClass = Class.forName(options.valueOf(messageFormatterOpt)); Class<?> messageFormatterClass = Class.forName(options.valueOf(messageFormatterOpt));
formatter = (MessageFormatter) messageFormatterClass.getDeclaredConstructor().newInstance(); 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(); Properties formatterArgs = formatterArgs();
Map<String, String> formatterConfigs = new HashMap<>(); Map<String, String> formatterConfigs = new HashMap<>();
for (final String name : formatterArgs.stringPropertyNames()) { for (final String name : formatterArgs.stringPropertyNames()) {

View File

@ -37,7 +37,9 @@ import joptsimple.OptionSpec;
public final class ConsoleShareConsumerOptions extends CommandDefaultOptions { public final class ConsoleShareConsumerOptions extends CommandDefaultOptions {
private final OptionSpec<String> messageFormatterOpt; private final OptionSpec<String> messageFormatterOpt;
private final OptionSpec<String> messageFormatterConfigOpt; 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> keyDeserializerOpt;
private final OptionSpec<String> valueDeserializerOpt; private final OptionSpec<String> valueDeserializerOpt;
private final OptionSpec<Integer> maxMessagesOpt; private final OptionSpec<Integer> maxMessagesOpt;
@ -52,17 +54,30 @@ public final class ConsoleShareConsumerOptions extends CommandDefaultOptions {
private final MessageFormatter formatter; private final MessageFormatter formatter;
private final OptionSpec<?> enableSystestEventsLoggingOpt; private final OptionSpec<?> enableSystestEventsLoggingOpt;
@SuppressWarnings("deprecation")
public ConsoleShareConsumerOptions(String[] args) throws IOException { public ConsoleShareConsumerOptions(String[] args) throws IOException {
super(args); super(args);
topicOpt = parser.accepts("topic", "The topic to consume from.") topicOpt = parser.accepts("topic", "The topic to consume from.")
.withRequiredArg() .withRequiredArg()
.describedAs("topic") .describedAs("topic")
.ofType(String.class); .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() .withRequiredArg()
.describedAs("consumer_prop") .describedAs("consumer_prop")
.ofType(String.class); .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() .withRequiredArg()
.describedAs("config file") .describedAs("config file")
.ofType(String.class); .ofType(String.class);
@ -71,7 +86,29 @@ public final class ConsoleShareConsumerOptions extends CommandDefaultOptions {
.describedAs("class") .describedAs("class")
.ofType(String.class) .ofType(String.class)
.defaultsTo(DefaultMessageFormatter.class.getName()); .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" + "The properties to initialize the message formatter. Default properties include: \n" +
" print.timestamp=true|false\n" + " print.timestamp=true|false\n" +
" print.key=true|false\n" + " print.key=true|false\n" +
@ -114,7 +151,7 @@ public final class ConsoleShareConsumerOptions extends CommandDefaultOptions {
.ofType(String.class); .ofType(String.class);
keyDeserializerOpt = parser.accepts("key-deserializer", "The name of the class to use for deserializing keys.") keyDeserializerOpt = parser.accepts("key-deserializer", "The name of the class to use for deserializing keys.")
.withRequiredArg() .withRequiredArg()
.describedAs("deserializer for key") .describedAs("deserializer for keys")
.ofType(String.class); .ofType(String.class);
valueDeserializerOpt = parser.accepts("value-deserializer", "The name of the class to use for deserializing values.") valueDeserializerOpt = parser.accepts("value-deserializer", "The name of the class to use for deserializing values.")
.withRequiredArg() .withRequiredArg()
@ -141,10 +178,26 @@ public final class ConsoleShareConsumerOptions extends CommandDefaultOptions {
CommandLineUtils.printUsageAndExit(parser, "At most one of --reject and --release may be specified."); CommandLineUtils.printUsageAndExit(parser, "At most one of --reject and --release may be specified.");
} }
Properties consumerPropsFromFile = options.has(consumerConfigOpt) if (options.has(consumerPropertyOpt) && options.has(commandPropertyOpt)) {
? Utils.loadProps(options.valueOf(consumerConfigOpt)) 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(); : new Properties();
Properties extraConsumerProps = CommandLineUtils.parseKeyValueArgs(options.valuesOf(consumerPropertyOpt)); Properties extraConsumerProps = CommandLineUtils.parseKeyValueArgs(options.valuesOf(commandPropertyOpt));
Set<String> groupIdsProvided = checkShareGroup(consumerPropsFromFile, extraConsumerProps); Set<String> groupIdsProvided = checkShareGroup(consumerPropsFromFile, extraConsumerProps);
consumerProps = buildConsumerProps(consumerPropsFromFile, extraConsumerProps, groupIdsProvided); consumerProps = buildConsumerProps(consumerPropsFromFile, extraConsumerProps, groupIdsProvided);
@ -203,6 +256,13 @@ public final class ConsoleShareConsumerOptions extends CommandDefaultOptions {
Class<?> messageFormatterClass = Class.forName(options.valueOf(messageFormatterOpt)); Class<?> messageFormatterClass = Class.forName(options.valueOf(messageFormatterOpt));
formatter = (MessageFormatter) messageFormatterClass.getDeclaredConstructor().newInstance(); 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(); Properties formatterArgs = formatterArgs();
Map<String, String> formatterConfigs = new HashMap<>(); Map<String, String> formatterConfigs = new HashMap<>();
for (final String name : formatterArgs.stringPropertyNames()) { for (final String name : formatterArgs.stringPropertyNames()) {

View File

@ -34,6 +34,7 @@ import java.io.InputStream;
import java.io.OutputStream; import java.io.OutputStream;
import java.nio.file.Files; import java.nio.file.Files;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -57,11 +58,16 @@ public class ConsoleProducerTest {
"--bootstrap-server", "localhost:1002", "--bootstrap-server", "localhost:1002",
"--topic", "t3", "--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", "--bootstrap-server", "localhost:1001",
"--topic", "t3", "--topic", "t3",
"--producer-property", "client.id=producer-1" "--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[]{ private static final String[] BATCH_SIZE_OVERRIDDEN_BY_MAX_PARTITION_MEMORY_BYTES_VALUE = new String[]{
"--bootstrap-server", "localhost:1002", "--bootstrap-server", "localhost:1002",
"--topic", "t3", "--topic", "t3",
@ -151,8 +157,8 @@ public class ConsoleProducerTest {
} }
@Test @Test
public void testClientIdOverride() throws IOException { public void testClientIdOverrideDeprecated() throws IOException {
ConsoleProducerOptions opts = new ConsoleProducerOptions(CLIENT_ID_OVERRIDE); ConsoleProducerOptions opts = new ConsoleProducerOptions(CLIENT_ID_OVERRIDE_DEPRECATED);
ProducerConfig producerConfig = new ProducerConfig(opts.producerProps()); ProducerConfig producerConfig = new ProducerConfig(opts.producerProps());
assertEquals("producer-1", producerConfig.getString(ProducerConfig.CLIENT_ID_CONFIG)); assertEquals("producer-1", producerConfig.getString(ProducerConfig.CLIENT_ID_CONFIG));
@ -222,6 +228,107 @@ public class ConsoleProducerTest {
assertEquals(1, reader.closeCount()); 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 { public static class TestRecordReader implements RecordReader {
private int configureCount = 0; private int configureCount = 0;
private int closeCount = 0; private int closeCount = 0;

View File

@ -172,7 +172,7 @@ public class ConsoleConsumerOptionsTest {
} }
@Test @Test
public void shouldParseValidConsumerConfigWithAutoOffsetResetLatest() throws IOException { public void shouldParseValidConsumerConfigWithAutoOffsetResetLatestDeprecated() throws IOException {
String[] args = new String[]{ String[] args = new String[]{
"--bootstrap-server", "localhost:9092", "--bootstrap-server", "localhost:9092",
"--topic", "test", "--topic", "test",
@ -189,7 +189,7 @@ public class ConsoleConsumerOptionsTest {
} }
@Test @Test
public void shouldParseValidConsumerConfigWithAutoOffsetResetEarliest() throws IOException { public void shouldParseValidConsumerConfigWithAutoOffsetResetEarliestDeprecated() throws IOException {
String[] args = new String[]{ String[] args = new String[]{
"--bootstrap-server", "localhost:9092", "--bootstrap-server", "localhost:9092",
"--topic", "test", "--topic", "test",
@ -206,7 +206,7 @@ public class ConsoleConsumerOptionsTest {
} }
@Test @Test
public void shouldParseValidConsumerConfigWithAutoOffsetResetAndMatchingFromBeginning() throws IOException { public void shouldParseValidConsumerConfigWithAutoOffsetResetAndMatchingFromBeginningDeprecated() throws IOException {
String[] args = new String[]{ String[] args = new String[]{
"--bootstrap-server", "localhost:9092", "--bootstrap-server", "localhost:9092",
"--topic", "test", "--topic", "test",
@ -240,7 +240,7 @@ public class ConsoleConsumerOptionsTest {
} }
@Test @Test
public void shouldExitOnInvalidConfigWithAutoOffsetResetAndConflictingFromBeginning() { public void shouldExitOnInvalidConfigWithAutoOffsetResetAndConflictingFromBeginningDeprecated() {
Exit.setExitProcedure((code, message) -> { Exit.setExitProcedure((code, message) -> {
throw new IllegalArgumentException(message); throw new IllegalArgumentException(message);
}); });
@ -259,7 +259,7 @@ public class ConsoleConsumerOptionsTest {
} }
@Test @Test
public void shouldParseConfigsFromFile() throws IOException { public void shouldParseConfigsFromFileDeprecated() throws IOException {
Map<String, String> configs = new HashMap<>(); Map<String, String> configs = new HashMap<>();
configs.put("request.timeout.ms", "1000"); configs.put("request.timeout.ms", "1000");
configs.put("group.id", "group1"); configs.put("group.id", "group1");
@ -276,10 +276,11 @@ public class ConsoleConsumerOptionsTest {
} }
@Test @Test
public void groupIdsProvidedInDifferentPlacesMustMatch() throws IOException { public void groupIdsProvidedInDifferentPlacesMustMatchDeprecated() throws IOException {
Exit.setExitProcedure((code, message) -> { Exit.setExitProcedure((code, message) -> {
throw new IllegalArgumentException(message); throw new IllegalArgumentException(message);
}); });
try {
// different in all three places // different in all three places
File propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file")); File propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file"));
@ -348,9 +349,10 @@ public class ConsoleConsumerOptionsTest {
config = new ConsoleConsumerOptions(args5); config = new ConsoleConsumerOptions(args5);
props = config.consumerProps(); props = config.consumerProps();
assertEquals("group-from-arguments", props.getProperty("group.id")); assertEquals("group-from-arguments", props.getProperty("group.id"));
} finally {
Exit.resetExitProcedure(); Exit.resetExitProcedure();
} }
}
@Test @Test
public void testCustomPropertyShouldBePassedToConfigureMethod() throws Exception { public void testCustomPropertyShouldBePassedToConfigureMethod() throws Exception {
@ -508,7 +510,7 @@ public class ConsoleConsumerOptionsTest {
} }
@Test @Test
public void testClientIdOverride() throws IOException { public void testClientIdOverrideDeprecated() throws IOException {
String[] args = new String[]{ String[] args = new String[]{
"--bootstrap-server", "localhost:9092", "--bootstrap-server", "localhost:9092",
"--topic", "test", "--topic", "test",
@ -618,4 +620,234 @@ public class ConsoleConsumerOptionsTest {
"--formatter", formatter, "--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));
}
} }

View File

@ -20,6 +20,7 @@ import org.apache.kafka.clients.consumer.AcknowledgeType;
import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.common.utils.Exit; import org.apache.kafka.common.utils.Exit;
import org.apache.kafka.common.utils.Utils; import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.test.MockDeserializer;
import org.apache.kafka.tools.ToolsTestUtils; import org.apache.kafka.tools.ToolsTestUtils;
import org.junit.jupiter.api.Test; 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.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse; 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.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
public class ConsoleShareConsumerOptionsTest { public class ConsoleShareConsumerOptionsTest {
@ -72,7 +75,7 @@ public class ConsoleShareConsumerOptionsTest {
} }
@Test @Test
public void shouldParseValidConsumerConfigWithSessionTimeout() throws IOException { public void shouldParseValidConsumerConfigWithSessionTimeoutDeprecated() throws IOException {
String[] args = new String[]{ String[] args = new String[]{
"--bootstrap-server", "localhost:9092", "--bootstrap-server", "localhost:9092",
"--topic", "test", "--topic", "test",
@ -88,7 +91,7 @@ public class ConsoleShareConsumerOptionsTest {
} }
@Test @Test
public void shouldParseConfigsFromFile() throws IOException { public void shouldParseConfigsFromFileDeprecated() throws IOException {
Map<String, String> configs = new HashMap<>(); Map<String, String> configs = new HashMap<>();
configs.put("request.timeout.ms", "1000"); configs.put("request.timeout.ms", "1000");
configs.put("group.id", "group1"); configs.put("group.id", "group1");
@ -109,11 +112,12 @@ public class ConsoleShareConsumerOptionsTest {
} }
@Test @Test
public void groupIdsProvidedInDifferentPlacesMustMatch() throws IOException { public void groupIdsProvidedInDifferentPlacesMustMatchDeprecated() throws IOException {
Exit.setExitProcedure((code, message) -> { Exit.setExitProcedure((code, message) -> {
throw new IllegalArgumentException(message); throw new IllegalArgumentException(message);
}); });
try {
// different in all three places // different in all three places
File propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file")); File propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file"));
final String[] args = new String[]{ final String[] args = new String[]{
@ -181,9 +185,10 @@ public class ConsoleShareConsumerOptionsTest {
config = new ConsoleShareConsumerOptions(args5); config = new ConsoleShareConsumerOptions(args5);
props = config.consumerProps(); props = config.consumerProps();
assertEquals("group-from-arguments", props.getProperty("group.id")); assertEquals("group-from-arguments", props.getProperty("group.id"));
} finally {
Exit.resetExitProcedure(); Exit.resetExitProcedure();
} }
}
@Test @Test
public void shouldExitIfNoTopicSpecified() { public void shouldExitIfNoTopicSpecified() {
@ -203,7 +208,7 @@ public class ConsoleShareConsumerOptionsTest {
} }
@Test @Test
public void testClientIdOverride() throws IOException { public void testClientIdOverrideDeprecated() throws IOException {
String[] args = new String[]{ String[] args = new String[]{
"--bootstrap-server", "localhost:9092", "--bootstrap-server", "localhost:9092",
"--topic", "test", "--topic", "test",
@ -216,6 +221,56 @@ public class ConsoleShareConsumerOptionsTest {
assertEquals("consumer-1", consumerProperties.getProperty(ConsumerConfig.CLIENT_ID_CONFIG)); 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 @Test
public void testDefaultClientId() throws IOException { public void testDefaultClientId() throws IOException {
String[] args = new String[]{ String[] args = new String[]{
@ -271,4 +326,182 @@ public class ConsoleShareConsumerOptionsTest {
Exit.resetExitProcedure(); 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));
}
} }