Skip to content

[KAFKA-7994] Improve Stream time accuracy for restarts and rebalances #6694

Merged
mjsax merged 80 commits intoapache:trunkfrom
ConcurrencyPractitioner:kafka-7994
Sep 19, 2019
Merged

[KAFKA-7994] Improve Stream time accuracy for restarts and rebalances #6694
mjsax merged 80 commits intoapache:trunkfrom
ConcurrencyPractitioner:kafka-7994

Conversation

@ConcurrencyPractitioner
Copy link
Copy Markdown
Contributor

@ConcurrencyPractitioner ConcurrencyPractitioner commented May 7, 2019

The issue for this PR could be found here:
https://issues.apache.org/jira/browse/KAFKA-7994?jql=project%20%3D%20KAFKA%20AND%20status%20in%20(Open%2C%20%22In%20Progress%22%2C%20Reopened%2C%20%22Patch%20Available%22)

As noted in the JIRA description, stream time is incorrectly set to -1 after rebalances and restarts. To help resolve this issue, one approach is to commit the individual partition time with last message processed for each RecordQueue. Hence, after a restart, we could set the partition time to the last committed partition time.

We would also have to forward timestamps down stream after the head of the DAG receives records that updates stream time and global time.

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

@ConcurrencyPractitioner
Copy link
Copy Markdown
Contributor Author

ping @mjsax @guozhangwang @vvcephei for review

@SuppressWarnings("unchecked")
public boolean process() {
// if condition put here in case of restarts and rebalances to check for correct timestamp
if (recordInfo.queue() != null && partitionGroup.getPartitionTimestamp(recordInfo.partition()) == -1) {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: use RecordQueue.UNKNOWN instead of -1

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No problem, could fix that.

@ableegoldman
Copy link
Copy Markdown
Member

Thanks for the PR! I agree this seems like a straightforward patch but I'm wondering if we shouldn't try and think through the eos case a bit more? Or is there really no way to safely cover it as well?

@ConcurrencyPractitioner
Copy link
Copy Markdown
Contributor Author

ConcurrencyPractitioner commented May 8, 2019

Hi @ableegoldman Thanks for reviewing! I was planning on attacking the eos case. But as you could guess from the code, trying to retrieve the metadata committed is not as simple as in the non eos case. I was hoping for some input on that. So some small amount of advice is greatly appreciated. :)

@mjsax mjsax added the streams label May 8, 2019
@ConcurrencyPractitioner
Copy link
Copy Markdown
Contributor Author

Oh, just found out something. Regardless if it is eos case or not, calling KafkaConsumer#committed() should be sufficient. So there shouldn't be any big problems with this. I will try to add a test case. :)

@ableegoldman
Copy link
Copy Markdown
Member

That sounds reasonable. Looks like the build failed on checkstyle, can you try running it?

+1 on adding a test case(s)


// confirm that timestamp was correctly committed
assertTrue(Long.parseLong(task.consumer.committed(partition1).metadata())
== task.getPartitionTime(partition1));
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we add separate unit tests to confirm this produces the expected behavior? I think the JIRA had some examples highlighting why this is a problem, it would be good to convert those into tests to make sure we're really fixing the problem at hand :)

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No problem. Added a new test case to confirm behavior.

@ConcurrencyPractitioner
Copy link
Copy Markdown
Contributor Author

Alright, done. @mjsax Added a test case as well. Would be good if you could take a look. :)

@ConcurrencyPractitioner
Copy link
Copy Markdown
Contributor Author

pinging @mjsax and @guozhangwang for review

@ConcurrencyPractitioner
Copy link
Copy Markdown
Contributor Author

Oh sorry, my bad. Underestimated the scope of the PR. Sorry for pinging you guys. Will dig some more.

@ConcurrencyPractitioner
Copy link
Copy Markdown
Contributor Author

Retest this please.

@ConcurrencyPractitioner
Copy link
Copy Markdown
Contributor Author

pinging @mjsax @ableegoldman @abbccdda @guozhangwang for final review.

Copy link
Copy Markdown
Member

@cadonna cadonna left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you for the PR @ConcurrencyPractitioner!
Here my comments:

import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.cleanStateBeforeTest;
import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.getStartedStreams;
import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.MatcherAssert.assertThat;
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The order of the imports in Kafka Streams is usually as follows:

Kafka imports and 3rd-party imports in one block

