Skip to content

MINOR: fix stream failing tests#13512

Merged
guozhangwang merged 2 commits intoapache:trunkfrom
showuon:fixStreamTests
Apr 6, 2023
Merged

MINOR: fix stream failing tests#13512
guozhangwang merged 2 commits intoapache:trunkfrom
showuon:fixStreamTests

Conversation

@showuon
Copy link
Copy Markdown
Member

@showuon showuon commented Apr 6, 2023

After #13300, there are a bunch of failing tests. Trying to fix them or workaround them to make the CI test healthy.

Committer Checklist (excluded from commit message)

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


if (numRecords != 0) {
final List<ConsumerRecord<byte[], byte[]>> records = changelogMetadata.bufferedRecords.subList(0, numRecords);
final List<ConsumerRecord<byte[], byte[]>> records = new ArrayList<>(changelogMetadata.bufferedRecords.subList(0, numRecords));
Copy link
Copy Markdown
Member Author

@showuon showuon Apr 6, 2023

Choose a reason for hiding this comment

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

We'll get concurrent modification for the list exception in L660 records.size() since there are also other threads updating buffer records. Fixing it by creating a clone of the list, and remove all from the bufferedRecords below. We can optimize it later.

}

if (numRecords > 0 || changelogMetadata.state().equals(ChangelogState.COMPLETED)) {
if (task != null && (numRecords > 0 || changelogMetadata.state().equals(ChangelogState.COMPLETED))) {
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

In https://github.com/apache/kafka/pull/13300/files#r1123945673 , we discussed if the null check is necessary, and then decided to drop it. But it turns out it's necessary. Otherwise, NPE will be thrown in some test cases.

Comment on lines -992 to -997
final StreamTask task = (StreamTask) tasks.get(taskId);
// if the log is truncated between when we get the log end offset and when we get the
// consumer position, then it's possible that the difference become negative and there's actually
// no records to restore; in this case we just initialize the sensor to zero
final long recordsToRestore = Math.max(changelogMetadata.restoreEndOffset - startOffset, 0L);
task.initRemainingRecordsToRestore(time, recordsToRestore);
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

With these lines, we got some errors related to "cannot cast the task to StreamTask". Might need to investigate why it will happen. Remove them first to fix the tests.


@Test
@Ignore
public void shouldGetDroppedRecordsSensor() {
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

@showuon
Copy link
Copy Markdown
Member Author

showuon commented Apr 6, 2023

@lucasbru @guozhangwang @mjsax , I can see there's still a related test failed:

org.apache.kafka.streams.processor.internals.ReadOnlyTaskTest.shouldThrowUnsupportedOperationExceptionForForbiddenMethods

But overall, it looks better now. I think we should merge this PR soon, and make further improvement/fix later. WDYT?

PS. I'm going to log off now. So I'll leave this PR to you! :)

@lucasbru
Copy link
Copy Markdown
Member

lucasbru commented Apr 6, 2023

LGTM, but we will need @mjsax or @guozhangwang to merge this

@lucasbru
Copy link
Copy Markdown
Member

lucasbru commented Apr 6, 2023

I couldn't change this PR, so I added a separate PR for the last failing test: #13519

@guozhangwang guozhangwang merged commit f02f5f8 into apache:trunk Apr 6, 2023
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

streams tests Test fixes (including flaky tests)

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants