-
Notifications
You must be signed in to change notification settings - Fork 15k
KAFKA-5697: Use nonblocking poll in Streams #5107
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -50,6 +50,7 @@ | |
| import org.apache.kafka.streams.state.internals.ThreadCache; | ||
| import org.slf4j.Logger; | ||
|
|
||
| import java.time.Duration; | ||
| import java.util.ArrayList; | ||
| import java.util.Arrays; | ||
| import java.util.Collection; | ||
|
|
@@ -212,7 +213,7 @@ State setState(final State newState) { | |
| if (newState == State.RUNNING) { | ||
| updateThreadMetadata(taskManager.activeTasks(), taskManager.standbyTasks()); | ||
| } else { | ||
| updateThreadMetadata(Collections.<TaskId, StreamTask>emptyMap(), Collections.<TaskId, StandbyTask>emptyMap()); | ||
| updateThreadMetadata(Collections.emptyMap(), Collections.emptyMap()); | ||
|
||
| } | ||
| } | ||
|
|
||
|
|
@@ -555,7 +556,7 @@ static class StreamsMetricsThreadImpl extends StreamsMetricsImpl { | |
| } | ||
|
|
||
| private final Time time; | ||
| private final long pollTimeMs; | ||
| private final Duration pollTime; | ||
| private final long commitTimeMs; | ||
| private final Object stateLock; | ||
| private final Logger log; | ||
|
|
@@ -602,7 +603,8 @@ public static StreamThread create(final InternalTopologyBuilder builder, | |
| log.info("Creating restore consumer client"); | ||
| final Map<String, Object> restoreConsumerConfigs = config.getRestoreConsumerConfigs(threadClientId); | ||
| final Consumer<byte[], byte[]> restoreConsumer = clientSupplier.getRestoreConsumer(restoreConsumerConfigs); | ||
| final StoreChangelogReader changelogReader = new StoreChangelogReader(restoreConsumer, userStateRestoreListener, logContext); | ||
| final Duration pollTime = Duration.ofMillis(config.getLong(StreamsConfig.POLL_MS_CONFIG)); | ||
|
||
| final StoreChangelogReader changelogReader = new StoreChangelogReader(restoreConsumer, pollTime, userStateRestoreListener, logContext); | ||
|
|
||
| Producer<byte[], byte[]> threadProducer = null; | ||
| final boolean eosEnabled = StreamsConfig.EXACTLY_ONCE.equals(config.getString(StreamsConfig.PROCESSING_GUARANTEE_CONFIG)); | ||
|
|
@@ -710,10 +712,10 @@ public StreamThread(final Time time, | |
| this.originalReset = originalReset; | ||
| this.versionProbingFlag = versionProbingFlag; | ||
|
|
||
| this.pollTimeMs = config.getLong(StreamsConfig.POLL_MS_CONFIG); | ||
| this.pollTime = Duration.ofMillis(config.getLong(StreamsConfig.POLL_MS_CONFIG)); | ||
|
||
| this.commitTimeMs = config.getLong(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG); | ||
|
|
||
| updateThreadMetadata(Collections.<TaskId, StreamTask>emptyMap(), Collections.<TaskId, StandbyTask>emptyMap()); | ||
| updateThreadMetadata(Collections.emptyMap(), Collections.emptyMap()); | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -801,14 +803,14 @@ long runOnce(final long recordsProcessedBeforeCommit) { | |
| if (state == State.PARTITIONS_ASSIGNED) { | ||
| // try to fetch some records with zero poll millis | ||
| // to unblock the restoration as soon as possible | ||
| records = pollRequests(0L); | ||
| records = pollRequests(Duration.ZERO); | ||
|
|
||
| if (taskManager.updateNewAndRestoringTasks()) { | ||
| setState(State.RUNNING); | ||
| } | ||
| } else { | ||
| // try to fetch some records if necessary | ||
| records = pollRequests(pollTimeMs); | ||
| records = pollRequests(pollTime); | ||
|
|
||
| // if state changed after the poll call, | ||
| // try to initialize the assigned tasks again | ||
|
|
@@ -843,15 +845,15 @@ long runOnce(final long recordsProcessedBeforeCommit) { | |
| /** | ||
| * Get the next batch of records by polling. | ||
| * | ||
| * @param pollTimeMs poll time millis parameter for the consumer poll | ||
| * @param pollTime how long to block in Consumer#poll | ||
| * @return Next batch of records or null if no records available. | ||
| * @throws TaskMigratedException if the task producer got fenced (EOS only) | ||
| */ | ||
| private ConsumerRecords<byte[], byte[]> pollRequests(final long pollTimeMs) { | ||
| private ConsumerRecords<byte[], byte[]> pollRequests(final Duration pollTime) { | ||
| ConsumerRecords<byte[], byte[]> records = null; | ||
|
|
||
| try { | ||
| records = consumer.poll(pollTimeMs); | ||
| records = consumer.poll(pollTime); | ||
| } catch (final InvalidOffsetException e) { | ||
| resetInvalidOffsets(e); | ||
| } | ||
|
|
@@ -1078,7 +1080,11 @@ private void maybeUpdateStandbyTasks(final long now) { | |
| } | ||
|
|
||
| try { | ||
| final ConsumerRecords<byte[], byte[]> records = restoreConsumer.poll(0); | ||
| // poll(0): Since this is during the normal processing, not during restoration. | ||
| // We can afford to have slower restore (because we don't wait inside poll for results). | ||
| // Instead, we want to proceed to the next iteration to call the main consumer#poll() | ||
| // as soon as possible so as to not be kicked out of the group. | ||
| final ConsumerRecords<byte[], byte[]> records = restoreConsumer.poll(Duration.ZERO); | ||
|
|
||
| if (!records.isEmpty()) { | ||
| for (final TopicPartition partition : records.partitions()) { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -54,6 +54,7 @@ | |
| import org.apache.kafka.streams.state.WindowStore; | ||
|
|
||
| import java.io.IOException; | ||
| import java.time.Duration; | ||
| import java.util.ArrayList; | ||
| import java.util.List; | ||
| import java.util.Locale; | ||
|
|
@@ -334,7 +335,7 @@ private void consumeAndProduce(final String topic) { | |
| consumer.seekToBeginning(partitions); | ||
|
|
||
| while (true) { | ||
| final ConsumerRecords<Integer, byte[]> records = consumer.poll(POLL_MS); | ||
| final ConsumerRecords<Integer, byte[]> records = consumer.poll(Duration.ofMillis(POLL_MS)); | ||
|
||
| if (records.isEmpty()) { | ||
| if (processedRecords == numRecords) { | ||
| break; | ||
|
|
@@ -372,7 +373,7 @@ private void consume(final String topic) { | |
| consumer.seekToBeginning(partitions); | ||
|
|
||
| while (true) { | ||
| final ConsumerRecords<Integer, byte[]> records = consumer.poll(POLL_MS); | ||
| final ConsumerRecords<Integer, byte[]> records = consumer.poll(Duration.ofMillis(POLL_MS)); | ||
| if (records.isEmpty()) { | ||
| if (processedRecords == numRecords) { | ||
| break; | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This will return the value for
"poll.ms"-- Should we check for"global.consumer.poll.ms"and user this value if present instead?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Hmm..
poll.msis not a ConsumerConfig, but a StreamsConfig, so users are not expected to prefix it with the consumer.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I see what you mean. Still I am wondering if a single config for all consumers is smart. Also, the config is only used to pass into
Consumer#poll()-- so even if it's not a config parameter, it's effectively still very similar to a "consumer config". I agree that it would required explicit documentation.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I agree it is generally better to allow different values for this; on the other hand, I felt it is too much of a lower-level details for people to really figure it out right. For example, today global consumers have two callers of
poll(), restore consumer has two as well, and normal consumer has one. We can argue further that all these five calls may prefer different values for the reason we discussed in this PR, like whether it is call in the main loop or not.Besides, if want to allow prefix for this config we can rush it into 2.0, but it may be considered a public API change as well.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ack. I just convinced myself that it makes sense what you say. As you pointed out, we call poll() on different consumers with different purpose:
normale processing:
updating standby tasks:
restoring:
Applying the same value
poll.msto main and global consumer in normal processing case makes sense to me. For updating standby-tasks, we hard-code "zero" already. Same for the main-consumer during restore phase, to speed up restoring. Both are reasonable choices. The last case, is the restore case for restore-consumer and global-consumer -- it might make sense to apply a different poll-time for thiscase, but I also think that using the same as for the normal processing case is fine.Thus, long story short, iff we ever introduce a prefix for
poll.msparameters, it should be for "normal processing phase" and "restore phase" but should not be tied to the actual used consumer-instance. Thus, using the consumer prefix does not make sense.