@Override public synchronized Set<TopicPartition> assignment() { return this.subscriptions.assignedPartitions(); }
public synchronized void addRecord(ConsumerRecord<K, V> record) { ensureNotClosed(); TopicPartition tp = new TopicPartition(record.topic(), record.partition()); Set<TopicPartition> currentAssigned = new HashSet<>(this.subscriptions.assignedPartitions()); if (!currentAssigned.contains(tp)) throw new IllegalStateException("Cannot add records for a partition that is not assigned to the consumer"); List<ConsumerRecord<K, V>> recs = this.records.computeIfAbsent(tp, k -> new ArrayList<>()); recs.add(record); }
/** * Clear the buffered data which are not a part of newly assigned topics * * @param assignedTopics newly assigned topics */ public void clearBufferedDataForUnassignedTopics(Collection<String> assignedTopics) { Set<TopicPartition> currentTopicPartitions = new HashSet<>(); for (TopicPartition tp : subscriptions.assignedPartitions()) { if (assignedTopics.contains(tp.topic())) { currentTopicPartitions.add(tp); } } clearBufferedDataForUnassignedPartitions(currentTopicPartitions); }
/** * Get the set of partitions currently assigned to this consumer. If subscription happened by directly assigning * partitions using {@link #assign(Collection)} then this will simply return the same partitions that * were assigned. If topic subscription was used, then this will give the set of topic partitions currently assigned * to the consumer (which may be none if the assignment hasn't happened yet, or the partitions are in the * process of getting reassigned). * @return The set of partitions currently assigned to this consumer */ public Set<TopicPartition> assignment() { acquireAndEnsureOpen(); try { return Collections.unmodifiableSet(new HashSet<>(this.subscriptions.assignedPartitions())); } finally { release(); } }
/** * Seek to the first offset for each of the given partitions. This function evaluates lazily, seeking to the * first offset in all partitions only when {@link #poll(Duration)} or {@link #position(TopicPartition)} are called. * If no partitions are provided, seek to the first offset for all of the currently assigned partitions. * * @throws IllegalArgumentException if {@code partitions} is {@code null} * @throws IllegalStateException if any of the provided partitions are not currently assigned to this consumer */ @Override public void seekToBeginning(Collection<TopicPartition> partitions) { if (partitions == null) throw new IllegalArgumentException("Partitions collection cannot be null"); acquireAndEnsureOpen(); try { Collection<TopicPartition> parts = partitions.size() == 0 ? this.subscriptions.assignedPartitions() : partitions; for (TopicPartition tp : parts) { log.debug("Seeking to beginning of partition {}", tp); subscriptions.requestOffsetReset(tp, OffsetResetStrategy.EARLIEST); } } finally { release(); } }
/** * Seek to the last offset for each of the given partitions. This function evaluates lazily, seeking to the * final offset in all partitions only when {@link #poll(Duration)} or {@link #position(TopicPartition)} are called. * If no partitions are provided, seek to the final offset for all of the currently assigned partitions. * <p> * If {@code isolation.level=read_committed}, the end offset will be the Last Stable Offset, i.e., the offset * of the first message with an open transaction. * * @throws IllegalArgumentException if {@code partitions} is {@code null} * @throws IllegalStateException if any of the provided partitions are not currently assigned to this consumer */ @Override public void seekToEnd(Collection<TopicPartition> partitions) { if (partitions == null) throw new IllegalArgumentException("Partitions collection cannot be null"); acquireAndEnsureOpen(); try { Collection<TopicPartition> parts = partitions.size() == 0 ? this.subscriptions.assignedPartitions() : partitions; for (TopicPartition tp : parts) { log.debug("Seeking to end of partition {}", tp); subscriptions.requestOffsetReset(tp, OffsetResetStrategy.LATEST); } } finally { release(); } }
@Override protected void onJoinPrepare(int generation, String memberId) { // commit offsets prior to rebalance if auto-commit enabled maybeAutoCommitOffsetsSync(time.timer(rebalanceTimeoutMs)); // execute the user's callback before rebalance ConsumerRebalanceListener listener = subscriptions.rebalanceListener(); // copy since about to be handed to user code Set<TopicPartition> revoked = new HashSet<>(subscriptions.assignedPartitions()); log.info("Revoking previously assigned partitions {}", revoked); try { listener.onPartitionsRevoked(revoked); } catch (WakeupException | InterruptException e) { throw e; } catch (Exception e) { log.error("User provided listener {} failed on partition revocation", listener.getClass().getName(), e); } isLeader = false; subscriptions.resetGroupSubscription(); }
@Test public void unsubscribeUserSubscribe() { state.subscribe(singleton(topic), rebalanceListener); state.unsubscribe(); state.assignFromUser(singleton(tp0)); assertEquals(singleton(tp0), state.assignedPartitions()); assertEquals(1, state.numAssignedPartitions()); }
@Test public void partitionAssignmentChangeOnTopicSubscription() { state.assignFromUser(new HashSet<>(Arrays.asList(tp0, tp1))); // assigned partitions should immediately change assertEquals(2, state.assignedPartitions().size()); assertEquals(2, state.numAssignedPartitions()); assertTrue(state.assignedPartitions().contains(tp0)); assertTrue(state.assignedPartitions().contains(tp1)); state.unsubscribe(); // assigned partitions should immediately change assertTrue(state.assignedPartitions().isEmpty()); assertEquals(0, state.numAssignedPartitions()); state.subscribe(singleton(topic1), rebalanceListener); // assigned partitions should remain unchanged assertTrue(state.assignedPartitions().isEmpty()); assertEquals(0, state.numAssignedPartitions()); state.assignFromSubscribed(singleton(t1p0)); // assigned partitions should immediately change assertEquals(singleton(t1p0), state.assignedPartitions()); assertEquals(1, state.numAssignedPartitions()); state.subscribe(singleton(topic), rebalanceListener); // assigned partitions should remain unchanged assertEquals(singleton(t1p0), state.assignedPartitions()); assertEquals(1, state.numAssignedPartitions()); state.unsubscribe(); // assigned partitions should immediately change assertTrue(state.assignedPartitions().isEmpty()); assertEquals(0, state.numAssignedPartitions()); }
@Test public void unsubscription() { state.subscribe(Pattern.compile(".*"), rebalanceListener); state.subscribeFromPattern(new HashSet<>(Arrays.asList(topic, topic1))); state.assignFromSubscribed(singleton(tp1)); assertEquals(singleton(tp1), state.assignedPartitions()); assertEquals(1, state.numAssignedPartitions()); state.unsubscribe(); assertEquals(0, state.subscription().size()); assertTrue(state.assignedPartitions().isEmpty()); assertEquals(0, state.numAssignedPartitions()); state.assignFromUser(singleton(tp0)); assertEquals(singleton(tp0), state.assignedPartitions()); assertEquals(1, state.numAssignedPartitions()); state.unsubscribe(); assertEquals(0, state.subscription().size()); assertTrue(state.assignedPartitions().isEmpty()); assertEquals(0, state.numAssignedPartitions()); }
state.subscribe(Pattern.compile(".*"), rebalanceListener); assertTrue(state.assignedPartitions().isEmpty()); assertEquals(0, state.numAssignedPartitions()); assertTrue(state.assignedPartitions().isEmpty()); assertEquals(0, state.numAssignedPartitions()); assertEquals(singleton(tp1), state.assignedPartitions()); assertEquals(1, state.numAssignedPartitions()); assertEquals(singleton(topic), state.subscription()); assertEquals(singleton(t1p0), state.assignedPartitions()); assertEquals(1, state.numAssignedPartitions()); assertEquals(singleton(topic), state.subscription()); assertEquals(singleton(t1p0), state.assignedPartitions()); assertEquals(1, state.numAssignedPartitions()); assertEquals(singleton(t1p0), state.assignedPartitions()); assertEquals(1, state.numAssignedPartitions()); assertEquals(singleton(tp0), state.assignedPartitions()); assertEquals(1, state.numAssignedPartitions()); assertEquals(singleton(topic), state.subscription()); assertTrue(state.assignedPartitions().isEmpty()); assertEquals(0, state.numAssignedPartitions());
@Test public void partitionAssignment() { state.assignFromUser(singleton(tp0)); assertEquals(singleton(tp0), state.assignedPartitions()); assertEquals(1, state.numAssignedPartitions()); assertFalse(state.hasAllFetchPositions()); state.seek(tp0, 1); assertTrue(state.isFetchable(tp0)); assertEquals(1L, state.position(tp0).longValue()); state.assignFromUser(Collections.<TopicPartition>emptySet()); assertTrue(state.assignedPartitions().isEmpty()); assertEquals(0, state.numAssignedPartitions()); assertFalse(state.isAssigned(tp0)); assertFalse(state.isFetchable(tp0)); }
@Test public void topicSubscription() { state.subscribe(singleton(topic), rebalanceListener); assertEquals(1, state.subscription().size()); assertTrue(state.assignedPartitions().isEmpty()); assertEquals(0, state.numAssignedPartitions()); assertTrue(state.partitionsAutoAssigned()); state.assignFromSubscribed(singleton(tp0)); state.seek(tp0, 1); assertEquals(1L, state.position(tp0).longValue()); state.assignFromSubscribed(singleton(tp1)); assertTrue(state.isAssigned(tp1)); assertFalse(state.isAssigned(tp0)); assertFalse(state.isFetchable(tp1)); assertEquals(singleton(tp1), state.assignedPartitions()); assertEquals(1, state.numAssignedPartitions()); }
assertEquals(singleton(t1p), subscriptions.assignedPartitions()); assertEquals(singleton(topic1), subscriptions.groupSubscription()); assertEquals(1, rebalanceListener.revokedCount);
@Test public void testWakeupDuringJoin() { final String consumerId = "leader"; subscriptions.subscribe(singleton(topic1), rebalanceListener); // ensure metadata is up-to-date for leader metadata.setTopics(singletonList(topic1)); client.updateMetadata(metadataResponse); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE)); Map<String, List<String>> memberSubscriptions = singletonMap(consumerId, singletonList(topic1)); partitionAssignor.prepare(singletonMap(consumerId, singletonList(t1p))); // prepare only the first half of the join and then trigger the wakeup client.prepareResponse(joinGroupLeaderResponse(1, consumerId, memberSubscriptions, Errors.NONE)); consumerClient.wakeup(); try { coordinator.poll(time.timer(Long.MAX_VALUE)); } catch (WakeupException e) { // ignore } // now complete the second half client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE)); coordinator.poll(time.timer(Long.MAX_VALUE)); assertFalse(coordinator.rejoinNeededOrPending()); assertEquals(singleton(t1p), subscriptions.assignedPartitions()); assertEquals(1, rebalanceListener.revokedCount); assertEquals(Collections.emptySet(), rebalanceListener.revoked); assertEquals(1, rebalanceListener.assignedCount); assertEquals(singleton(t1p), rebalanceListener.assigned); }
@Test public void testRebalanceInProgressOnSyncGroup() { final String consumerId = "consumer"; subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE)); // join initially, but let coordinator rebalance on sync client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE)); client.prepareResponse(syncGroupResponse(Collections.<TopicPartition>emptyList(), Errors.REBALANCE_IN_PROGRESS)); // then let the full join/sync finish successfully client.prepareResponse(joinGroupFollowerResponse(2, consumerId, "leader", Errors.NONE)); client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE)); coordinator.joinGroupIfNeeded(time.timer(Long.MAX_VALUE)); assertFalse(coordinator.rejoinNeededOrPending()); assertEquals(singleton(t1p), subscriptions.assignedPartitions()); }
@Test public void testNormalJoinGroupFollower() { final String consumerId = "consumer"; subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE)); // normal join group client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE)); client.prepareResponse(new MockClient.RequestMatcher() { @Override public boolean matches(AbstractRequest body) { SyncGroupRequest sync = (SyncGroupRequest) body; return sync.memberId().equals(consumerId) && sync.generationId() == 1 && sync.groupAssignment().isEmpty(); } }, syncGroupResponse(singletonList(t1p), Errors.NONE)); coordinator.joinGroupIfNeeded(time.timer(Long.MAX_VALUE)); assertFalse(coordinator.rejoinNeededOrPending()); assertEquals(singleton(t1p), subscriptions.assignedPartitions()); assertEquals(singleton(topic1), subscriptions.groupSubscription()); assertEquals(1, rebalanceListener.revokedCount); assertEquals(Collections.emptySet(), rebalanceListener.revoked); assertEquals(1, rebalanceListener.assignedCount); assertEquals(singleton(t1p), rebalanceListener.assigned); }
@Test public void testUnknownMemberIdOnSyncGroup() { final String consumerId = "consumer"; subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE)); // join initially, but let coordinator returns unknown member id client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE)); client.prepareResponse(syncGroupResponse(Collections.<TopicPartition>emptyList(), Errors.UNKNOWN_MEMBER_ID)); // now we should see a new join with the empty UNKNOWN_MEMBER_ID client.prepareResponse(new MockClient.RequestMatcher() { @Override public boolean matches(AbstractRequest body) { JoinGroupRequest joinRequest = (JoinGroupRequest) body; return joinRequest.memberId().equals(JoinGroupRequest.UNKNOWN_MEMBER_ID); } }, joinGroupFollowerResponse(2, consumerId, "leader", Errors.NONE)); client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE)); coordinator.joinGroupIfNeeded(time.timer(Long.MAX_VALUE)); assertFalse(coordinator.rejoinNeededOrPending()); assertEquals(singleton(t1p), subscriptions.assignedPartitions()); }
@Test public void testIllegalGenerationOnSyncGroup() { final String consumerId = "consumer"; subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE)); // join initially, but let coordinator rebalance on sync client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE)); client.prepareResponse(syncGroupResponse(Collections.<TopicPartition>emptyList(), Errors.ILLEGAL_GENERATION)); // then let the full join/sync finish successfully client.prepareResponse(new MockClient.RequestMatcher() { @Override public boolean matches(AbstractRequest body) { JoinGroupRequest joinRequest = (JoinGroupRequest) body; return joinRequest.memberId().equals(JoinGroupRequest.UNKNOWN_MEMBER_ID); } }, joinGroupFollowerResponse(2, consumerId, "leader", Errors.NONE)); client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE)); coordinator.joinGroupIfNeeded(time.timer(Long.MAX_VALUE)); assertFalse(coordinator.rejoinNeededOrPending()); assertEquals(singleton(t1p), subscriptions.assignedPartitions()); }
@Test public void testDisconnectInJoin() { subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE)); // disconnected from original coordinator will cause re-discover and join again client.prepareResponse(joinGroupFollowerResponse(1, "consumer", "leader", Errors.NONE), true); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); client.prepareResponse(joinGroupFollowerResponse(1, "consumer", "leader", Errors.NONE)); client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE)); coordinator.joinGroupIfNeeded(time.timer(Long.MAX_VALUE)); assertFalse(coordinator.rejoinNeededOrPending()); assertEquals(singleton(t1p), subscriptions.assignedPartitions()); assertEquals(1, rebalanceListener.revokedCount); assertEquals(1, rebalanceListener.assignedCount); assertEquals(singleton(t1p), rebalanceListener.assigned); }