a block of java.* imports

import static.

task.commit();
assertEquals(DEFAULT_TIMESTAMP, task.decodeTimestamp(consumer.committed(partition1).metadata()));
// reset times here to artificially represent a restart
task.resetTimes();
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Wouldn't creating a new task be better? AFAIK, that is what happens during a restart. No need to simulate anything. Furthermore, it avoids introducing a new method just for testing.

// extract the committed metadata from MockProducer
final List<Map<String, Map<TopicPartition, OffsetAndMetadata>>> metadataList =
producer.consumerGroupOffsetsHistory();
final String storedMetadata = metadataList.get(0).get("stream-task-test").get(partition1).metadata();
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Would be good to extract "stream-task-test" to a member field of the test and use it in createConfig() and here.

consumer.commitSync(offsetMap);

// reset times here to artificially represent a restart
task.resetTimes();
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

see above

}

// visible for testing
String encodeTimestamp(final long partitionTime) {
Copy link
Copy Markdown
Member

@cadonna cadonna Aug 15, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would put methods to write and read record metadata in their own classes. Those classes would be kind of SerDes for metadata. Such SerDes would make the code better testable and separates the concerns of a task and reading and writing metadata which are completely independent. It does not need to be done in this PR. I just wanted to mention it.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, that would probably be a good idea in the future.

EasyMock.expect(t2.partitions()).andReturn(t2partitions);
EasyMock.expect(t2.changelogPartitions()).andReturn(Collections.emptyList());

t1.initializeTaskTime();
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please remove empty line before this line.

assertThrows(errMessage, NullPointerException.class, () -> {
group.setPartitionTime(randomPartition, 0L);
});
}
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This tests misses to verify whether streamTime is set or not.

Furthermore, I would write two (or three) distinct tests:

  • partitionTimestamp is set (could be further split for streamTime is set or not)
  • NullPointerException is thrown

task.addRecords(partition1, singletonList(getConsumerRecord(partition1, DEFAULT_TIMESTAMP)));

task.process();
task.commit();
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The code block from the beginning of the method until here can be extracted and re-used in this and the previous test methods.

}

@Test
public void testSetPartitionTimestamp() {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think, we use should... for newly added test methods.

@ConcurrencyPractitioner
Copy link
Copy Markdown
Contributor Author

@cadonna Alright, done.

@ConcurrencyPractitioner
Copy link
Copy Markdown
Contributor Author

@mjsax @abbccdda @guozhangwang @ableegoldman
Another round of comments would be good or approval of PR if everything looks right. :)

@marcospassos
Copy link
Copy Markdown

This issue is the cause of critical bugs we recently faced up in our applications that rely on the SessionStore for processing retroactive events.

@mjsax do you think this fix can be included as part of 2.3.1?

@ConcurrencyPractitioner
Copy link
Copy Markdown
Contributor Author

@mjsax @cadonna Any last comments?

@ConcurrencyPractitioner
Copy link
Copy Markdown
Contributor Author

@mjsax pinging.

@mjsax
Copy link
Copy Markdown
Member

mjsax commented Sep 5, 2019

@marcospassos I don't think that we will include it in 2.3.1 -- it's not really a bug fix but an improvement.

@ConcurrencyPractitioner I try to review again in the next days.

Copy link
Copy Markdown
Member

@cadonna cadonna left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@ConcurrencyPractitioner Sorry for the delay.

if (partitionQueues.get(partition) == null) {
throw new NullPointerException("Partition " + partition + " not found.");
}
return partitionQueues.get(partition).partitionTime();
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Here it would be better to call partitionQueues.get(partition) only once and store its result in a variable. Then check the variable for null and call partitionTime() on the variable.

if (streamTime < partitionTime) {
streamTime = partitionTime;
}
partitionQueues.get(partition).setPartitionTime(partitionTime);
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Same as above.

