Skip to content

[HUDI-8433] Fix not update issuedOffset when stream read empty commits#12166

Merged
danny0405 merged 5 commits intoapache:masterfrom
fhan688:fix-not-update-issuedinstant-when-stream-read-empty-commits
Nov 8, 2024
Merged

[HUDI-8433] Fix not update issuedOffset when stream read empty commits#12166
danny0405 merged 5 commits intoapache:masterfrom
fhan688:fix-not-update-issuedinstant-when-stream-read-empty-commits

Conversation

@fhan688
Copy link
Contributor

@fhan688 fhan688 commented Oct 25, 2024

Change Logs

When empty commits are allowed , if number of continuous empty commits exceeds 'read.commits.limit' , IssuedInstant will not be updated which will block stream read.

Impact

hudi-flink-datasource

Risk level (write none, low medium or high below)

low

Documentation Update

None

Contributor's checklist

  • Read through contributor's guide
  • Change Logs and Impact were stated clearly
  • Adequate tests were added if applicable
  • CI passed

@github-actions github-actions bot added the size:S PR with lines of changes in (10, 100] label Oct 25, 2024
IncrementalInputSplits.Result result =
incrementalInputSplits.inputSplits(metaClient, this.issuedOffset, this.cdcEnabled);
if (result.isEmpty()) {
if (result.equals(IncrementalInputSplits.Result.EMPTY)) {
Copy link
Contributor

Choose a reason for hiding this comment

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

should we always update the issuedInstant if it is not null?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

If value of read.speed.limit is smaller than number of empty instants , result.isEmpty is true and issuedInstant is not updated.

Copy link
Contributor

Choose a reason for hiding this comment

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

it is not updated but can still be used to reset the variable.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

In speed limit scenario, it can not be updated any more. I try to give an example to explain this:

assume current active timeline is instant1、instant2、instant3、instant4、instant5、instant6 ...
If the issuedInstant is instant2 and read.speed.limit=2 and instant3 and instant4 are empty commits, then the inputSplits method return Result.instance(inputSplits, endInstant, offsetToIssue).
In this return, inputSplits is Collections.emptyList() , endInstant is instant4, offsetToIssue is max(instant3 completionTime, instant4.completionTime).

So result.isEmpty() is true and monitorDirAndForwardSplits() returned, but offsetToIssue and issuedInstant are not updated. Next time when monitorDirAndForwardSplits() is called, the inputSplits method return the same as above, endless loop.
@danny0405

Copy link
Contributor

Choose a reason for hiding this comment

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

I meant just to use instant4.completionTime to update the issuedInstant in this case.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This bug is easy to be reproduced. Let us take UT TestStreamReadMonitoringFunction#testConsumeForSpeedLimitWhenEmptyCommitExists() as an example:
Step1: create 4 empty commit
Step2: trigger streaming read from first instant and set READ_COMMITS_LIMIT 2
Step3: assert current IssuedOffset couldn't be null.
Base on IncrementalInputSplits#inputSplits => .startCompletionTime(issuedOffset != null ? issuedOffset : this.conf.getString(FlinkOptions.READ_START_COMMIT))
If IssuedOffset still was null, hudi would take FlinkOptions.READ_START_COMMIT again, which means streaming read is blocked.

Without this PR this UT will be failed which means streaming reading empty commits will be blocked.

Copy link
Contributor

Choose a reason for hiding this comment

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

We only update the issuedOffset with non-null issuedInstant, does it make sense?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

make sense. PTAL

@fhan688 fhan688 changed the title [HUDI-8433] Fix not update IssuedInstant when stream read empty commits [HUDI-8433] Fix not update issuedOffset when stream read empty commits Oct 31, 2024
}

private Integer intervalBetween2Instants(HoodieTimeline timeline, String instant1, String instant2) {
Integer idxInstant1 = getInstantIdxInTimeline(timeline, instant1);
Copy link
Contributor

Choose a reason for hiding this comment

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

Can we just filter the timeline with proper range and count the instants instead?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

not called, just remove it.

// Step3: assert current IssuedOffset couldn't be null.
// Base on "IncrementalInputSplits#inputSplits => .startCompletionTime(issuedOffset != null ? issuedOffset : this.conf.getString(FlinkOptions.READ_START_COMMIT))"
// If IssuedOffset still was null, hudi would take FlinkOptions.READ_START_COMMIT again, which means streaming read is blocked.
assertTrue(function.getIssuedOffset() != null);
Copy link
Contributor

Choose a reason for hiding this comment

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

Use assertNotNull


if (result.isEmpty() && StringUtils.isNullOrEmpty(result.getEndInstant())) {
// no new instants, returns early
LOG.info("result is empty, do not update issuedInstant.");
Copy link
Contributor

Choose a reason for hiding this comment

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

result -> Result,and use log.warn instead, maybe we just eliminate the log message because it's almost useless.

@hudi-bot
Copy link
Collaborator

hudi-bot commented Nov 6, 2024

CI report:

Bot commands @hudi-bot supports the following commands:
  • @hudi-bot run azure re-run the last Azure build

@danny0405
Copy link
Contributor

The Azure failure is not related.

@fhan688
Copy link
Contributor Author

fhan688 commented Nov 7, 2024

The Azure failure is not related.

got it. thank you so much for Teacher Danny's advice and is there anything esle to revise?

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

Labels

size:S PR with lines of changes in (10, 100]

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants