Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-36379] Improve (Global)Committer with UC disabled [1.20] #25660

Open
wants to merge 4 commits into
base: release-1.20
Choose a base branch
from

Conversation

AHeise
Copy link
Contributor

@AHeise AHeise commented Nov 15, 2024

Backport of #25456 with some adjustments in GlobalCommitterOperater (main doesn't have GlobalCommitter code anymore).

@flinkbot
Copy link
Collaborator

flinkbot commented Nov 15, 2024

CI report:

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

Use more of the assertj native patterns to compare results.
AHeise and others added 2 commits November 15, 2024 17:40
Global committers used to trail a full checkpoint behind committer. That means that data appeared only after >2*checkpoint interval in the sinks that use it (e.g. delta). However, committables in the global committers are already part of the first checkpoint and are idempotent: On recovery, they are resend from the committer to the global committer. Thus, the global committer can actually be seen as stateless and doesn't need to conduct its own 2PC protocol.

This commit lets the global committer collect all committables on input (as before) but immediately tries to commit when it has received all (deducible from CommitterSummary - which was always the original intent of that message). Thus, in most cases, GlobalCommitter ignores notifyCheckpointCompleted now as the state of the checkpoint can be inferred by received committables from upstream.

There are special cases where a global committer is directly chained to a writer. In this case, the global committer does need to conduct a 2PC protocol in place of the committer. To differentiate these cases, the global committer now has its own transformation.

(cherry picked from commit 67be29a)
Without UCs, a committer doesn't need to do anything on #processInput except collecting. It emits only on notifyCheckpointCompleted (or endInput for batch).

We can also harden some contracts:
* NotifyCheckpointCompleted can assert that all committables are received.
* Emit committables downstream only if all committables are finished.

(cherry picked from commit 7f40ab9)
@davidradl
Copy link

Reviewed by Chi on 21/11/24. Looks in hand, test failures currently

}

private static <T> void copySafely(Consumer<T> consumer, Supplier<T> provider) {
T value = provider.get();

Choose a reason for hiding this comment

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

I am wondering

  • are name, uid or uidHash nullable? The uid seems like a mandatory non nullable identifier.
  • should we error if any of these values are null?
  • or is there a valid case for having any of these values as null, if so should we not copy over the null?

return translateInternal(transformation, false);
}

private Collection<Integer> translateInternal(

Choose a reason for hiding this comment

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

should we add junits for this method - or do you think the higher level tests already cover us?

@@ -181,7 +181,7 @@ private void commitAndEmitCheckpoints() throws IOException, InterruptedException
private void commitAndEmit(CheckpointCommittableManager<CommT> committableManager)
throws IOException, InterruptedException {
Collection<CommittableWithLineage<CommT>> committed = committableManager.commit(committer);

Choose a reason for hiding this comment

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

NIT: as we are no longer using committed in the if, we could move committableManager.commit(committer) inlined to line 189 and remove the committed variable

/**
* Commits all due committables if all respective committables of the specific subtask and
* checkpoint have been received.
*
* @param committer used to commit to the external system
* @return successfully committed committables with meta information
* @throws IOException

Choose a reason for hiding this comment

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

why are these removed, we still throw these exceptions.

@@ -160,13 +169,37 @@ public Collection<CommittableWithLineage<CommT>> commit(Committer<CommT> committ
return committed;
}

Collection<CommitRequestImpl<CommT>> getPendingRequests(boolean onlyIfFullyReceived) {
Collection<CommitRequestImpl<CommT>> getPendingRequests(boolean assertFull) {

Choose a reason for hiding this comment

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

could the junits cover all the permutations of this stream - so we know there are no side effects?

if (transformation instanceof OneInputTransformation) {
StreamOperatorFactory<?> operatorFactory =
((OneInputTransformation<?, ?>) transformation).getOperatorFactory();
if (operatorFactory instanceof CommitterOperatorFactory) {

Choose a reason for hiding this comment

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

I think a comment to describe why we return true here would be useful.

private <R> void repeatUntilConverged(Supplier<R> producer) {
R lastResult = producer.get();
R nextResult;
while (!lastResult.equals(nextResult = producer.get())) {

Choose a reason for hiding this comment

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

can we get nulls here?

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

Successfully merging this pull request may close these issues.

3 participants