final ByteBuffer buffer = ByteBuffer.allocate(9);
buffer.put(LATEST_MAGIC_BYTE);
buffer.putLong(partitionTime);
return Base64.getEncoder().encodeToString(buffer.array());
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am wondering whether we can do better here. Encoding partition time in Base64 seems to me a bit a waste of space. As far as I can see, a 8 byte value is encoded in 11 bytes with Base64. Would be great, if we could store partition time in 8 bytes.
I am also wondering why metadata in OffsetAndMetadata is a String and not something more bytes friendly.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@cadonna Yeah, it is still unclear at this point if the metadata field in OffsetAndMetadata could be used in this manner. @guozhangwang or @hachikuji knows this matter better. Anyhow, OffsetAndMetadata right now is the only medium through which we can checkpoint partition time anyways. So we might be stuck with using the metadata field.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't have the full context on the history, but it would not be easy to change the API... I talked to Jason about it, and it seem we can just move forward with this PR as-is, and could do a KIP later that allows us to store metadata as byte[] type if we really need to change it. Atm, the metadata is just a few bytes and the overhead does not really matter IMHO.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Agreed

@ConcurrencyPractitioner
Copy link
Copy Markdown
Contributor Author

Retest this please.

*/
// visible for testing
void commit(final boolean startNewTransaction) {
void commit(final boolean startNewTransaction, final Map<TopicPartition, Long> partitionTimes) {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I know that I recommended to add this parameter, but now, after more refactoring of the code, I am not sure any longer why we need it? It seems that this method is called twice and both calls pass in the result of extractPartitionTimes() as parameter -- hence, it seems we can remove the parameter and do the call to extractPartitionTimes() within the method itself?

}

private void initializeCommittedTimestamp(final TopicPartition partition) {
final OffsetAndMetadata metadata = consumer.committed(partition);
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is a blocking call, and @guozhangwang just proposed KIP-520 to make it more efficient by allowing to pass in multiple partitions at once. Should we wait for KIP-520 to be implemented? If now, we should make sure the update this code after KIP-520 is merged.

I am also wondering how we should handle TimeoutException for this call? Maybe not, but might be worth to clarify?

\cc @guozhangwang

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In my PR (#7304) I've refactored this part in StreamTask. I'd suggest we merge that one before this.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just realized I need to do another rebase on my PR. So if this PR is closer to be merged I'd suggest @RichardYuSTUG @mjsax you guys just move forward and I will rebase mine later.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@mjsax Cool, sounds good. In that case, we could get this one merged since it is about complete.

final ByteBuffer buffer = ByteBuffer.allocate(9);
buffer.put(LATEST_MAGIC_BYTE);
buffer.putLong(partitionTime);
return Base64.getEncoder().encodeToString(buffer.array());
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't have the full context on the history, but it would not be easy to change the API... I talked to Jason about it, and it seem we can just move forward with this PR as-is, and could do a KIP later that allows us to store metadata as byte[] type if we really need to change it. Atm, the metadata is just a few bytes and the overhead does not really matter IMHO.

@cadonna
Copy link
Copy Markdown
Member

cadonna commented Sep 12, 2019

@ConcurrencyPractitioner All builds reported SpotBug issues.

@ConcurrencyPractitioner
Copy link
Copy Markdown
Contributor Author

Yeah, got it fixed.

@cadonna
Copy link
Copy Markdown
Member

cadonna commented Sep 13, 2019

The following test failures seem related:

org.apache.kafka.streams.integration.ResetPartitionTimeIntegrationTest.shouldPreservePartitionTimeOnKafkaStreamRestart[0: eosEnabled=false]
org.apache.kafka.streams.integration.ResetPartitionTimeIntegrationTest.shouldPreservePartitionTimeOnKafkaStreamRestart[1: eosEnabled=true]

@ConcurrencyPractitioner
Copy link
Copy Markdown
Contributor Author

@cadonna Oh, just realized that @mjsax's comment caused a regression. If you would look earlier in the conversation, you would find a segment where a call to close() resets all partition times to negative one. Therefore, we need to store the partition times in a map before they are reset, and then they are passed into the commit() method.

The extra parameter is needed after all due to the order of operations in close(). We will need to rollback some changes.

// visible for testing
void suspend(final boolean clean,
final boolean isZombie) {
final Map<TopicPartition, Long> partitionTimes = extractPartitionTimes();
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I remember now -- can we add a comment to explain that we need to get partitionTimes before we closeTopology() (sorry for my previous comment -- forgot about that)

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Cool, got it done.

Copy link
Copy Markdown
Member

@cadonna cadonna left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM, Thank you @ConcurrencyPractitioner

@mjsax mjsax merged commit 73c6bd8 into apache:trunk Sep 19, 2019
@mjsax
Copy link
Copy Markdown
Member

mjsax commented Sep 19, 2019

Thanks for the hard work @ConcurrencyPractitioner!

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

10 participants