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

do not block other threads from health checking if a stream is blocked #34283

Open
wants to merge 4 commits into
base: master
Choose a base branch
from

Conversation

m-trieu
Copy link
Contributor

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

Currently a single blocked stream can prevent health checks from multiple streams. Use the stream's internal executor to hand do the health check RPC in a way that does not block the health check scheduling thread.

R: @acrites @scwhittle


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

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).

logger.debug("Received exception sending health check.", e);
}
}
public final void maybeSendHealthCheck(Instant lastSendThreshold) {
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 that we might want to have the check if a health-check should be scheduled before scheduling on the executor.

Otherwise there may be more churn on the executor than necessary maybe adding to contention if maybeSendHealthCheck is called more often. We probably want to keep track if there is an active health-check with a member boolean as well to avoid piling up in the executor since maybeSendHealthCheck is no longer blocking and may be called more frequently than actually sending the health checks takes.

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 19, 2025

ready for a another look @scwhittle

* synchronized block, we skip the attempt to send scheduled the health check.
*/
public final void maybeSendHealthCheck(Instant lastSendThreshold) {
if (debugMetrics.getLastSendTimeMs() < lastSendThreshold.getMillis() && !isHealthCheckActive) {
Copy link
Contributor

Choose a reason for hiding this comment

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

maybe it should be !isHealthCheckScheduled and then set it to true here before putting on the executor?

if the executor gets backlogged we could still end up queuing a lot of things on the executor since currently we don't mark active until it schedules

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, opted to use an AtomicBoolean

// Make sure the future completes.
sendFuture.get();

assertThat(testStream.numHealthChecks.get()).isEqualTo(1);
Copy link
Contributor

Choose a reason for hiding this comment

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

not sure if this is going to be flaky since lambda queued by trySend could possibly not have actually executed yet. Seems like it could be safer to loop while it is not yet 1.

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

testStream.start();
ExecutorService sendExecutor = Executors.newSingleThreadExecutor();
Instant reportingThreshold = Instant.now().minus(Duration.millis(1));
Future<?> sendFuture =
Copy link
Contributor

Choose a reason for hiding this comment

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

I don't think you need to schedule this on a executor because maybeSendHealthCheck is no longer blocking.

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

testStream.maybeSendHealthCheck(reportingThreshold);
});

callStreamObserver.unblockSend();
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 start the test with it unblocked? Seems like you could just remove the sendFuture then and just do everything from the main test thread.

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

// Set a really long reporting threshold.
Instant reportingThreshold = Instant.now().minus(Duration.standardHours(1));
// Should not send health checks since we just sent the above message.
testStream.maybeSendHealthCheck(reportingThreshold);
Copy link
Contributor

Choose a reason for hiding this comment

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

might need to sleep or verify the internal executor for the stream has nothing queud before you check no health-checks were sent

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 26, 2025

ready for a another look @scwhittle

}

// Ready to send another health check after we attempt the scheduled health check.
isHealthCheckScheduled.set(false);
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 showing if health check is scheduled in the appendSummaryHtml?

}

// Ready to send another health check after we attempt the scheduled health check.
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 it would be a little safer to move the try to cover all of this lambda
so that we are sure to set it back to false.

try {
sycnrhonized () { ... maybe send }
} catch {
log
} finally {
isHealthCheckScheduled.set(false);
}

throw new RuntimeException(e);
}

// Sleep a bit to give sendExecutor time to execute the send().
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 trySend send inline? can this be removed?

Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);

callStreamObserver.waitForSend();
Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
Copy link
Contributor

Choose a reason for hiding this comment

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

// Sleep just to ensure a aysnc health check doesn't show up

int waitedMillis = 0;
while (!sendBlocker.await(100, TimeUnit.MILLISECONDS)) {
waitedMillis += 100;
LOG.info("Waiting from send for {}ms", waitedMillis);
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 sendBlocker is being used two different ways and is a little confusing.

How about naming this method to waitForSendUnblocked and logging here that you are waiting for send to be unblocked.
And then below in onNext instead of decrementing sendblocker, instead you increment a count of number of received and can add a new method to wait for N messages to be sent.

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