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,9 +180,11 @@ 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
* @throws org.apache.kafka.common.errors.WakeupException If raised from a nested call to {@link KafkaConsumer} * included in this list under the Classic/Cooperative or Consumer protocols. THe full assignment
* will be received under the Classic/Eager protocol.
* @throws org.apache.kafka.common.errors.WakeupException If raised from a nested call to {@link KafkaConsumer}
* @throws org.apache.kafka.common.errors.InterruptException If raised from a nested call to {@link KafkaConsumer} * @throws org.apache.kafka.common.errors.InterruptException If raised from a nested call to {@link KafkaConsumer}
*/ */
void onPartitionsAssigned(Collection<TopicPartition> partitions); void onPartitionsAssigned(Collection<TopicPartition> partitions);
@ -187,10 +209,9 @@ 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}
*/ */
default void onPartitionsLost(Collection<TopicPartition> partitions) { default void onPartitionsLost(Collection<TopicPartition> partitions) {

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,26 +1511,39 @@ 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();
Set<TopicPartition> assignedPartitions = groupAssignmentSnapshot.get(); Exception error = null;
if (assignedPartitions.isEmpty()) if (streamsRebalanceListenerInvoker != null && streamsRebalanceListenerInvoker.isPresent()) {
// Nothing to revoke.
return;
SortedSet<TopicPartition> droppedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); if (memberEpoch > 0) {
droppedPartitions.addAll(assignedPartitions); error = streamsRebalanceListenerInvoker.get().invokeAllTasksRevoked();
} else {
error = streamsRebalanceListenerInvoker.get().invokeAllTasksLost();
}
final Exception error; } else if (rebalanceListenerInvoker != null) {
if (memberEpoch > 0) Set<TopicPartition> assignedPartitions = groupAssignmentSnapshot.get();
error = rebalanceListenerInvoker.invokePartitionsRevoked(droppedPartitions);
else if (assignedPartitions.isEmpty())
error = rebalanceListenerInvoker.invokePartitionsLost(droppedPartitions); // Nothing to revoke.
return;
SortedSet<TopicPartition> droppedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR);
droppedPartitions.addAll(assignedPartitions);
if (memberEpoch > 0) {
error = rebalanceListenerInvoker.invokePartitionsRevoked(droppedPartitions);
} else {
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,22 +71,25 @@ 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);
public CoordinatorLoaderImpl( public CoordinatorLoaderImpl(
Time time, Time time,
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();
} }
@ -89,7 +106,7 @@ public class CoordinatorLoaderImpl<T> implements CoordinatorLoader<T> {
long startTimeMs = time.milliseconds(); long startTimeMs = time.milliseconds();
try { try {
ScheduledFuture<?> result = scheduler.scheduleOnce(String.format("Load coordinator from %s", tp), ScheduledFuture<?> result = scheduler.scheduleOnce(String.format("Load coordinator from %s", tp),
() -> doLoad(tp, coordinator, future, startTimeMs)); () -> doLoad(tp, coordinator, future, startTimeMs));
if (result.isCancelled()) { if (result.isCancelled()) {
future.completeExceptionally(new RuntimeException("Coordinator loader is closed.")); future.completeExceptionally(new RuntimeException("Coordinator loader is closed."));
} }
@ -100,17 +117,17 @@ public class CoordinatorLoaderImpl<T> implements CoordinatorLoader<T> {
} }
private void doLoad( private void doLoad(
TopicPartition tp, TopicPartition tp,
CoordinatorPlayback<T> coordinator, CoordinatorPlayback<T> coordinator,
CompletableFuture<LoadSummary> future, CompletableFuture<LoadSummary> future,
long startTimeMs long startTimeMs
) { ) {
long schedulerQueueTimeMs = time.milliseconds() - startTimeMs; long schedulerQueueTimeMs = time.milliseconds() - startTimeMs;
try { try {
Optional<UnifiedLog> logOpt = partitionLogSupplier.apply(tp); Optional<UnifiedLog> logOpt = partitionLogSupplier.apply(tp);
if (logOpt.isEmpty()) { if (logOpt.isEmpty()) {
future.completeExceptionally(new NotLeaderOrFollowerException( future.completeExceptionally(new NotLeaderOrFollowerException(
"Could not load records from " + tp + " because the log does not exist.")); "Could not load records from " + tp + " because the log does not exist."));
return; return;
} }
@ -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 {
@ -186,7 +202,7 @@ public class CoordinatorLoaderImpl<T> implements CoordinatorLoader<T> {
if (buffer.capacity() < bytesNeeded) { if (buffer.capacity() < bytesNeeded) {
if (loadBufferSize < bytesNeeded) { if (loadBufferSize < bytesNeeded) {
LOG.warn("Loaded metadata from {} with buffer larger ({} bytes) than" + LOG.warn("Loaded metadata from {} with buffer larger ({} bytes) than" +
" configured buffer size ({} bytes).", tp, bytesNeeded, loadBufferSize); " configured buffer size ({} bytes).", tp, bytesNeeded, loadBufferSize);
} }
buffer = ByteBuffer.allocate(bytesNeeded); buffer = ByteBuffer.allocate(bytesNeeded);
@ -202,15 +218,14 @@ public class CoordinatorLoaderImpl<T> implements CoordinatorLoader<T> {
} }
private ReplayResult processMemoryRecords( private ReplayResult processMemoryRecords(
TopicPartition tp, TopicPartition tp,
UnifiedLog log, UnifiedLog log,
MemoryRecords memoryRecords, MemoryRecords memoryRecords,
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) {
@ -220,8 +235,8 @@ public class CoordinatorLoaderImpl<T> implements CoordinatorLoader<T> {
if (controlRecord == ControlRecordType.COMMIT) { if (controlRecord == ControlRecordType.COMMIT) {
if (LOG.isTraceEnabled()) { if (LOG.isTraceEnabled()) {
LOG.trace("Replaying end transaction marker from {} at offset {} to commit" + LOG.trace("Replaying end transaction marker from {} at offset {} to commit" +
" transaction with producer id {} and producer epoch {}.", " transaction with producer id {} and producer epoch {}.",
tp, record.offset(), batch.producerId(), batch.producerEpoch()); tp, record.offset(), batch.producerId(), batch.producerEpoch());
} }
coordinator.replayEndTransactionMarker( coordinator.replayEndTransactionMarker(
batch.producerId(), batch.producerId(),
@ -231,8 +246,8 @@ public class CoordinatorLoaderImpl<T> implements CoordinatorLoader<T> {
} else if (controlRecord == ControlRecordType.ABORT) { } else if (controlRecord == ControlRecordType.ABORT) {
if (LOG.isTraceEnabled()) { if (LOG.isTraceEnabled()) {
LOG.trace("Replaying end transaction marker from {} at offset {} to abort" + LOG.trace("Replaying end transaction marker from {} at offset {} to abort" +
" transaction with producer id {} and producer epoch {}.", " transaction with producer id {} and producer epoch {}.",
tp, record.offset(), batch.producerId(), batch.producerEpoch()); tp, record.offset(), batch.producerId(), batch.producerEpoch());
} }
coordinator.replayEndTransactionMarker( coordinator.replayEndTransactionMarker(
batch.producerId(), batch.producerId(),
@ -250,7 +265,7 @@ public class CoordinatorLoaderImpl<T> implements CoordinatorLoader<T> {
coordinatorRecordOpt = Optional.ofNullable(deserializer.deserialize(record.key(), record.value())); coordinatorRecordOpt = Optional.ofNullable(deserializer.deserialize(record.key(), record.value()));
} catch (Deserializer.UnknownRecordTypeException ex) { } catch (Deserializer.UnknownRecordTypeException ex) {
LOG.warn("Unknown record type {} while loading offsets and group metadata from {}." + LOG.warn("Unknown record type {} while loading offsets and group metadata from {}." +
" Ignoring it. It could be a left over from an aborted upgrade.", ex.unknownType(), tp); " Ignoring it. It could be a left over from an aborted upgrade.", ex.unknownType(), tp);
} catch (RuntimeException ex) { } catch (RuntimeException ex) {
String msg = String.format("Deserializing record %s from %s failed.", record, tp); String msg = String.format("Deserializing record %s from %s failed.", record, tp);
LOG.error(msg, ex); LOG.error(msg, ex);
@ -261,18 +276,18 @@ public class CoordinatorLoaderImpl<T> implements CoordinatorLoader<T> {
try { try {
if (LOG.isTraceEnabled()) { if (LOG.isTraceEnabled()) {
LOG.trace("Replaying record {} from {} at offset {} with producer id {}" + LOG.trace("Replaying record {} from {} at offset {} with producer id {}" +
" and producer epoch {}.", coordinatorRecord, tp, record.offset(), batch.producerId(), batch.producerEpoch()); " and producer epoch {}.", coordinatorRecord, tp, record.offset(), batch.producerId(), batch.producerEpoch());
} }
coordinator.replay( coordinator.replay(
record.offset(), record.offset(),
batch.producerId(), batch.producerId(),
batch.producerEpoch(), batch.producerEpoch(),
coordinatorRecord coordinatorRecord
); );
} catch (RuntimeException ex) { } catch (RuntimeException ex) {
String msg = String.format("Replaying record %s from %s at offset %d with producer id %d and" + String msg = String.format("Replaying record %s from %s at offset %d with producer id %d and" +
" producer epoch %d failed.", coordinatorRecord, tp, record.offset(), " producer epoch %d failed.", coordinatorRecord, tp, record.offset(),
batch.producerId(), batch.producerEpoch()); batch.producerId(), batch.producerEpoch());
LOG.error(msg, ex); LOG.error(msg, ex);
throw new RuntimeException(msg, ex); throw new RuntimeException(msg, ex);
} }
@ -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

@ -72,8 +72,8 @@ class CoordinatorLoaderImplTest {
@Override @Override
public Map.Entry<String, String> deserialize(ByteBuffer key, ByteBuffer value) throws RuntimeException { public Map.Entry<String, String> deserialize(ByteBuffer key, ByteBuffer value) throws RuntimeException {
return Map.entry( return Map.entry(
StandardCharsets.UTF_8.decode(key).toString(), StandardCharsets.UTF_8.decode(key).toString(),
StandardCharsets.UTF_8.decode(value).toString() StandardCharsets.UTF_8.decode(value).toString()
); );
} }
} }
@ -87,11 +87,12 @@ class CoordinatorLoaderImplTest {
CoordinatorPlayback<Map.Entry<String, String>> coordinator = mock(CoordinatorPlayback.class); CoordinatorPlayback<Map.Entry<String, String>> coordinator = mock(CoordinatorPlayback.class);
try (CoordinatorLoaderImpl<Map.Entry<String, String>> loader = new CoordinatorLoaderImpl<>( try (CoordinatorLoaderImpl<Map.Entry<String, String>> loader = new CoordinatorLoaderImpl<>(
Time.SYSTEM, Time.SYSTEM,
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));
} }
@ -106,11 +107,12 @@ class CoordinatorLoaderImplTest {
CoordinatorPlayback<Map.Entry<String, String>> coordinator = mock(CoordinatorPlayback.class); CoordinatorPlayback<Map.Entry<String, String>> coordinator = mock(CoordinatorPlayback.class);
try (CoordinatorLoaderImpl<Map.Entry<String, String>> loader = new CoordinatorLoaderImpl<>( try (CoordinatorLoaderImpl<Map.Entry<String, String>> loader = new CoordinatorLoaderImpl<>(
Time.SYSTEM, Time.SYSTEM,
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));
@ -127,59 +129,60 @@ class CoordinatorLoaderImplTest {
CoordinatorPlayback<Map.Entry<String, String>> coordinator = mock(CoordinatorPlayback.class); CoordinatorPlayback<Map.Entry<String, String>> coordinator = mock(CoordinatorPlayback.class);
try (CoordinatorLoaderImpl<Map.Entry<String, String>> loader = new CoordinatorLoaderImpl<>( try (CoordinatorLoaderImpl<Map.Entry<String, String>> loader = new CoordinatorLoaderImpl<>(
Time.SYSTEM, Time.SYSTEM,
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);
FetchDataInfo readResult1 = logReadResult(0, Arrays.asList( FetchDataInfo readResult1 = logReadResult(0, Arrays.asList(
new SimpleRecord("k1".getBytes(), "v1".getBytes()), new SimpleRecord("k1".getBytes(), "v1".getBytes()),
new SimpleRecord("k2".getBytes(), "v2".getBytes()) new SimpleRecord("k2".getBytes(), "v2".getBytes())
)); ));
when(log.read(0L, 1000, FetchIsolation.LOG_END, true)) when(log.read(0L, 1000, FetchIsolation.LOG_END, true))
.thenReturn(readResult1); .thenReturn(readResult1);
FetchDataInfo readResult2 = logReadResult(2, Arrays.asList( FetchDataInfo readResult2 = logReadResult(2, Arrays.asList(
new SimpleRecord("k3".getBytes(), "v3".getBytes()), new SimpleRecord("k3".getBytes(), "v3".getBytes()),
new SimpleRecord("k4".getBytes(), "v4".getBytes()), new SimpleRecord("k4".getBytes(), "v4".getBytes()),
new SimpleRecord("k5".getBytes(), "v5".getBytes()) new SimpleRecord("k5".getBytes(), "v5".getBytes())
)); ));
when(log.read(2L, 1000, FetchIsolation.LOG_END, true)) when(log.read(2L, 1000, FetchIsolation.LOG_END, true))
.thenReturn(readResult2); .thenReturn(readResult2);
FetchDataInfo readResult3 = logReadResult(5, 100L, (short) 5, Arrays.asList( FetchDataInfo readResult3 = logReadResult(5, 100L, (short) 5, Arrays.asList(
new SimpleRecord("k6".getBytes(), "v6".getBytes()), new SimpleRecord("k6".getBytes(), "v6".getBytes()),
new SimpleRecord("k7".getBytes(), "v7".getBytes()) new SimpleRecord("k7".getBytes(), "v7".getBytes())
)); ));
when(log.read(5L, 1000, FetchIsolation.LOG_END, true)) when(log.read(5L, 1000, FetchIsolation.LOG_END, true))
.thenReturn(readResult3); .thenReturn(readResult3);
FetchDataInfo readResult4 = logReadResult( FetchDataInfo readResult4 = logReadResult(
7, 7,
100L, 100L,
(short) 5, (short) 5,
ControlRecordType.COMMIT ControlRecordType.COMMIT
); );
when(log.read(7L, 1000, FetchIsolation.LOG_END, true)) when(log.read(7L, 1000, FetchIsolation.LOG_END, true))
.thenReturn(readResult4); .thenReturn(readResult4);
FetchDataInfo readResult5 = logReadResult( FetchDataInfo readResult5 = logReadResult(
8, 8,
500L, 500L,
(short) 10, (short) 10,
ControlRecordType.ABORT ControlRecordType.ABORT
); );
when(log.read(8L, 1000, FetchIsolation.LOG_END, true)) when(log.read(8L, 1000, FetchIsolation.LOG_END, true))
.thenReturn(readResult5); .thenReturn(readResult5);
CoordinatorLoader.LoadSummary summary = loader.load(tp, coordinator).get(10, TimeUnit.SECONDS); CoordinatorLoader.LoadSummary summary = loader.load(tp, coordinator).get(10, TimeUnit.SECONDS);
assertNotNull(summary); assertNotNull(summary);
@ -213,25 +216,26 @@ class CoordinatorLoaderImplTest {
CoordinatorPlayback<Map.Entry<String, String>> coordinator = mock(CoordinatorPlayback.class); CoordinatorPlayback<Map.Entry<String, String>> coordinator = mock(CoordinatorPlayback.class);
try (CoordinatorLoaderImpl<Map.Entry<String, String>> loader = new CoordinatorLoaderImpl<>( try (CoordinatorLoaderImpl<Map.Entry<String, String>> loader = new CoordinatorLoaderImpl<>(
Time.SYSTEM, Time.SYSTEM,
partitionLogSupplier, partitionLogSupplier,
partitionLogEndOffsetSupplier, partitionLogEndOffsetSupplier,
serde, serde,
1000 1000,
CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS
)) { )) {
when(log.logStartOffset()).thenReturn(0L); when(log.logStartOffset()).thenReturn(0L);
FetchDataInfo readResult = logReadResult(0, Arrays.asList( FetchDataInfo readResult = logReadResult(0, Arrays.asList(
new SimpleRecord("k1".getBytes(), "v1".getBytes()), new SimpleRecord("k1".getBytes(), "v1".getBytes()),
new SimpleRecord("k2".getBytes(), "v2".getBytes()) new SimpleRecord("k2".getBytes(), "v2".getBytes())
)); ));
CountDownLatch latch = new CountDownLatch(1); CountDownLatch latch = new CountDownLatch(1);
when(log.read( when(log.read(
anyLong(), anyLong(),
eq(1000), eq(1000),
eq(FetchIsolation.LOG_END), eq(FetchIsolation.LOG_END),
eq(true) eq(true)
)).thenAnswer((InvocationOnMock invocation) -> { )).thenAnswer((InvocationOnMock invocation) -> {
latch.countDown(); latch.countDown();
return readResult; return readResult;
@ -258,25 +262,26 @@ class CoordinatorLoaderImplTest {
CoordinatorPlayback<Map.Entry<String, String>> coordinator = mock(CoordinatorPlayback.class); CoordinatorPlayback<Map.Entry<String, String>> coordinator = mock(CoordinatorPlayback.class);
try (CoordinatorLoaderImpl<Map.Entry<String, String>> loader = new CoordinatorLoaderImpl<>( try (CoordinatorLoaderImpl<Map.Entry<String, String>> loader = new CoordinatorLoaderImpl<>(
Time.SYSTEM, Time.SYSTEM,
partitionLogSupplier, partitionLogSupplier,
partitionLogEndOffsetSupplier, partitionLogEndOffsetSupplier,
serde, serde,
1000 1000,
CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS
)) { )) {
when(log.logStartOffset()).thenReturn(0L); when(log.logStartOffset()).thenReturn(0L);
FetchDataInfo readResult = logReadResult(0, Arrays.asList( FetchDataInfo readResult = logReadResult(0, Arrays.asList(
new SimpleRecord("k1".getBytes(), "v1".getBytes()), new SimpleRecord("k1".getBytes(), "v1".getBytes()),
new SimpleRecord("k2".getBytes(), "v2".getBytes()) new SimpleRecord("k2".getBytes(), "v2".getBytes())
)); ));
when(log.read(0L, 1000, FetchIsolation.LOG_END, true)) when(log.read(0L, 1000, FetchIsolation.LOG_END, true))
.thenReturn(readResult); .thenReturn(readResult);
when(serde.deserialize(any(ByteBuffer.class), any(ByteBuffer.class))) when(serde.deserialize(any(ByteBuffer.class), any(ByteBuffer.class)))
.thenThrow(new Deserializer.UnknownRecordTypeException((short) 1)) .thenThrow(new Deserializer.UnknownRecordTypeException((short) 1))
.thenReturn(Map.entry("k2", "v2")); .thenReturn(Map.entry("k2", "v2"));
loader.load(tp, coordinator).get(10, TimeUnit.SECONDS); loader.load(tp, coordinator).get(10, TimeUnit.SECONDS);
@ -294,24 +299,25 @@ class CoordinatorLoaderImplTest {
CoordinatorPlayback<Map.Entry<String, String>> coordinator = mock(CoordinatorPlayback.class); CoordinatorPlayback<Map.Entry<String, String>> coordinator = mock(CoordinatorPlayback.class);
try (CoordinatorLoaderImpl<Map.Entry<String, String>> loader = new CoordinatorLoaderImpl<>( try (CoordinatorLoaderImpl<Map.Entry<String, String>> loader = new CoordinatorLoaderImpl<>(
Time.SYSTEM, Time.SYSTEM,
partitionLogSupplier, partitionLogSupplier,
partitionLogEndOffsetSupplier, partitionLogEndOffsetSupplier,
serde, serde,
1000 1000,
CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS
)) { )) {
when(log.logStartOffset()).thenReturn(0L); when(log.logStartOffset()).thenReturn(0L);
FetchDataInfo readResult = logReadResult(0, Arrays.asList( FetchDataInfo readResult = logReadResult(0, Arrays.asList(
new SimpleRecord("k1".getBytes(), "v1".getBytes()), new SimpleRecord("k1".getBytes(), "v1".getBytes()),
new SimpleRecord("k2".getBytes(), "v2".getBytes()) new SimpleRecord("k2".getBytes(), "v2".getBytes())
)); ));
when(log.read(0L, 1000, FetchIsolation.LOG_END, true)) when(log.read(0L, 1000, FetchIsolation.LOG_END, true))
.thenReturn(readResult); .thenReturn(readResult);
when(serde.deserialize(any(ByteBuffer.class), any(ByteBuffer.class))) when(serde.deserialize(any(ByteBuffer.class), any(ByteBuffer.class)))
.thenThrow(new RuntimeException("Error!")); .thenThrow(new RuntimeException("Error!"));
RuntimeException ex = assertFutureThrows(RuntimeException.class, loader.load(tp, coordinator)); RuntimeException ex = assertFutureThrows(RuntimeException.class, loader.load(tp, coordinator));
@ -333,18 +339,19 @@ class CoordinatorLoaderImplTest {
CoordinatorPlayback<Map.Entry<String, String>> coordinator = mock(CoordinatorPlayback.class); CoordinatorPlayback<Map.Entry<String, String>> coordinator = mock(CoordinatorPlayback.class);
try (CoordinatorLoaderImpl<Map.Entry<String, String>> loader = new CoordinatorLoaderImpl<>( try (CoordinatorLoaderImpl<Map.Entry<String, String>> loader = new CoordinatorLoaderImpl<>(
Time.SYSTEM, Time.SYSTEM,
partitionLogSupplier, partitionLogSupplier,
partitionLogEndOffsetSupplier, partitionLogEndOffsetSupplier,
serde, serde,
1000 1000,
CoordinatorLoaderImpl.DEFAULT_COMMIT_INTERVAL_OFFSETS
)) { )) {
when(log.logStartOffset()).thenReturn(0L); when(log.logStartOffset()).thenReturn(0L);
FetchDataInfo readResult = logReadResult(0, List.of()); FetchDataInfo readResult = logReadResult(0, List.of());
when(log.read(0L, 1000, FetchIsolation.LOG_END, true)) when(log.read(0L, 1000, FetchIsolation.LOG_END, true))
.thenReturn(readResult); .thenReturn(readResult);
assertNotNull(loader.load(tp, coordinator).get(10, TimeUnit.SECONDS)); assertNotNull(loader.load(tp, coordinator).get(10, TimeUnit.SECONDS));
} }
@ -361,34 +368,35 @@ class CoordinatorLoaderImplTest {
MockTime time = new MockTime(); MockTime time = new MockTime();
try (CoordinatorLoaderImpl<Map.Entry<String, String>> loader = new CoordinatorLoaderImpl<>( try (CoordinatorLoaderImpl<Map.Entry<String, String>> loader = new CoordinatorLoaderImpl<>(
time, time,
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);
FetchDataInfo readResult1 = logReadResult(0, Arrays.asList( FetchDataInfo readResult1 = logReadResult(0, Arrays.asList(
new SimpleRecord("k1".getBytes(), "v1".getBytes()), new SimpleRecord("k1".getBytes(), "v1".getBytes()),
new SimpleRecord("k2".getBytes(), "v2".getBytes()) new SimpleRecord("k2".getBytes(), "v2".getBytes())
)); ));
when(log.read(0L, 1000, FetchIsolation.LOG_END, true)) when(log.read(0L, 1000, FetchIsolation.LOG_END, true))
.thenAnswer((InvocationOnMock invocation) -> { .thenAnswer((InvocationOnMock invocation) -> {
time.sleep(1000); time.sleep(1000);
return readResult1; return readResult1;
}); });
FetchDataInfo readResult2 = logReadResult(2, Arrays.asList( FetchDataInfo readResult2 = logReadResult(2, Arrays.asList(
new SimpleRecord("k3".getBytes(), "v3".getBytes()), new SimpleRecord("k3".getBytes(), "v3".getBytes()),
new SimpleRecord("k4".getBytes(), "v4".getBytes()), new SimpleRecord("k4".getBytes(), "v4".getBytes()),
new SimpleRecord("k5".getBytes(), "v5".getBytes()) new SimpleRecord("k5".getBytes(), "v5".getBytes())
)); ));
when(log.read(2L, 1000, FetchIsolation.LOG_END, true)) when(log.read(2L, 1000, FetchIsolation.LOG_END, true))
.thenReturn(readResult2); .thenReturn(readResult2);
CoordinatorLoader.LoadSummary summary = loader.load(tp, coordinator).get(10, TimeUnit.SECONDS); CoordinatorLoader.LoadSummary summary = loader.load(tp, coordinator).get(10, TimeUnit.SECONDS);
assertEquals(startTimeMs, summary.startTimeMs()); assertEquals(startTimeMs, summary.startTimeMs());
@ -408,39 +416,40 @@ class CoordinatorLoaderImplTest {
CoordinatorPlayback<Map.Entry<String, String>> coordinator = mock(CoordinatorPlayback.class); CoordinatorPlayback<Map.Entry<String, String>> coordinator = mock(CoordinatorPlayback.class);
try (CoordinatorLoaderImpl<Map.Entry<String, String>> loader = new CoordinatorLoaderImpl<>( try (CoordinatorLoaderImpl<Map.Entry<String, String>> loader = new CoordinatorLoaderImpl<>(
Time.SYSTEM, Time.SYSTEM,
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);
FetchDataInfo readResult1 = logReadResult(0, Arrays.asList( FetchDataInfo readResult1 = logReadResult(0, Arrays.asList(
new SimpleRecord("k1".getBytes(), "v1".getBytes()), new SimpleRecord("k1".getBytes(), "v1".getBytes()),
new SimpleRecord("k2".getBytes(), "v2".getBytes()) new SimpleRecord("k2".getBytes(), "v2".getBytes())
)); ));
when(log.read(0L, 1000, FetchIsolation.LOG_END, true)) when(log.read(0L, 1000, FetchIsolation.LOG_END, true))
.thenReturn(readResult1); .thenReturn(readResult1);
FetchDataInfo readResult2 = logReadResult(2, Arrays.asList( FetchDataInfo readResult2 = logReadResult(2, Arrays.asList(
new SimpleRecord("k3".getBytes(), "v3".getBytes()), new SimpleRecord("k3".getBytes(), "v3".getBytes()),
new SimpleRecord("k4".getBytes(), "v4".getBytes()), new SimpleRecord("k4".getBytes(), "v4".getBytes()),
new SimpleRecord("k5".getBytes(), "v5".getBytes()) new SimpleRecord("k5".getBytes(), "v5".getBytes())
)); ));
when(log.read(2L, 1000, FetchIsolation.LOG_END, true)) when(log.read(2L, 1000, FetchIsolation.LOG_END, true))
.thenReturn(readResult2); .thenReturn(readResult2);
FetchDataInfo readResult3 = logReadResult(5, Arrays.asList( FetchDataInfo readResult3 = logReadResult(5, Arrays.asList(
new SimpleRecord("k6".getBytes(), "v6".getBytes()), new SimpleRecord("k6".getBytes(), "v6".getBytes()),
new SimpleRecord("k7".getBytes(), "v7".getBytes()) new SimpleRecord("k7".getBytes(), "v7".getBytes())
)); ));
when(log.read(5L, 1000, FetchIsolation.LOG_END, true)) when(log.read(5L, 1000, FetchIsolation.LOG_END, true))
.thenReturn(readResult3); .thenReturn(readResult3);
assertNotNull(loader.load(tp, coordinator).get(10, TimeUnit.SECONDS)); assertNotNull(loader.load(tp, coordinator).get(10, TimeUnit.SECONDS));
@ -471,11 +480,12 @@ class CoordinatorLoaderImplTest {
CoordinatorPlayback<Map.Entry<String, String>> coordinator = mock(CoordinatorPlayback.class); CoordinatorPlayback<Map.Entry<String, String>> coordinator = mock(CoordinatorPlayback.class);
try (CoordinatorLoaderImpl<Map.Entry<String, String>> loader = new CoordinatorLoaderImpl<>( try (CoordinatorLoaderImpl<Map.Entry<String, String>> loader = new CoordinatorLoaderImpl<>(
Time.SYSTEM, Time.SYSTEM,
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);
@ -497,39 +507,40 @@ class CoordinatorLoaderImplTest {
CoordinatorPlayback<Map.Entry<String, String>> coordinator = mock(CoordinatorPlayback.class); CoordinatorPlayback<Map.Entry<String, String>> coordinator = mock(CoordinatorPlayback.class);
try (CoordinatorLoaderImpl<Map.Entry<String, String>> loader = new CoordinatorLoaderImpl<>( try (CoordinatorLoaderImpl<Map.Entry<String, String>> loader = new CoordinatorLoaderImpl<>(
Time.SYSTEM, Time.SYSTEM,
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);
FetchDataInfo readResult1 = logReadResult(0, Arrays.asList( FetchDataInfo readResult1 = logReadResult(0, Arrays.asList(
new SimpleRecord("k1".getBytes(), "v1".getBytes()), new SimpleRecord("k1".getBytes(), "v1".getBytes()),
new SimpleRecord("k2".getBytes(), "v2".getBytes()) new SimpleRecord("k2".getBytes(), "v2".getBytes())
)); ));
when(log.read(0L, 1000, FetchIsolation.LOG_END, true)) when(log.read(0L, 1000, FetchIsolation.LOG_END, true))
.thenReturn(readResult1); .thenReturn(readResult1);
FetchDataInfo readResult2 = logReadResult(2, Arrays.asList( FetchDataInfo readResult2 = logReadResult(2, Arrays.asList(
new SimpleRecord("k3".getBytes(), "v3".getBytes()), new SimpleRecord("k3".getBytes(), "v3".getBytes()),
new SimpleRecord("k4".getBytes(), "v4".getBytes()), new SimpleRecord("k4".getBytes(), "v4".getBytes()),
new SimpleRecord("k5".getBytes(), "v5".getBytes()) new SimpleRecord("k5".getBytes(), "v5".getBytes())
)); ));
when(log.read(2L, 1000, FetchIsolation.LOG_END, true)) when(log.read(2L, 1000, FetchIsolation.LOG_END, true))
.thenReturn(readResult2); .thenReturn(readResult2);
FetchDataInfo readResult3 = logReadResult(5, Arrays.asList( FetchDataInfo readResult3 = logReadResult(5, Arrays.asList(
new SimpleRecord("k6".getBytes(), "v6".getBytes()), new SimpleRecord("k6".getBytes(), "v6".getBytes()),
new SimpleRecord("k7".getBytes(), "v7".getBytes()) new SimpleRecord("k7".getBytes(), "v7".getBytes())
)); ));
when(log.read(5L, 1000, FetchIsolation.LOG_END, true)) when(log.read(5L, 1000, FetchIsolation.LOG_END, true))
.thenReturn(readResult3); .thenReturn(readResult3);
assertNotNull(loader.load(tp, coordinator).get(10, TimeUnit.SECONDS)); assertNotNull(loader.load(tp, coordinator).get(10, TimeUnit.SECONDS));
@ -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);
@ -561,32 +645,33 @@ class CoordinatorLoaderImplTest {
CoordinatorPlayback<Map.Entry<String, String>> coordinator = mock(CoordinatorPlayback.class); CoordinatorPlayback<Map.Entry<String, String>> coordinator = mock(CoordinatorPlayback.class);
try (CoordinatorLoaderImpl<Map.Entry<String, String>> loader = new CoordinatorLoaderImpl<>( try (CoordinatorLoaderImpl<Map.Entry<String, String>> loader = new CoordinatorLoaderImpl<>(
Time.SYSTEM, Time.SYSTEM,
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);
when(partitionLogEndOffsetSupplier.apply(tp)).thenReturn(Optional.of(5L)).thenReturn(Optional.of(-1L)); when(partitionLogEndOffsetSupplier.apply(tp)).thenReturn(Optional.of(5L)).thenReturn(Optional.of(-1L));
FetchDataInfo readResult1 = logReadResult(0, Arrays.asList( FetchDataInfo readResult1 = logReadResult(0, Arrays.asList(
new SimpleRecord("k1".getBytes(), "v1".getBytes()), new SimpleRecord("k1".getBytes(), "v1".getBytes()),
new SimpleRecord("k2".getBytes(), "v2".getBytes()) new SimpleRecord("k2".getBytes(), "v2".getBytes())
)); ));
when(log.read(0L, 1000, FetchIsolation.LOG_END, true)) when(log.read(0L, 1000, FetchIsolation.LOG_END, true))
.thenReturn(readResult1); .thenReturn(readResult1);
FetchDataInfo readResult2 = logReadResult(2, Arrays.asList( FetchDataInfo readResult2 = logReadResult(2, Arrays.asList(
new SimpleRecord("k3".getBytes(), "v3".getBytes()), new SimpleRecord("k3".getBytes(), "v3".getBytes()),
new SimpleRecord("k4".getBytes(), "v4".getBytes()), new SimpleRecord("k4".getBytes(), "v4".getBytes()),
new SimpleRecord("k5".getBytes(), "v5".getBytes()) new SimpleRecord("k5".getBytes(), "v5".getBytes())
)); ));
when(log.read(2L, 1000, FetchIsolation.LOG_END, true)) when(log.read(2L, 1000, FetchIsolation.LOG_END, true))
.thenReturn(readResult2); .thenReturn(readResult2);
assertFutureThrows(NotLeaderOrFollowerException.class, loader.load(tp, coordinator)); assertFutureThrows(NotLeaderOrFollowerException.class, loader.load(tp, coordinator));
} }
@ -597,28 +682,28 @@ class CoordinatorLoaderImplTest {
} }
private FetchDataInfo logReadResult( private FetchDataInfo logReadResult(
long startOffset, long startOffset,
long producerId, long producerId,
short producerEpoch, short producerEpoch,
List<SimpleRecord> records List<SimpleRecord> records
) throws IOException { ) throws IOException {
FileRecords fileRecords = mock(FileRecords.class); FileRecords fileRecords = mock(FileRecords.class);
MemoryRecords memoryRecords; MemoryRecords memoryRecords;
if (producerId == RecordBatch.NO_PRODUCER_ID) { if (producerId == RecordBatch.NO_PRODUCER_ID) {
memoryRecords = MemoryRecords.withRecords( memoryRecords = MemoryRecords.withRecords(
startOffset, startOffset,
Compression.NONE, Compression.NONE,
records.toArray(new SimpleRecord[0]) records.toArray(new SimpleRecord[0])
); );
} else { } else {
memoryRecords = MemoryRecords.withTransactionalRecords( memoryRecords = MemoryRecords.withTransactionalRecords(
startOffset, startOffset,
Compression.NONE, Compression.NONE,
producerId, producerId,
producerEpoch, producerEpoch,
0, 0,
RecordBatch.NO_PARTITION_LEADER_EPOCH, RecordBatch.NO_PARTITION_LEADER_EPOCH,
records.toArray(new SimpleRecord[0]) records.toArray(new SimpleRecord[0])
); );
} }
@ -635,19 +720,19 @@ class CoordinatorLoaderImplTest {
} }
private FetchDataInfo logReadResult( private FetchDataInfo logReadResult(
long startOffset, long startOffset,
long producerId, long producerId,
short producerEpoch, short producerEpoch,
ControlRecordType controlRecordType ControlRecordType controlRecordType
) throws IOException { ) throws IOException {
FileRecords fileRecords = mock(FileRecords.class); FileRecords fileRecords = mock(FileRecords.class);
MemoryRecords memoryRecords = MemoryRecords.withEndTransactionMarker( MemoryRecords memoryRecords = MemoryRecords.withEndTransactionMarker(
startOffset, startOffset,
0L, 0L,
RecordBatch.NO_PARTITION_LEADER_EPOCH, RecordBatch.NO_PARTITION_LEADER_EPOCH,
producerId, producerId,
producerEpoch, producerEpoch,
new EndTransactionMarker(controlRecordType, 0) new EndTransactionMarker(controlRecordType, 0)
); );
when(fileRecords.sizeInBytes()).thenReturn(memoryRecords.sizeInBytes()); when(fileRecords.sizeInBytes()).thenReturn(memoryRecords.sizeInBytes());

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,17 +127,21 @@ class RaftManagerTest {
val logDir = TestUtils.tempDir() val logDir = TestUtils.tempDir()
val nodeId = 1 val nodeId = 1
val raftManager = createRaftManager( try {
new TopicPartition("__raft_id_test", 0), val raftManager = createRaftManager(
createConfig( new TopicPartition("__raft_id_test", 0),
processRolesSet, createConfig(
nodeId, processRolesSet,
Seq(logDir.toPath), nodeId,
None Seq(logDir.toPath),
None
)
) )
) assertEquals(nodeId, raftManager.client.nodeId.getAsInt)
assertEquals(nodeId, raftManager.client.nodeId.getAsInt) raftManager.shutdown()
raftManager.shutdown() } finally {
Utils.delete(logDir)
}
} }
@ParameterizedTest @ParameterizedTest
@ -155,22 +160,27 @@ class RaftManagerTest {
} }
val nodeId = 1 val nodeId = 1
val raftManager = createRaftManager( try {
new TopicPartition("__raft_id_test", 0), val raftManager = createRaftManager(
createConfig( new TopicPartition("__raft_id_test", 0),
Set(ProcessRole.ControllerRole), createConfig(
nodeId, Set(ProcessRole.ControllerRole),
logDir, nodeId,
metadataDir logDir,
metadataDir
)
) )
)
val lockPath = metadataDir.getOrElse(logDir.head).resolve(LogManager.LOCK_FILE_NAME) val lockPath = metadataDir.getOrElse(logDir.head).resolve(LogManager.LOCK_FILE_NAME)
assertTrue(fileLocked(lockPath)) assertTrue(fileLocked(lockPath))
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,22 +189,27 @@ class RaftManagerTest {
val metadataDir = Some(TestUtils.tempDir().toPath) val metadataDir = Some(TestUtils.tempDir().toPath)
val nodeId = 1 val nodeId = 1
val raftManager = createRaftManager( try {
new TopicPartition("__raft_id_test", 0), val raftManager = createRaftManager(
createConfig( new TopicPartition("__raft_id_test", 0),
Set(ProcessRole.BrokerRole), createConfig(
nodeId, Set(ProcessRole.BrokerRole),
logDir, nodeId,
metadataDir logDir,
metadataDir
)
) )
)
val lockPath = metadataDir.getOrElse(logDir.head).resolve(LogManager.LOCK_FILE_NAME) val lockPath = metadataDir.getOrElse(logDir.head).resolve(LogManager.LOCK_FILE_NAME)
assertTrue(fileLocked(lockPath)) assertTrue(fileLocked(lockPath))
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,8 +169,10 @@ public class IQv2IntegrationTest {
@AfterEach @AfterEach
public void afterTest() { public void afterTest() {
kafkaStreams.close(Duration.ofSeconds(60)); if (kafkaStreams != null) {
kafkaStreams.cleanUp(); kafkaStreams.close(Duration.ofSeconds(60));
kafkaStreams.cleanUp();
}
} }
@AfterAll @AfterAll
@ -176,8 +180,10 @@ public class IQv2IntegrationTest {
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
kafkaStreams.close(); if (kafkaStreams != null) {
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,49 +118,46 @@ 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(), "1",
Map.of( new StreamsRebalanceData.Subtopology(
"1", Set.of("source1"),
new StreamsRebalanceData.Subtopology( Set.of(),
Set.of("source1"), Map.of("repartition1", new StreamsRebalanceData.TopicInfo(Optional.of(1), Optional.of((short) 1), Map.of())),
Set.of(), Map.of(),
Map.of("repartition1", new StreamsRebalanceData.TopicInfo(Optional.of(1), Optional.of((short) 1), Map.of())), Set.of()
Map.of(),
Set.of()
),
"2",
new StreamsRebalanceData.Subtopology(
Set.of("source2"),
Set.of(),
Map.of("repartition2", new StreamsRebalanceData.TopicInfo(Optional.of(1), Optional.of((short) 1), Map.of())),
Map.of(),
Set.of()
),
"3",
new StreamsRebalanceData.Subtopology(
Set.of("source3"),
Set.of(),
Map.of("repartition3", new StreamsRebalanceData.TopicInfo(Optional.of(1), Optional.of((short) 1), Map.of())),
Map.of(),
Set.of()
)
), ),
Map.of() "2",
)); new StreamsRebalanceData.Subtopology(
Set.of("source2"),
final Optional<Exception> result = defaultStreamsRebalanceListener.onTasksAssigned( Set.of(),
new StreamsRebalanceData.Assignment( Map.of("repartition2", new StreamsRebalanceData.TopicInfo(Optional.of(1), Optional.of((short) 1), Map.of())),
Set.of(new StreamsRebalanceData.TaskId("1", 0)), Map.of(),
Set.of(new StreamsRebalanceData.TaskId("2", 0)), Set.of()
Set.of(new StreamsRebalanceData.TaskId("3", 0)) ),
"3",
new StreamsRebalanceData.Subtopology(
Set.of("source3"),
Set.of(),
Map.of("repartition3", new StreamsRebalanceData.TopicInfo(Optional.of(1), Optional.of((short) 1), Map.of())),
Map.of(),
Set.of()
) )
));
createRebalanceListenerWithRebalanceData(streamsRebalanceData);
final StreamsRebalanceData.Assignment assignment = new StreamsRebalanceData.Assignment(
Set.of(new StreamsRebalanceData.TaskId("1", 0)),
Set.of(new StreamsRebalanceData.TaskId("2", 0)),
Set.of(new StreamsRebalanceData.TaskId("3", 0))
); );
final Optional<Exception> result = defaultStreamsRebalanceListener.onTasksAssigned(assignment);
assertTrue(result.isEmpty()); 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,80 +276,82 @@ 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"));
final String[] args = new String[]{ final String[] args = new String[]{
"--bootstrap-server", "localhost:9092", "--bootstrap-server", "localhost:9092",
"--topic", "test", "--topic", "test",
"--group", "group-from-arguments", "--group", "group-from-arguments",
"--consumer-property", "group.id=group-from-properties", "--consumer-property", "group.id=group-from-properties",
"--consumer.config", propsFile.getAbsolutePath() "--consumer.config", propsFile.getAbsolutePath()
}; };
assertThrows(IllegalArgumentException.class, () -> new ConsoleConsumerOptions(args)); assertThrows(IllegalArgumentException.class, () -> new ConsoleConsumerOptions(args));
// the same in all three places // the same in all three places
propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "test-group")); propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "test-group"));
final String[] args1 = new String[]{ final String[] args1 = new String[]{
"--bootstrap-server", "localhost:9092", "--bootstrap-server", "localhost:9092",
"--topic", "test", "--topic", "test",
"--group", "test-group", "--group", "test-group",
"--consumer-property", "group.id=test-group", "--consumer-property", "group.id=test-group",
"--consumer.config", propsFile.getAbsolutePath() "--consumer.config", propsFile.getAbsolutePath()
}; };
ConsoleConsumerOptions config = new ConsoleConsumerOptions(args1); ConsoleConsumerOptions config = new ConsoleConsumerOptions(args1);
Properties props = config.consumerProps(); Properties props = config.consumerProps();
assertEquals("test-group", props.getProperty("group.id")); assertEquals("test-group", props.getProperty("group.id"));
// different via --consumer-property and --consumer.config // different via --consumer-property and --consumer.config
propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file")); propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file"));
final String[] args2 = new String[]{ final String[] args2 = new String[]{
"--bootstrap-server", "localhost:9092", "--bootstrap-server", "localhost:9092",
"--topic", "test", "--topic", "test",
"--consumer-property", "group.id=group-from-properties", "--consumer-property", "group.id=group-from-properties",
"--consumer.config", propsFile.getAbsolutePath() "--consumer.config", propsFile.getAbsolutePath()
}; };
assertThrows(IllegalArgumentException.class, () -> new ConsoleConsumerOptions(args2)); assertThrows(IllegalArgumentException.class, () -> new ConsoleConsumerOptions(args2));
// different via --consumer-property and --group // different via --consumer-property and --group
final String[] args3 = new String[]{ final String[] args3 = new String[]{
"--bootstrap-server", "localhost:9092", "--bootstrap-server", "localhost:9092",
"--topic", "test", "--topic", "test",
"--group", "group-from-arguments", "--group", "group-from-arguments",
"--consumer-property", "group.id=group-from-properties" "--consumer-property", "group.id=group-from-properties"
}; };
assertThrows(IllegalArgumentException.class, () -> new ConsoleConsumerOptions(args3)); assertThrows(IllegalArgumentException.class, () -> new ConsoleConsumerOptions(args3));
// different via --group and --consumer.config // different via --group and --consumer.config
propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file")); propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file"));
final String[] args4 = new String[]{ final String[] args4 = new String[]{
"--bootstrap-server", "localhost:9092", "--bootstrap-server", "localhost:9092",
"--topic", "test", "--topic", "test",
"--group", "group-from-arguments", "--group", "group-from-arguments",
"--consumer.config", propsFile.getAbsolutePath() "--consumer.config", propsFile.getAbsolutePath()
}; };
assertThrows(IllegalArgumentException.class, () -> new ConsoleConsumerOptions(args4)); assertThrows(IllegalArgumentException.class, () -> new ConsoleConsumerOptions(args4));
// via --group only // via --group only
final String[] args5 = new String[]{ final String[] args5 = new String[]{
"--bootstrap-server", "localhost:9092", "--bootstrap-server", "localhost:9092",
"--topic", "test", "--topic", "test",
"--group", "group-from-arguments" "--group", "group-from-arguments"
}; };
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
@ -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,80 +112,82 @@ 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);
}); });
// different in all three places try {
File propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file")); // different in all three places
final String[] args = new String[]{ File propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file"));
"--bootstrap-server", "localhost:9092", final String[] args = new String[]{
"--topic", "test", "--bootstrap-server", "localhost:9092",
"--group", "group-from-arguments", "--topic", "test",
"--consumer-property", "group.id=group-from-properties", "--group", "group-from-arguments",
"--consumer-config", propsFile.getAbsolutePath() "--consumer-property", "group.id=group-from-properties",
}; "--consumer-config", propsFile.getAbsolutePath()
};
assertThrows(IllegalArgumentException.class, () -> new ConsoleShareConsumerOptions(args)); assertThrows(IllegalArgumentException.class, () -> new ConsoleShareConsumerOptions(args));
// the same in all three places // the same in all three places
propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "test-group")); propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "test-group"));
final String[] args1 = new String[]{ final String[] args1 = new String[]{
"--bootstrap-server", "localhost:9092", "--bootstrap-server", "localhost:9092",
"--topic", "test", "--topic", "test",
"--group", "test-group", "--group", "test-group",
"--consumer-property", "group.id=test-group", "--consumer-property", "group.id=test-group",
"--consumer-config", propsFile.getAbsolutePath() "--consumer-config", propsFile.getAbsolutePath()
}; };
ConsoleShareConsumerOptions config = new ConsoleShareConsumerOptions(args1); ConsoleShareConsumerOptions config = new ConsoleShareConsumerOptions(args1);
Properties props = config.consumerProps(); Properties props = config.consumerProps();
assertEquals("test-group", props.getProperty("group.id")); assertEquals("test-group", props.getProperty("group.id"));
// different via --consumer-property and --consumer-config // different via --consumer-property and --consumer-config
propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file")); propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file"));
final String[] args2 = new String[]{ final String[] args2 = new String[]{
"--bootstrap-server", "localhost:9092", "--bootstrap-server", "localhost:9092",
"--topic", "test", "--topic", "test",
"--consumer-property", "group.id=group-from-properties", "--consumer-property", "group.id=group-from-properties",
"--consumer-config", propsFile.getAbsolutePath() "--consumer-config", propsFile.getAbsolutePath()
}; };
assertThrows(IllegalArgumentException.class, () -> new ConsoleShareConsumerOptions(args2)); assertThrows(IllegalArgumentException.class, () -> new ConsoleShareConsumerOptions(args2));
// different via --consumer-property and --group // different via --consumer-property and --group
final String[] args3 = new String[]{ final String[] args3 = new String[]{
"--bootstrap-server", "localhost:9092", "--bootstrap-server", "localhost:9092",
"--topic", "test", "--topic", "test",
"--group", "group-from-arguments", "--group", "group-from-arguments",
"--consumer-property", "group.id=group-from-properties" "--consumer-property", "group.id=group-from-properties"
}; };
assertThrows(IllegalArgumentException.class, () -> new ConsoleShareConsumerOptions(args3)); assertThrows(IllegalArgumentException.class, () -> new ConsoleShareConsumerOptions(args3));
// different via --group and --consumer-config // different via --group and --consumer-config
propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file")); propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file"));
final String[] args4 = new String[]{ final String[] args4 = new String[]{
"--bootstrap-server", "localhost:9092", "--bootstrap-server", "localhost:9092",
"--topic", "test", "--topic", "test",
"--group", "group-from-arguments", "--group", "group-from-arguments",
"--consumer-config", propsFile.getAbsolutePath() "--consumer-config", propsFile.getAbsolutePath()
}; };
assertThrows(IllegalArgumentException.class, () -> new ConsoleShareConsumerOptions(args4)); assertThrows(IllegalArgumentException.class, () -> new ConsoleShareConsumerOptions(args4));
// via --group only // via --group only
final String[] args5 = new String[]{ final String[] args5 = new String[]{
"--bootstrap-server", "localhost:9092", "--bootstrap-server", "localhost:9092",
"--topic", "test", "--topic", "test",
"--group", "group-from-arguments" "--group", "group-from-arguments"
}; };
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
@ -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));
}
} }