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

add graceful restart mechanism for GetWorkStream to prevent DEADLINE_… #34367

Open
wants to merge 1 commit into
base: master
Choose a base branch
from

Conversation

m-trieu
Copy link
Contributor

@m-trieu m-trieu commented Mar 20, 2025

GetWorkStream was previously terminating due to DEADLINE_EXCEEDED status leading to stream breaks and unnecessary retry spikes in the streaming backend.

  • Add GetWorkStream management loop that gracefully terminates and restarts the stream
  • Extend deadlines of direct streams to 1hr for less churn in restarting the streams internally
  • Add worker token to stream manager thread name

R: @scwhittle @acrites


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

…EXCEEDED; add long deadlines of 1hr to direct streams
@m-trieu
Copy link
Contributor Author

m-trieu commented Mar 23, 2025

R: @scwhittle

Copy link
Contributor

Stopping reviewer notifications for this pull request: review requested by someone other than the bot, ceding control. If you'd like to restart, comment assign set of reviewers

@m-trieu
Copy link
Contributor Author

m-trieu commented Mar 24, 2025

assign set of reviewers

Copy link
Contributor

Assigning reviewers. If you would like to opt out of this review, comment assign to next reviewer:

R: @johnjcasey added as fallback since no labels match configuration

Available commands:

  • stop reviewer notifications - opt out of the automated review tooling
  • remind me after tests pass - tag the comment author after tests pass
  • waiting on author - shift the attention set back to the author (any comment or push by the author will return the attention set to the reviewers)

The PR bot will only process comments in the main thread (not review comments).

@m-trieu
Copy link
Contributor Author

m-trieu commented Mar 25, 2025

Run Java precommit

@m-trieu
Copy link
Contributor Author

m-trieu commented Mar 26, 2025

test failure is unrelated

streamingEngineStreamFactory.createDirectGetWorkStream(
connection,
withRequestBudget(getWorkRequest, getWorkBudget.get()),
streamingEngineThrottleTimers.getWorkThrottleTimer(),
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 just use the same throttle timer, heartbeat sender, and getdataclientfactory for each stream?

FixedStreamHeartbeatSender.create(getDataStream),
getDataClientFactory.apply(getDataStream),
workCommitter,
workItemScheduler);
// 3 threads, 1 for each stream type (GetWork, GetData, CommitWork).
this.streamStarter =
Executors.newFixedThreadPool(
Copy link
Contributor

Choose a reason for hiding this comment

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

how about newCachedThreadPool? it seems like 2 of these threads are just for start() and then won't be used and we might as well have them go away.

if (started.get()) {
getWorkStream.setBudget(budget);
synchronized (activeGetWorkStream) {
GetWorkBudget budget = GetWorkBudget.builder().setItems(items).setBytes(bytes).build();
Copy link
Contributor

Choose a reason for hiding this comment

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

this build and setting on the atomic, could be outside synchronized block

synchronized (activeGetWorkStream) {
GetWorkBudget budget = GetWorkBudget.builder().setItems(items).setBytes(bytes).build();
getWorkBudget.set(budget);
if (isRunning.get()) {
Copy link
Contributor

Choose a reason for hiding this comment

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

remove running check and just use the null below? seems like if activeGetWorkStream is set it is ok to call and it's one less interleaving to think about

}

} catch (InterruptedException e) {
// continue until !isRunning.
Copy link
Contributor

Choose a reason for hiding this comment

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

Could we instead force that isRunning is set to false here? we don't expect interruptions to happen for any other reason.

try {
// Try to gracefully terminate the stream.
if (!newStream.awaitTermination(GET_WORK_STREAM_TTL_MINUTES, TimeUnit.MINUTES)) {
newStream.halfClose();
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 as we half-close here we probably want to create a new stream to take over.
That way we aren't idle while we're waiting for the termination.

}

// If graceful termination is unsuccessful, forcefully shutdown.
if (!newStream.awaitTermination(30, TimeUnit.SECONDS)) {
Copy link
Contributor

Choose a reason for hiding this comment

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

maybe better to increase this? If we lose getwork responses then windmill worker has to retry, if we can get them to flush with a little more time that seems fine.

@@ -189,6 +189,12 @@ private static <T extends AbstractStub<T>> T withDefaultDeadline(T stub) {
return stub.withDeadlineAfter(DEFAULT_STREAM_RPC_DEADLINE_SECONDS, TimeUnit.SECONDS);
}

private static <T extends AbstractStub<T>> T withLongDeadline(T stub) {
Copy link
Contributor

Choose a reason for hiding this comment

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

withDirectPathDeadline?

private final GetWorkStream getWorkStream;
private static final Logger LOG = LoggerFactory.getLogger(WindmillStreamSender.class);
private static final String STREAM_MANAGER_THREAD_NAME_FORMAT = "WindmillStreamManagerThread";
private static final int GET_WORK_STREAM_TTL_MINUTES = 45;
Copy link
Contributor

Choose a reason for hiding this comment

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

could note that this needs to be less than the deadline in the other file

? "Waited "
+ totalSecondsWaited
+ "s which exceeds given deadline of "
+ deadlineSeconds
+ inactivityTimeout
+ "s for the outboundObserver to become ready meaning "
+ "that the stream deadline was not respected."
Copy link
Contributor

Choose a reason for hiding this comment

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

this seems like the wrong message if it isn't the stream deadline

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.

2 participants