/** * Get the current subscription. Will return the same topics used in the most recent call to * {@link #subscribe(Collection, ConsumerRebalanceListener)}, or an empty set if no such call has been made. * @return The set of topics currently subscribed to */ public Set<String> subscription() { acquireAndEnsureOpen(); try { return Collections.unmodifiableSet(new HashSet<>(this.subscriptions.subscription())); } finally { release(); } }
/** * 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(); } }
/** * Get the set of partitions that were previously paused by a call to {@link #pause(Collection)}. * * @return The set of paused partitions */ @Override public Set<TopicPartition> paused() { acquireAndEnsureOpen(); try { return Collections.unmodifiableSet(subscriptions.pausedPartitions()); } finally { release(); } }
/** * Suspend fetching from the requested partitions. Future calls to {@link #poll(Duration)} will not return * any records from these partitions until they have been resumed using {@link #resume(Collection)}. * Note that this method does not affect partition subscription. In particular, it does not cause a group * rebalance when automatic assignment is used. * @param partitions The partitions which should be paused * @throws IllegalStateException if any of the provided partitions are not currently assigned to this consumer */ @Override public void pause(Collection<TopicPartition> partitions) { acquireAndEnsureOpen(); try { log.debug("Pausing partitions {}", partitions); for (TopicPartition partition: partitions) { subscriptions.pause(partition); } } finally { release(); } }
/** * Resume specified partitions which have been paused with {@link #pause(Collection)}. New calls to * {@link #poll(Duration)} will return records from these partitions if there are any to be fetched. * If the partitions were not previously paused, this method is a no-op. * @param partitions The partitions which should be resumed * @throws IllegalStateException if any of the provided partitions are not currently assigned to this consumer */ @Override public void resume(Collection<TopicPartition> partitions) { acquireAndEnsureOpen(); try { log.debug("Resuming partitions {}", partitions); for (TopicPartition partition: partitions) { subscriptions.resume(partition); } } 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(); } }
acquireAndEnsureOpen(); try { for (Map.Entry<TopicPartition, Long> entry : timestampsToSearch.entrySet()) {
acquireAndEnsureOpen(); try { maybeThrowInvalidGroupIdException();
/** * Unsubscribe from topics currently subscribed with {@link #subscribe(Collection)} or {@link #subscribe(Pattern)}. * This also clears any partitions directly assigned through {@link #assign(Collection)}. */ public void unsubscribe() { acquireAndEnsureOpen(); try { fetcher.clearBufferedDataForUnassignedPartitions(Collections.emptySet()); this.subscriptions.unsubscribe(); if (this.coordinator != null) this.coordinator.maybeLeaveGroup(); this.metadata.needMetadataForAllTopics(false); log.info("Unsubscribed all topics or patterns and assigned partitions"); } finally { release(); } }
/** * Overrides the fetch offsets that the consumer will use on the next {@link #poll(Duration) poll(timeout)}. If this API * is invoked for the same partition more than once, the latest offset will be used on the next poll(). Note that * you may lose data if this API is arbitrarily used in the middle of consumption, to reset the fetch offsets. This * method allows for setting the leaderEpoch along with the desired offset. * * @throws IllegalArgumentException if the provided offset is negative * @throws IllegalStateException if the provided TopicPartition is not assigned to this consumer */ @Override public void seek(TopicPartition partition, OffsetAndMetadata offsetAndMetadata) { long offset = offsetAndMetadata.offset(); if (offset < 0) { throw new IllegalArgumentException("seek offset must not be a negative number"); } acquireAndEnsureOpen(); try { if (offsetAndMetadata.leaderEpoch().isPresent()) { log.debug("Seeking to offset {} for partition {} with epoch {}", offset, partition, offsetAndMetadata.leaderEpoch().get()); } else { log.debug("Seeking to offset {} for partition {}", offset, partition); } this.updateLastSeenEpochIfNewer(partition, offsetAndMetadata); this.subscriptions.seek(partition, offset); } finally { release(); } }
acquireAndEnsureOpen(); try { maybeThrowInvalidGroupIdException();
/** * Get the first offset for the given partitions. * <p> * This method does not change the current consumer position of the partitions. * * @see #seekToBeginning(Collection) * * @param partitions the partitions to get the earliest offsets * @param timeout The maximum amount of time to await retrieval of the beginning offsets * * @return The earliest available offsets for the given partitions * @throws org.apache.kafka.common.errors.AuthenticationException if authentication fails. See the exception for more details * @throws org.apache.kafka.common.errors.AuthorizationException if not authorized to the topic(s). See the exception for more details * @throws org.apache.kafka.common.errors.TimeoutException if the offset metadata could not be fetched before * expiration of the passed timeout */ @Override public Map<TopicPartition, Long> beginningOffsets(Collection<TopicPartition> partitions, Duration timeout) { acquireAndEnsureOpen(); try { return fetcher.beginningOffsets(partitions, time.timer(timeout)); } finally { release(); } }
/** * Get metadata about partitions for all topics that the user is authorized to view. This method will issue a * remote call to the server. * * @param timeout The maximum time this operation will block to fetch topic metadata * * @return The map of topics and its partitions * @throws org.apache.kafka.common.errors.WakeupException if {@link #wakeup()} is called before or while this * function is called * @throws org.apache.kafka.common.errors.InterruptException if the calling thread is interrupted before or while * this function is called * @throws org.apache.kafka.common.errors.TimeoutException if the topic metadata could not be fetched before * expiration of the passed timeout * @throws org.apache.kafka.common.KafkaException for any other unrecoverable errors */ @Override public Map<String, List<PartitionInfo>> listTopics(Duration timeout) { acquireAndEnsureOpen(); try { return fetcher.getAllTopicMetadata(time.timer(timeout)); } finally { release(); } }
/** * Commit offsets returned on the last {@link #poll(Duration) poll()} for the subscribed list of topics and partitions. * <p> * This commits offsets only to Kafka. The offsets committed using this API will be used on the first fetch after * every rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API * should not be used. * <p> * This is an asynchronous call and will not block. Any errors encountered are either passed to the callback * (if provided) or discarded. * <p> * Offsets committed through multiple calls to this API are guaranteed to be sent in the same order as * the invocations. Corresponding commit callbacks are also invoked in the same order. Additionally note that * offsets committed through this API are guaranteed to complete before a subsequent call to {@link #commitSync()} * (and variants) returns. * * @param callback Callback to invoke when the commit completes */ @Override public void commitAsync(OffsetCommitCallback callback) { acquireAndEnsureOpen(); try { commitAsync(subscriptions.allConsumed(), callback); } finally { release(); } }
/** * Get the end offsets for the given partitions. In the default {@code read_uncommitted} isolation level, the end * offset is the high watermark (that is, the offset of the last successfully replicated message plus one). For * {@code read_committed} consumers, the end offset is the last stable offset (LSO), which is the minimum of * the high watermark and the smallest offset of any open transaction. Finally, if the partition has never been * written to, the end offset is 0. * * <p> * This method does not change the current consumer position of the partitions. * * @see #seekToEnd(Collection) * * @param partitions the partitions to get the end offsets. * @param timeout The maximum amount of time to await retrieval of the end offsets * * @return The end offsets for the given partitions. * @throws org.apache.kafka.common.errors.AuthenticationException if authentication fails. See the exception for more details * @throws org.apache.kafka.common.errors.AuthorizationException if not authorized to the topic(s). See the exception for more details * @throws org.apache.kafka.common.errors.TimeoutException if the offsets could not be fetched before * expiration of the passed timeout */ @Override public Map<TopicPartition, Long> endOffsets(Collection<TopicPartition> partitions, Duration timeout) { acquireAndEnsureOpen(); try { return fetcher.endOffsets(partitions, time.timer(timeout)); } finally { release(); } }
acquireAndEnsureOpen(); try { Cluster cluster = this.metadata.fetch();
acquireAndEnsureOpen(); try { maybeThrowInvalidGroupIdException();
throw new IllegalArgumentException("Topic pattern to subscribe to cannot be null"); acquireAndEnsureOpen(); try { throwIfNoAssignorsConfigured();
private ConsumerRecords<K, V> poll(final Timer timer, final boolean includeMetadataInTimeout) { acquireAndEnsureOpen(); try { if (this.subscriptions.hasNoSubscriptionOrUserAssignment()) {