Skip to content

Refactor commit logic out of StreamingDataflowWorker #30312

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

Merged
merged 22 commits into from
Mar 15, 2024

Conversation

m-trieu
Copy link
Contributor

@m-trieu m-trieu commented Feb 14, 2024

StreamingDataflowWorker is a rather large class

For maintainability it would be helpful in the long run to encapsulate different pieces of the worker harness logic into their own class.


Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:

  • Mention the appropriate issue in your description (for example: addresses #123), if applicable. This will automatically add a link to the pull request in the issue. If you would like the issue to automatically close on merging the pull request, comment fixes #<ISSUE NUMBER> instead.
  • Update CHANGES.md with noteworthy changes.
  • If this contribution is large, please file an Apache Individual Contributor License Agreement.

See the Contributor Guide for more tips on how to make review process smoother.

To check the build health, please visit https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md

GitHub Actions Tests Status (on master branch)

Build python source distribution and wheels
Python tests
Java tests
Go tests

See CI.md for more information about GitHub Actions CI or the workflows README to see a list of phrases to trigger workflows.

Copy link
Contributor

Checks are failing. Will not request review until checks are succeeding. If you'd like to override that behavior, comment assign set of reviewers

@m-trieu m-trieu force-pushed the mt-refactor-commits branch from 5260d57 to 4a70e02 Compare February 15, 2024 00:38
@m-trieu
Copy link
Contributor Author

m-trieu commented Feb 15, 2024

R: @scwhittle still working on the tests but curious about initial thoughts.
also opted to offload the thread management to executors instead of using raw threads. This may allow us to play around with the thread pool implementation to possibly increase performance (i.e work stealing thread pool).

Thanks!

Copy link
Contributor

Stopping reviewer notifications for this pull request: review requested by someone other than the bot, ceding control

@m-trieu m-trieu force-pushed the mt-refactor-commits branch from 74f8a1a to c9aab72 Compare February 21, 2024 22:11
@m-trieu
Copy link
Contributor Author

m-trieu commented Feb 21, 2024

@scwhittle ready for a look! thanks

Copy link
Contributor

@scwhittle scwhittle left a comment

Choose a reason for hiding this comment

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

Didn't get very far through but given round-trip time thought I'd send the comment I did have.


public WindmillStreamPoolCloseableStreamFactory(Supplier<StreamT> streamFactory) {
this.streamPool =
WindmillStreamPool.create(NUM_COMMIT_STREAMS, COMMIT_STREAM_TIMEOUT, streamFactory);
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 change StreamPool to have a method returning ClosableStreams? Not sure we need this new class. This class also hard-codes parameters that we might want to change for different stream pools.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done! good idea

this::onStreamingCommitComplete,
started)
: WorkCommitters.createApplianceWorkCommitter(
windmillServer::commitWork, numCommitThreads, running::get, started);
Copy link
Contributor

Choose a reason for hiding this comment

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

the commit threads is always 1 for appliance, should we remove the param?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done

() -> {
while (true) {
try {
ready.await();
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 get rid of this countdownlatch?

It seems that we'd just fall through to blocking on the commitQueue which is contained in this class. Alternatively if that doesn't work for some reason, having some explicit start method on the committer interface seems better than passing in this latch that does so out-of-band.

ditto for the SE impl

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done

return create(stream, () -> {});
}

public abstract Supplier<StreamT> stream();
Copy link
Contributor

Choose a reason for hiding this comment

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

seems like this should be StreamT, otherwise we could bind a supplier that creates distinct streams on each call, and we're only closing it once.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done

private boolean tryAddToCommitStream(Commit commit, CommitWorkStream commitStream) {
Preconditions.checkNotNull(commit);
// Drop commits for failed work. Such commits will be dropped by Windmill anyway.
if (commit.work().isFailed()) {
Copy link
Contributor

Choose a reason for hiding this comment

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

seems better to do this when getting the initial commit in the streamingCommitLoop and in batchCommitsToStream

otherwise we go onto batchCommitsToStream in a weird state that there was no commit added to the stream or we may count commits in batchCommitsToStream which weren't actually added.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done

activeCommitBytes.addAndGet(-size);
return false;
}
private void onStreamingCommitFailed(Commit commit) {
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 get rid of the separate failed handler? Can we just create a CompletedCommit with appropriate error status and use single onCompletedCommit handler?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done

private static final Logger LOG = LoggerFactory.getLogger(StreamingApplianceWorkCommitter.class);
private static final long TARGET_COMMIT_BUNDLE_BYTES = 32 << 20;

private final Consumer<CommitWorkRequest> commitWork;
Copy link
Contributor

Choose a reason for hiding this comment

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

some suffix to make it clearer this is funciton/consumer? makes reading code a little confusing below, there is a function of the same name.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done

@m-trieu m-trieu force-pushed the mt-refactor-commits branch from 210b847 to 540acdb Compare March 1, 2024 02:08
@ThreadSafe
final class StreamingEngineWorkCommitter implements WorkCommitter {
private static final Logger LOG = LoggerFactory.getLogger(StreamingEngineWorkCommitter.class);
private static final int COMMIT_BATCH_SIZE = 5;
Copy link
Contributor

Choose a reason for hiding this comment

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

I think you might have changed the wrong one, the bytes is still a max since that is the queue limit. I meant what is now COMMIT_BATCH_SIZE to be TARGET_COMMIT_BATCH_SIZE (could use KEYS nstead of SIZE as well)

@@ -310,6 +311,10 @@ public void invalidate(ByteString processingKey, long shardingKey) {
keyIndex.remove(key);
}

public void invalidate(ShardedKey shardedKey) {
Copy link
Contributor

Choose a reason for hiding this comment

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

final

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done

fakeWindmillServer.waitForAndGetCommits(commits.size() / 2);

for (Commit commit : commits) {
if (commit.work().isFailed()) {
Copy link
Contributor

Choose a reason for hiding this comment

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

verify that non-failed ones are in committed

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done

WorkItemCommitRequest request = committed.get(commit.work().getWorkItem().getWorkToken());
assertNotNull(request);
assertThat(request).isEqualTo(commit.request());
assertThat(completeCommits).contains(asCompleteCommit(commit));
Copy link
Contributor

Choose a reason for hiding this comment

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

add a test that verifies some particular keys failing in windmill, that the non-ok commit status is plumbed correctly

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done had to make some changes to FakeWindmillServer


@Test
public void testParallelism() {
workCommitter = createWorkCommitter(ignored -> {});
Copy link
Contributor

Choose a reason for hiding this comment

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

doesn't seem worth testing IMO

Copy link
Contributor Author

Choose a reason for hiding this comment

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

removed

@Abacn
Copy link
Contributor

Abacn commented Mar 6, 2024

fyi today is release cut. What is the status of this PR? I see the comments are all replied? CC: @scwhittle @m-trieu

@scwhittle
Copy link
Contributor

There are still some open comments. This is part of work to support direct path which won't be ready for the cut. So I don't think this should hold up the cut.

@m-trieu m-trieu force-pushed the mt-refactor-commits branch from 540acdb to 7e28f3c Compare March 7, 2024 02:25
@m-trieu
Copy link
Contributor Author

m-trieu commented Mar 7, 2024

ready for another look @scwhittle thanks!

@m-trieu
Copy link
Contributor Author

m-trieu commented Mar 8, 2024

presubmit failure is due to flaky test (working on a PR to fix) and not related.

@m-trieu
Copy link
Contributor Author

m-trieu commented Mar 8, 2024

test flakiness attempted to be fixed here #30572

.setRemainingBytesForWorkItem(0)
.build());
StreamingCommitResponse response =
streamingCommitsToOffer.getOrDefault(streamingRequestBuilder.build());
Copy link
Contributor

Choose a reason for hiding this comment

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

you are specifying hte default here and when creating the queue, seems like you could remove here.

I think instead of request/response pair injection you could set commitstatus for each worktoken. Then you wouldn't need the useInjectableStreamingCommitResponses, you could always lookup and default to OK.

The full request/response isn't adding any additonal testing since both sides are fake. And we need to call onDone with exactly one commit status for the request.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done using map instead of response queue

@@ -122,6 +131,11 @@ public void setDropStreamingCommits(boolean dropStreamingCommits) {
this.dropStreamingCommits = dropStreamingCommits;
}

public void setUseInjectableStreamingCommitResponses(
Copy link
Contributor

Choose a reason for hiding this comment

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

add a comment on what this means

Copy link
Contributor Author

Choose a reason for hiding this comment

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

removed

}

@Test
public void testCommit_handlesCompleteCommits_commitStatusOK() {
Copy link
Contributor

Choose a reason for hiding this comment

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

merge with testCommit_sendsCommitsToStreamingEngine?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done

commits.add(Commit.create(commitRequest, createComputationState("computationId-" + i), work));
fakeWindmillServer
.whenCommitWorkStreamCalled()
.thenAnswer(request -> toResponse(request, INVALID_TOKEN));
Copy link
Contributor

Choose a reason for hiding this comment

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

would be better to have some ok and some different commit status to verify

Copy link
Contributor Author

Choose a reason for hiding this comment

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

added

@m-trieu
Copy link
Contributor Author

m-trieu commented Mar 12, 2024

@scwhittle any more comments here? Thank you!

Copy link
Contributor

@scwhittle scwhittle left a comment

Choose a reason for hiding this comment

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

mostly done

commit.work().setState(Work.State.COMMIT_QUEUED);
}

activeCommitBytes.addAndGet(-commit.getSize());
Copy link
Contributor

Choose a reason for hiding this comment

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

activeCommitBytes is no longer tracking the bytes until they complete, just until they are added on the stream

You should:

  • move the decrement into the commitWorkItem ondonefn above. That handles the case it was added successfully, tracking bytes until the commit response is recieved
  • decrement here only if !isCommitSuccesful to cover the case we're keeping it until next time.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done

}

workCommitter.start();
commits.forEach(workCommitter::commit);
Copy link
Contributor

Choose a reason for hiding this comment

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

use a paralleliszed forEach to cover thread-safety a little?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done

@m-trieu
Copy link
Contributor Author

m-trieu commented Mar 12, 2024

added drain (fail work and clear queues) when we close the SEWorkCommitter

will need this for direct path. in non direct path mode we would never call stop (except when we call stop on the entire harness in StreamingDataflowWorker) so it will not affect that code path. @scwhittle

}

public void clear() {
queue.clear();
Copy link
Contributor

Choose a reason for hiding this comment

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

this is not updating limit. To do so properly it seems easiest to poll() until null, but actually perhaps a better alternative is to remove stream() and clear() and just do that loop on poll() until null in StreamingEngineWorkCommitter

That is less API of this class to test and in general if queue is still in use, stream()+clear() may process something that is poll()/take() in between the calls.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done

}

private void drainCommitQueue() {
commitQueue.stream().forEach(this::failCommit);
Copy link
Contributor

Choose a reason for hiding this comment

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

see other comment, how about loop on poll 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.

done

(commitStatus) -> {
onCommitComplete.accept(CompleteCommit.create(commit, commitStatus));
activeCommitBytes.addAndGet(-commit.getSize());
});

if (!isCommitSuccessful) {
Copy link
Contributor

Choose a reason for hiding this comment

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

// Since the commit was not accepted, revert the changes made above.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

added

@scwhittle
Copy link
Contributor

Run Java PreCommit

@m-trieu
Copy link
Contributor Author

m-trieu commented Mar 14, 2024

test failures should be fixed in #30572 since it removes arbitrary waiting from the tests (replaced with countdownlatch as event triggers)

@scwhittle
Copy link
Contributor

The other PR is merged, woudl like to verify this passes now

@scwhittle
Copy link
Contributor

Run Java PreCommit

@m-trieu m-trieu force-pushed the mt-refactor-commits branch from dfb5341 to cfa5039 Compare March 14, 2024 16:16
@m-trieu
Copy link
Contributor Author

m-trieu commented Mar 14, 2024

wooh! looks good

@scwhittle scwhittle merged commit c298da5 into apache:master Mar 15, 2024
17 checks passed
hjtran pushed a commit to hjtran/beam that referenced this pull request Apr 4, 2024
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