-
Notifications
You must be signed in to change notification settings - Fork 15.1k
KAFKA-4317: Regularly checkpoint StateStore changelog offsets #2471
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
6743dc6
291ff37
c908a0d
d0cf6d8
f7553a3
2531a3c
9659669
52745a5
5cd9295
6959156
05e08c1
90ca3e8
dfe52c6
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 |
|---|---|---|
| @@ -0,0 +1,27 @@ | ||
| /** | ||
| * 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 | ||
| * <p> | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * <p> | ||
| * 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.streams.processor.internals; | ||
|
|
||
| import org.apache.kafka.common.TopicPartition; | ||
|
|
||
| import java.util.Map; | ||
|
|
||
| // Interface to indicate that an object has associated partition offsets that can be checkpointed | ||
| interface Checkpointable { | ||
| void checkpoint(final Map<TopicPartition, Long> offsets); | ||
| Map<TopicPartition, Long> checkpointed(); | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -68,6 +68,7 @@ public class ProcessorStateManager implements StateManager { | |
| // TODO: this map does not work with customized grouper where multiple partitions | ||
| // of the same topic can be assigned to the same topic. | ||
| private final Map<String, TopicPartition> partitionForTopic; | ||
| private final OffsetCheckpoint checkpoint; | ||
|
|
||
| /** | ||
| * @throws LockException if the state directory cannot be locked because another thread holds the lock | ||
|
|
@@ -103,11 +104,8 @@ public ProcessorStateManager(final TaskId taskId, | |
| } | ||
|
|
||
| // load the checkpoint information | ||
| OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(this.baseDir, CHECKPOINT_FILE_NAME)); | ||
| checkpoint = new OffsetCheckpoint(new File(this.baseDir, CHECKPOINT_FILE_NAME)); | ||
| this.checkpointedOffsets = new HashMap<>(checkpoint.read()); | ||
|
|
||
| // delete the checkpoint file after finish loading its stored offsets | ||
| checkpoint.delete(); | ||
| } | ||
|
|
||
|
|
||
|
|
@@ -250,7 +248,7 @@ private void restoreActiveState(String topicName, StateRestoreCallback stateRest | |
| } | ||
| } | ||
|
|
||
| public Map<TopicPartition, Long> checkpointedOffsets() { | ||
| public Map<TopicPartition, Long> checkpointed() { | ||
| Map<TopicPartition, Long> partitionsAndOffsets = new HashMap<>(); | ||
|
|
||
| for (Map.Entry<String, StateRestoreCallback> entry : restoreCallbacks.entrySet()) { | ||
|
|
@@ -347,29 +345,7 @@ public void close(Map<TopicPartition, Long> ackedOffsets) throws IOException { | |
| } | ||
|
|
||
| if (ackedOffsets != null) { | ||
| Map<TopicPartition, Long> checkpointOffsets = new HashMap<>(); | ||
| for (String storeName : stores.keySet()) { | ||
| // only checkpoint the offset to the offsets file if | ||
| // it is persistent AND changelog enabled | ||
| if (stores.get(storeName).persistent() && storeToChangelogTopic.containsKey(storeName)) { | ||
| String changelogTopic = storeToChangelogTopic.get(storeName); | ||
| TopicPartition topicPartition = new TopicPartition(changelogTopic, getPartition(storeName)); | ||
| Long offset = ackedOffsets.get(topicPartition); | ||
|
|
||
| if (offset != null) { | ||
| // store the last offset + 1 (the log position after restoration) | ||
| checkpointOffsets.put(topicPartition, offset + 1); | ||
| } else { | ||
| // if no record was produced. we need to check the restored offset. | ||
| offset = restoredOffsets.get(topicPartition); | ||
| if (offset != null) | ||
| checkpointOffsets.put(topicPartition, offset); | ||
| } | ||
| } | ||
| } | ||
| // write the checkpoint file before closing, to indicate clean shutdown | ||
| OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(this.baseDir, CHECKPOINT_FILE_NAME)); | ||
| checkpoint.write(checkpointOffsets); | ||
| checkpoint(ackedOffsets); | ||
| } | ||
|
|
||
| } | ||
|
|
@@ -379,6 +355,31 @@ public void close(Map<TopicPartition, Long> ackedOffsets) throws IOException { | |
| } | ||
| } | ||
|
|
||
| // write the checkpoint | ||
| @Override | ||
| public void checkpoint(final Map<TopicPartition, Long> ackedOffsets) { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Nah, the offsets are acked first. They need to be otherwise we risk writing a checkpoint for data that has not been acked.
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes. Got confused about the correct order... |
||
| for (String storeName : stores.keySet()) { | ||
| // only checkpoint the offset to the offsets file if | ||
| // it is persistent AND changelog enabled | ||
| if (stores.get(storeName).persistent() && storeToChangelogTopic.containsKey(storeName)) { | ||
| final String changelogTopic = storeToChangelogTopic.get(storeName); | ||
| final TopicPartition topicPartition = new TopicPartition(changelogTopic, getPartition(storeName)); | ||
| if (ackedOffsets.containsKey(topicPartition)) { | ||
| // store the last offset + 1 (the log position after restoration) | ||
| checkpointedOffsets.put(topicPartition, ackedOffsets.get(topicPartition) + 1); | ||
| } else if (restoredOffsets.containsKey(topicPartition)) { | ||
| checkpointedOffsets.put(topicPartition, restoredOffsets.get(topicPartition)); | ||
| } | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. what about the case that both if's are
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yeah it can happen. The table may not have received any updates in the period, so both would be empty |
||
| } | ||
| } | ||
| // write the checkpoint file before closing, to indicate clean shutdown | ||
| try { | ||
| checkpoint.write(checkpointedOffsets); | ||
| } catch (IOException e) { | ||
| log.warn("Failed to write checkpoint file to {}", new File(baseDir, CHECKPOINT_FILE_NAME), e); | ||
| } | ||
| } | ||
|
|
||
| private int getPartition(String topic) { | ||
| TopicPartition partition = partitionForTopic.get(topic); | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -51,14 +51,15 @@ public class StandbyTask extends AbstractTask { | |
| * @param metrics the {@link StreamsMetrics} created by the thread | ||
| * @param stateDirectory the {@link StateDirectory} created by the thread | ||
| */ | ||
| public StandbyTask(TaskId id, | ||
| String applicationId, | ||
| Collection<TopicPartition> partitions, | ||
| ProcessorTopology topology, | ||
| Consumer<byte[], byte[]> consumer, | ||
| Consumer<byte[], byte[]> restoreConsumer, | ||
| StreamsConfig config, | ||
| StreamsMetrics metrics, final StateDirectory stateDirectory) { | ||
| public StandbyTask(final TaskId id, | ||
| final String applicationId, | ||
| final Collection<TopicPartition> partitions, | ||
| final ProcessorTopology topology, | ||
| final Consumer<byte[], byte[]> consumer, | ||
| final Consumer<byte[], byte[]> restoreConsumer, | ||
| final StreamsConfig config, | ||
| final StreamsMetrics metrics, | ||
| final StateDirectory stateDirectory) { | ||
| super(id, applicationId, partitions, topology, consumer, restoreConsumer, true, stateDirectory, null); | ||
|
|
||
| // initialize the topology with its own context | ||
|
|
@@ -67,9 +68,9 @@ public StandbyTask(TaskId id, | |
| log.info("standby-task [{}] Initializing state stores", id()); | ||
| initializeStateStores(); | ||
|
|
||
| ((StandbyContextImpl) this.processorContext).initialized(); | ||
| this.processorContext.initialized(); | ||
|
|
||
| this.checkpointedOffsets = Collections.unmodifiableMap(stateMgr.checkpointedOffsets()); | ||
| this.checkpointedOffsets = Collections.unmodifiableMap(stateMgr.checkpointed()); | ||
| } | ||
|
|
||
| public Map<TopicPartition, Long> checkpointedOffsets() { | ||
|
|
@@ -92,7 +93,7 @@ public List<ConsumerRecord<byte[], byte[]>> update(TopicPartition partition, Lis | |
| public void commit() { | ||
| log.debug("standby-task [{}] Committing its state", id()); | ||
| stateMgr.flush(processorContext); | ||
|
|
||
| stateMgr.checkpoint(Collections.<TopicPartition, Long>emptyMap()); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is this correct, to always write empty map (is it interpreted as offset 0)?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. yes it is correct as for standby task there are no ackedOffsets. It uses the restoredOffsets in
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes, as there are no ackedOffsets for standby tasks. The |
||
| // reinitialize offset limits | ||
| initializeOffsetLimits(); | ||
| } | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -24,7 +24,7 @@ | |
| import java.io.IOException; | ||
| import java.util.Map; | ||
|
|
||
| interface StateManager { | ||
| interface StateManager extends Checkpointable { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Should we move function |
||
| File baseDir(); | ||
|
|
||
| void register(final StateStore store, final boolean loggingEnabled, final StateRestoreCallback stateRestoreCallback); | ||
|
|
@@ -36,6 +36,4 @@ interface StateManager { | |
| StateStore getGlobalStore(final String name); | ||
|
|
||
| StateStore getStore(final String name); | ||
|
|
||
| Map<TopicPartition, Long> checkpointedOffsets(); | ||
| } | ||
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.
Can the checkpoint file grow indefinitely?
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.
no - it is overwritten each time. See
OffsetCheckpoint#write