[KAFKA-6730] Simplify State Store Recovery#5013
[KAFKA-6730] Simplify State Store Recovery#5013mjsax merged 11 commits intoapache:trunkfrom ConcurrencyPractitioner:KAFKA-6730
Conversation
|
@mjsax One last push. :) |
mjsax
left a comment
There was a problem hiding this comment.
Thanks for the update. We are getting close :)
I think, we can still improve updating the end-offsets. Let's see what Guozhang thinks.
| assertThat(callbackTwo.restored.size(), equalTo(3)); | ||
| } | ||
|
|
||
| @Ignore |
There was a problem hiding this comment.
I think we can remove the ignored tests completely. \cc @bbejeck
|
|
||
| final Set<TopicPartition> restoringPartitions = new HashSet<>(needsRestoring.keySet()); | ||
| try { | ||
| final ConsumerRecords<byte[], byte[]> allRecords = poll(restoreConsumer, 10); |
There was a problem hiding this comment.
Line 73 above: we can remove the whole JavaDoc comment because this method does not throw TaskMigratedException any longer.
| final ConsumerRecords<byte[], byte[]> allRecords = poll(restoreConsumer, 10); | ||
| for (final TopicPartition partition : restoringPartitions) { | ||
| restorePartition(allRecords, partition, active.restoringTaskFor(partition)); | ||
| if (!needsRestoring.isEmpty()) { |
There was a problem hiding this comment.
IMHO, we should add a second check to make sure updatedEndOffsets is set only once:
if (!needsRestoring.isEmpty() && updatedEndOffsets == null) {
Additionally, we should set updatedEndOffsets = null in Line 117 below (wondering if this is good enough, or if we need to reset updatedEndOffsets = null somewhere else, too. If we are in the middle of restore, and a rebalance happens, we get new partitions and need to refresh the corresponding endOffsets... \cc @guozhangwang WDYT?
There was a problem hiding this comment.
Hi @mjsax . When I added the check as you suggested, I got the following:
org.apache.kafka.streams.processor.internals.StoreChangelogReaderTest > shouldRestorePartitionsRegisteredPostInitialization FAILED
java.lang.NullPointerException
at org.apache.kafka.streams.processor.internals.StoreChangelogReader.processNext(StoreChangelogReader.java:266)
at org.apache.kafka.streams.processor.internals.StoreChangelogReader.restore(StoreChangelogReader.java:92)
at org.apache.kafka.streams.processor.internals.StoreChangelogReaderTest.shouldRestorePartitionsRegisteredPostInitialization(StoreChangelogReaderTest.java:374).
Do you think you know the cause for this failure? I suggested something that might have been the cause in #4901, although that might not precisely be the source of the error.
|
I have found that when you remove the check ( |
|
@ConcurrencyPractitioner Could you provide a more detailed reasoning of why the end offsets could be changed after we fetch it at the beginning of the restoration? The goal of https://issues.apache.org/jira/browse/KAFKA-6730 itself is that, since now we call |
|
@guozhangwang In the test, I found that end offsets were updated twice for what appears to be two distinct topic partitions. In pure honesty, I do not know myseIf why tests are faiIing. Do you have any ideas? |
|
One thing: we need to set Not sure if this fixed the error. |
|
@ConcurrencyPractitioner I had a look into the test, and the issue is the test setup itself. The mock-consumer setup is split into two parts: and later Thus, when restore is called the first time, the mock consumer does not know about the other partition -- the real consumer however would know about the partition -- thus, we need to change the test setup such that the mocked consumer has the information about end-offsets for all partitions from the beginning on. |
|
@mjsax Correct me if I am wrong, but here is my current idea of what is happening. When the first restore is called, |
|
Note that with the solution I am proposing, it would allow mock consumer to know about partitions that it had missed. |
|
@ConcurrencyPractitioner You observation sound correct to me. It think, we need to update the endOffsets each time when there is some new partitions to be initilized: Does this make sense? |
|
Hi @mjsax any comments? |
mjsax
left a comment
There was a problem hiding this comment.
Build fails with compile error. Can you please rebase this PR?
| if (!needsRestoring.isEmpty()) { | ||
| final Set<TopicPartition> remainingPartitions = new HashSet<>(needsRestoring.keySet()); | ||
| remainingPartitions.removeAll(updatedEndOffsets.keySet()); | ||
| updatedEndOffsets.putAll(restoreConsumer.endOffsets(remainingPartitions)); |
There was a problem hiding this comment.
I think we can move this into the block above line 70 to 72 and simplify:
if (!needsInitializing.isEmpty()) {
initialize();
updatedEndOffsets.putAll(restoreConsumer.endOffsets(needsInitializing.keySet()));
}
| final StateRestorer restorer = stateRestorers.get(partition); | ||
| final long pos = processNext(records.records(partition), restorer, updatedEndOffsets.get(partition)); | ||
| restorer.setRestoredOffset(pos); | ||
| if (restorer.hasCompleted(pos, updatedEndOffsets.get(partition))) { |
There was a problem hiding this comment.
we can also remove the entry in updatedEndOffsets if this is true.
|
Hi @mjsax , it appears that needsInitializing is distinct from needsRestoring. When I tested your approach, it didn't work because the topic partitions in needsInitializing are different from the ones that we actually need to restore. |
|
Only needsRestoring contains the needed partitions, not needsInitializing. |
|
I see. Guess we need to update the end-offsets before the call to |
|
Hi @mjsax, when I updated the end offsets before As you can see, some partitions found in needsInitializing does not have an endOffsets, probably because they have not even been assigned one yet. |
|
Hi @mjsax any other comments that we would need to address? |
|
@mjsax If you feel you have time, could you tell me exactly what this PR is lacking? |
|
@ConcurrencyPractitioner Feature freeze deadline does not affect this PR, because it's an internal change. Code freeze deadline is in 2 weeks, so we still got time. Reviewing now though :) |
mjsax
left a comment
There was a problem hiding this comment.
Call for second review @guozhangwang @bbejeck @vvcephei
| @@ -81,10 +78,25 @@ public Collection<TopicPartition> restore(final RestoringTasks active) { | |||
|
|
|||
| final Set<TopicPartition> restoringPartitions = new HashSet<>(needsRestoring.keySet()); | |||
There was a problem hiding this comment.
I think we can remove this variable to simplify the code further.
| remainingPartitions.removeAll(updatedEndOffsets.keySet()); | ||
| updatedEndOffsets.putAll(restoreConsumer.endOffsets(remainingPartitions)); | ||
| final ConsumerRecords<byte[], byte[]> records = restoreConsumer.poll(10); | ||
| final Iterator<TopicPartition> iterator = restoringPartitions.iterator(); |
There was a problem hiding this comment.
replace restoringPartitions with needsRestoring.keySet() to get rid of the unnecessary variable.
| restorePartition(allRecords, partition, active.restoringTaskFor(partition)); | ||
| final Set<TopicPartition> remainingPartitions = new HashSet<>(needsRestoring.keySet()); | ||
| remainingPartitions.removeAll(updatedEndOffsets.keySet()); | ||
| updatedEndOffsets.putAll(restoreConsumer.endOffsets(remainingPartitions)); |
There was a problem hiding this comment.
This seems to be correct. Still wondering why you did not move this code into
if (!needsInitializing.isEmpty()) {
initialize();
// put the three lines here
}
I think that remainingPartitions can only contain data if !needsInitializing.isEmpty() is true -- thus, no need to execute the code in each iteration. Or do I miss anything?
There was a problem hiding this comment.
Sorry about my delay. It should be fine now.
There was a problem hiding this comment.
@mjsax Actually, needsInitializing can still be empty, but remainingPartitions would still have partitions which needs to be added from needsRestoring. I think the main reason is that needsInitializing and needsRestoring are completely independent of one another. If one does not contain partitions, that does not neccesarily effect the other. In other words, as long as needsRestoring has partitions which needs to be restored, we will probably need to check for partitions every time -- regardless of what needsInitializing contains.
There was a problem hiding this comment.
I cannot follow. Not sure what I am missing. From my understanding the flow is as follows: (1) after new partitions are assigned, they are added to needsInitializing. (2) If partitions/task are initialized, it's check if the need restoring: (2a) if they need restoring they are removed from needsInitializing and added to needsRestoring (2b) if they don't need restoring, they are only removed from needsInitializing.
We need to maintain updatedEndOffsets only for partitions that needs restoring. And we need to get the endOffset for those partitions only once. Thus, new end-offsets only need to be added, after new partitions are assigned and thus, needsInitializing.isEmpty() is false.
Actually, needsInitializing can still be empty, but remainingPartitions would still have partitions which needs to be added from needsRestoring
Why? Those partitions should have been added to updatedEndOffsets after they got assigned initially.
I think the main reason is that needsInitializing and needsRestoring are completely independent of one another.
From my understanding, they are not independent. Each partitions is first in needsInitializing and might be "moved" from there to needsRestoring.
In other words, as long as needsRestoring has partitions which needs to be restored, we will probably need to check for partitions every time -- regardless of what needsInitializing contains
Why? The end-offset should not change and thus, for each partition that is moved from needsInitializing to needsRestoring we only add the end-offset once to updatedEndOffsets
Please let me know what I miss.
There was a problem hiding this comment.
Oh, I see what happened. Sorry, this was my bad. What happened was that I was adding the end offsets to updatedEndOffsets before initialize() in the conditional was called. Consequently, I never detected any change in needsRestoring.
|
HI @mjsax I think this should be all of it. |
mjsax
left a comment
There was a problem hiding this comment.
Thanks for the update. One more comment.
| if (restorer.hasCompleted(pos, updatedEndOffsets.get(partition))) { | ||
| restorer.restoreDone(); | ||
| updatedEndOffsets.remove(partition); | ||
| completedPartitions.add(partition); |
There was a problem hiding this comment.
One more thing -- missed this before: can't we remove completedPartitions and call iterator.remove() instead?
Maybe we need to change the iterator to iterate over the HashMap instead of the "key-set" of the HashMap though.
|
retest this please |
guozhangwang
left a comment
There was a problem hiding this comment.
Made my pass, other than @mjsax 's comment it lgtm.
|
@mjsax Sorry about the delay. Fixed now. |
|
retest this please |
mjsax
left a comment
There was a problem hiding this comment.
LGTM. Thanks for the patch and for being patient!
…grained-acl-create-topics * apache-github/trunk: KAFKA-5588: Remove deprecated --new-consumer tools option (apache#5097) MINOR: Fix for the location of the trogdor.sh executable file in the documentation. (apache#5040) KAFKA-6997: Exclude test-sources.jar when $INCLUDE_TEST_JARS is FALSE MINOR: docs should point to latest version (apache#5132) KAFKA-6981: Move the error handling configuration properties into the ConnectorConfig and SinkConnectorConfig classes (KIP-298) [KAFKA-6730] Simplify State Store Recovery (apache#5013) MINOR: Rename package `internal` to `internals` for consistency (apache#5137) KAFKA-6704: InvalidStateStoreException from IQ when StreamThread closes store (apache#4801) MINOR: Add missing configs for resilience settings MINOR: Add regression tests for KTable mapValues and filter (apache#5134) KAFKA-6750: Add listener name to authentication context (KIP-282) (apache#4829) KAFKA-3665: Enable TLS hostname verification by default (KIP-294) (apache#4956) KAFKA-6938: Add documentation for accessing Headers on Kafka Streams Processor API (apache#5128) KAFKA-6813: return to double-counting for count topology names (apache#5075) KAFKA-5919; Adding checks on "version" field for tools using it MINOR: Remove deprecated KafkaStreams constructors in docs (apache#5118)
…refix * apache-github/trunk: KAFKA-6726: Fine Grained ACL for CreateTopics (KIP-277) (apache#4795) KAFKA-5588: Remove deprecated --new-consumer tools option (apache#5097) MINOR: Fix for the location of the trogdor.sh executable file in the documentation. (apache#5040) KAFKA-6997: Exclude test-sources.jar when $INCLUDE_TEST_JARS is FALSE MINOR: docs should point to latest version (apache#5132) KAFKA-6981: Move the error handling configuration properties into the ConnectorConfig and SinkConnectorConfig classes (KIP-298) [KAFKA-6730] Simplify State Store Recovery (apache#5013) MINOR: Rename package `internal` to `internals` for consistency (apache#5137) KAFKA-6704: InvalidStateStoreException from IQ when StreamThread closes store (apache#4801) MINOR: Add missing configs for resilience settings MINOR: Add regression tests for KTable mapValues and filter (apache#5134) KAFKA-6750: Add listener name to authentication context (KIP-282) (apache#4829) KAFKA-3665: Enable TLS hostname verification by default (KIP-294) (apache#4956) KAFKA-6938: Add documentation for accessing Headers on Kafka Streams Processor API (apache#5128) KAFKA-6813: return to double-counting for count topology names (apache#5075) KAFKA-5919; Adding checks on "version" field for tools using it MINOR: Remove deprecated KafkaStreams constructors in docs (apache#5118)
Reviewer: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <guozhang@confluent.io>, Bill Bejeck <bill@confluent.io>
No description provided.