KAFKA-5998: fix checkpointableOffsets handling#7030
KAFKA-5998: fix checkpointableOffsets handling#7030bbejeck merged 3 commits intoapache:trunkfrom vvcephei:KAFKA-5998-fix-checkpointableOffsets-handling-trunk
Conversation
| * of Global State Stores. There is only ever 1 instance of this class per Application Instance. | ||
| */ | ||
| public class GlobalStateManagerImpl extends AbstractStateManager implements GlobalStateManager { | ||
| public class GlobalStateManagerImpl implements GlobalStateManager { |
There was a problem hiding this comment.
Stop sharing mutable state between a superclass and subclass. The only reason to do it was to support the re-initialization logic, but the checkpoint map can just as easily be passed in as a parameter.
| eosEnabled = StreamsConfig.EXACTLY_ONCE.equals(config.getString(StreamsConfig.PROCESSING_GUARANTEE_CONFIG)); | ||
| baseDir = stateDirectory.globalStateDir(); | ||
| checkpointFile = new OffsetCheckpoint(new File(baseDir, CHECKPOINT_FILE_NAME)); | ||
| checkpointFileCache = new HashMap<>(); |
There was a problem hiding this comment.
It took me a really long time to decipher the actual purposes of "checkpoint" and "checkpointableOffsets". I've renamed them to "checkpointFile" and "checkpointFileCache" to be more self-documenting.
| try { | ||
| entry.getValue().get().close(); | ||
| } catch (final Exception e) { | ||
| } catch (final RuntimeException e) { |
There was a problem hiding this comment.
Since this PR is to clean up difficult-to-maintain code, I also included other cleanups, like dropping unnecessary this modifiers, restricting too-broad catch blocks, etc.
| .append(entry.getKey()) | ||
| .append(". Reason: ") | ||
| .append(e.toString()) | ||
| .append(e) |
There was a problem hiding this comment.
unnecessary toString
|
|
||
| // TODO: this map does not work with customized grouper where multiple partitions | ||
| // of the same topic can be assigned to the same topic. | ||
| // of the same topic can be assigned to the same task. |
There was a problem hiding this comment.
pretty sure this was a typo
| import static org.apache.kafka.streams.state.internals.WrappedStateStore.isTimestamped; | ||
|
|
||
| abstract class AbstractStateManager implements StateManager { | ||
| final class StateManagerUtil { |
There was a problem hiding this comment.
This has changed from an abstract class to a static utility class.
| throw new ProcessorStateException(String.format("%sError while deleting the checkpoint file", logPrefix), e); | ||
| } | ||
| try { | ||
| stateMgr.clearCheckpoints(); |
There was a problem hiding this comment.
checking for null is now encapsulated.
| false, | ||
| stateDirectory, | ||
| emptyMap(), | ||
| singletonMap(persistentStoreName, persistentStorePartition.topic()), |
There was a problem hiding this comment.
This test erroneously didn't include a changelog topic for the store in question. Now that we are actually verifying the checkpoints before we write them, we have to get this right.
There was a problem hiding this comment.
Edit: some refactoring I did actually removed this enforcement. I'm working out how to keep it, but it's pretty complicated...
| false, | ||
| stateDirectory, | ||
| emptyMap(), | ||
| singletonMap(persistentStoreName, persistentStoreTopicName), |
| } | ||
|
|
||
| private StreamTask createStatefulTask(final StreamsConfig config, final boolean logged) { | ||
| final StateStore stateStore = new MockKeyValueStore(storeName, logged); |
There was a problem hiding this comment.
We're no longer overlooking the fact that this store wasn't "logged" when we write the checkpoints.
vvcephei
left a comment
There was a problem hiding this comment.
@mjsax @cadonna @ableegoldman @abbccdda @guozhangwang @pkleindl ... did I miss anyone?
This PR should fix the long-running KAFKA-5998 bug.
The change I'm proposing is bigger in scope than the actual fix, though, because I wanted to take steps to prevent a similar bug from cropping up in the same code in the future. My theory is that the bug had an easy time hiding in this code because the handling of checkpointable offsets was so complex. I'm hoping that by reducing the mutable scope and also tightening up the invariants around the checkpointable offsets, we will have an easier time maintaining this module.
Let me know what you think!
Thanks,
-John
|
|
||
|
|
||
| public class ProcessorStateManager extends AbstractStateManager { | ||
| public class ProcessorStateManager implements StateManager { |
There was a problem hiding this comment.
Also here, no longer sharing mutable state between super and sub classes.
| private final File baseDir; | ||
| private OffsetCheckpoint checkpointFile; | ||
| private final Map<TopicPartition, Long> checkpointFileCache = new HashMap<>(); | ||
| private final Map<TopicPartition, Long> initialLoadedCheckpoints; |
There was a problem hiding this comment.
Adding this collection breaks a circular dependency in this class:
- the checkpoints we load from disk are potentially not valid for the current topology
- we have to load the checkpoints immediately because we have to delete the checkpoint file before processing in the case of EOS
- we also need to have read the checkpoint file before registering stores, since it might be needed to create a restorer
- we can't know if a checkpoint from the file is valid until after registering stores
In other words, if the prior code wanted to validate the loaded checkpoints, it would have to register the stores before loading checkpoints, but it also needs to load the checkpoints before registering the stores.
We're breaking the cycle here by keeping the loaded checkpoints separate. Now we read the checkpoint file into initialLoadedCheckpoints, which is used to register the stores, and then we are able to make sure that we only ever write valid checkpoints into the checkpointFileCache, which is used to update the checkpoint file later on.
| recordConverters.put(topic, recordConverter); | ||
| } else { | ||
| log.trace("Restoring state store {} from changelog topic {} at checkpoint {}", storeName, topic, checkpointableOffsets.get(storePartition)); | ||
| final Long restoreCheckpoint = store.persistent() ? initialLoadedCheckpoints.get(storePartition) : null; |
There was a problem hiding this comment.
This is where we're using the loaded checkpoint for store registration. Note the missing condition which is now handled... if the store is not persistent, it should not use the loaded checkpoint.
| ); | ||
| } | ||
|
|
||
| void clearCheckpoints() throws IOException { |
There was a problem hiding this comment.
encapsulating this operation so that outside classes don't have to directly mutate our checkpointFile field.
| checkpointFile.delete(); | ||
| checkpointFile = null; | ||
|
|
||
| checkpointFileCache.clear(); |
There was a problem hiding this comment.
We didn't previously clear the cache on the blocks that this method replaces, but after reading the code, I'm pretty sure this is the right thing to do.
| fileOutputStream.getFD().sync(); | ||
| } | ||
|
|
||
| LOG.trace("Swapping tmp checkpoint file {} {}", temp.toPath(), file.toPath()); |
There was a problem hiding this comment.
Having these logs would have demystified a large part of the prior (misdirected) investigation, since we were never sure whether the tmp file existed or not, or what was going on.
| return restoredOffsets; | ||
| } | ||
|
|
||
| void setRestoredOffsets(final Map<TopicPartition, Long> restoredOffsets) { |
There was a problem hiding this comment.
Added to support some of the needed test changes.
| public void testChangeLogOffsets() throws IOException { | ||
| final TaskId taskId = new TaskId(0, 0); | ||
| final long lastCheckpointedOffset = 10L; | ||
| final long storeTopic1LoadedCheckpoint = 10L; |
There was a problem hiding this comment.
renamed for clarity.
| } | ||
|
|
||
| @Test | ||
| public void shouldIgnoreIrrlevantLoadedCheckpoints() throws IOException { |
There was a problem hiding this comment.
Added a bunch of new tests to cover both the bug itself and also previously untested code paths in the ProcessorStateManager.
There was a problem hiding this comment.
Note: this particular test shows that we will actually repair all the corrupted checkpoint files that buggy Streams versions wrote.
| } | ||
|
|
||
| @Test | ||
| public void shouldIgnoreIrrelevantRestoredCheckpoints() throws IOException { |
There was a problem hiding this comment.
Note: this particular test verifies that the bug is fixed.
cadonna
left a comment
There was a problem hiding this comment.
Thanks for the PR @vvcephei and congratulations to you and @pkleindl for finding and fixing this bug. I left a couple of comments. I had hard time to find the code that actually fixes the bug (and I am still not sure if I found it). Could you please add some specific comments about the fix, next time, since this fix is not that trivial? I am also wondering if you could have divided this PR in two: one for the fix itself and one for the repair of old corrupted checkpoints. IMO, it would have made reviewing the PRs easier.
|
|
||
|
|
||
| public class ProcessorStateManager extends AbstractStateManager { | ||
| public class ProcessorStateManager implements StateManager { |
There was a problem hiding this comment.
Wouldn't it be more meaningful to rename this class to TaskStateManager?
There was a problem hiding this comment.
Maybe, I'm not sure of the historical reason to name it this way.
| } | ||
|
|
||
| private void updateCheckpointFileCache(final Map<TopicPartition, Long> checkpointableOffsetsFromProcessing) { | ||
| final Map<TopicPartition, Long> restoredOffsets = validCheckpointableOffsets(changelogReader.restoredOffsets()); |
There was a problem hiding this comment.
This is the most important line to fix the bug, right?
There was a problem hiding this comment.
Yes. I'd marked it in an earlier version of this PR. I guess that comment became "outdated" at some point. Sorry about that.
| storePartition, | ||
| new CompositeRestoreListener(stateRestoreCallback), | ||
| checkpointableOffsets.get(storePartition), | ||
| restoreCheckpoint, |
There was a problem hiding this comment.
If the store is not peristent or the read checkpoint file does not contain the partition, this will throw a NPE, right? If yes, you should add unit tests for these cases.
There was a problem hiding this comment.
No, I guess you were thinking the Long would become unboxed at this point? It's actually a Long parameter, and the StateRestorer constructor checks for null... Not the cleanest code, I guess, but it looks like it's been this way since 2017.
There was a problem hiding this comment.
My fault! I missed the parameter. I looked at the next parameter in the StateRestorer constructor which is a long.
|
Thanks for the review, @cadonna ! I'm sorry that my earlier call-out of the actual bugfix got marked "outdated" at some point, so I guess it doesn't show up in the diff anymore. I didn't notice when that happened, or I would have re-marked it. Regarding splitting up the PRs, I do agree with you. It would have been nice to get a smaller fix in, and then tackled the refactoring separately. If I can make one excuse for myself, it would be that in this case, it wasn't clear to me that the fix was good enough because the scope of In retrospect, though, I could have submitted the refactor first, and then followed up with the bugfix. It just didn't occur to me, for whatever reason. In any case, thanks for wading through the code review! I think I addressed all your comments. WDYT? |
|
retest this please |
|
Merged #7030 into trunk |
fix checkpoint file warning by filtering checkpointable offsets per task clean up state manager hierarchy to prevent similar bugs Reviewers: Bruno Cadonna <bruno@confluent.io>, Bill Bejeck <bbejeck@gmail.com>
fix checkpoint file warning by filtering checkpointable offsets per task clean up state manager hierarchy to prevent similar bugs Reviewers: Bruno Cadonna <bruno@confluent.io>, Bill Bejeck <bbejeck@gmail.com>
|
cherry-picked to 2.3 and 2.2 |
|
@bbejeck it means that the fix is available in 2.2.x version? |
|
@Tin-Nguyen yes if check out the |
|
thanks @bbejeck |
|
@bbejeck I'm wondering if we have an updated binary download includes the fix? |
| this.eosEnabled = eosEnabled; | ||
| this.checkpoint = new OffsetCheckpoint(new File(baseDir, CHECKPOINT_FILE_NAME)); | ||
| } | ||
| private StateManagerUtil() {} |
There was a problem hiding this comment.
Do we need this constructor explicitly? Would this just be default in java?
| storeToChangelogTopic, | ||
| partitions, | ||
| processorContext); | ||
| StateManagerUtil.reinitializeStateStoresForPartitions(log, |
There was a problem hiding this comment.
I liked this refactoring a lot, thanks @vvcephei !
* apache-github/2.3: MINOR: Update documentation for enabling optimizations (apache#7099) MINOR: Remove stale streams producer retry default docs. (apache#6844) KAFKA-8635; Skip client poll in Sender loop when no request is sent (apache#7085) KAFKA-8615: Change to track partition time breaks TimestampExtractor (apache#7054) KAFKA-8670; Fix exception for kafka-topics.sh --describe without --topic mentioned (apache#7094) KAFKA-8602: Separate PR for 2.3 branch (apache#7092) KAFKA-8530; Check for topic authorization errors in OffsetFetch response (apache#6928) KAFKA-8662; Fix producer metadata error handling and consumer manual assignment (apache#7086) KAFKA-8637: WriteBatch objects leak off-heap memory (apache#7050) KAFKA-8620: fix NPE due to race condition during shutdown while rebalancing (apache#7021) HOT FIX: close RocksDB objects in correct order (apache#7076) KAFKA-7157: Fix handling of nulls in TimestampConverter (apache#7070) KAFKA-6605: Fix NPE in Flatten when optional Struct is null (apache#5705) Fixes apache#8198 KStreams testing docs use non-existent method pipe (apache#6678) KAFKA-5998: fix checkpointableOffsets handling (apache#7030) KAFKA-8653; Default rebalance timeout to session timeout for JoinGroup v0 (apache#7072) KAFKA-8591; WorkerConfigTransformer NPE on connector configuration reloading (apache#6991) MINOR: add upgrade text (apache#7013) Bump version to 2.3.1-SNAPSHOT
…ling (apache#7030) TICKET = KAFKA-5998 LI_DESCRIPTION = EXIT_CRITERIA = HASH [1052d87] ORIGINAL_DESCRIPTION = fix checkpoint file warning by filtering checkpointable offsets per task clean up state manager hierarchy to prevent similar bugs Reviewers: Bruno Cadonna <bruno@confluent.io>, Bill Bejeck <bbejeck@gmail.com> (cherry picked from commit 1052d87)
Committer Checklist (excluded from commit message)