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

extract semaphore logic out of WeightBoundedQueue to allow for sharing the weigher #32905

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

Conversation

m-trieu
Copy link
Contributor

@m-trieu m-trieu commented Oct 22, 2024

Factor out semaphore logic out of WeightBoundedQueue to allow for sharing the weigher

R: @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

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
Copy link
Contributor Author

m-trieu commented Oct 24, 2024

@scwhittle ready to for review, the failure to unrelated thanks!

@m-trieu
Copy link
Contributor Author

m-trieu commented Oct 25, 2024

Run Java Precommit

@@ -118,19 +118,17 @@ public final class StreamingDataflowWorker {
*/
public static final int MAX_SINK_BYTES = 10_000_000;

public static final String STREAMING_ENGINE_USE_JOB_SETTINGS_FOR_HEARTBEAT_POOL =
Copy link
Contributor

Choose a reason for hiding this comment

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

revert this file

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 new WeightedSemaphore<>(maxWeight, new Semaphore(maxWeight, true), weigherFn);
}

void acquire(V value) {
Copy link
Contributor

Choose a reason for hiding this comment

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

public acquire and release? seems like basic functionality

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 new WeightedSemaphore<>(maxWeight, new Semaphore(maxWeight, true), weigherFn);
}

void acquire(V value) {
Copy link
Contributor

Choose a reason for hiding this comment

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

name acquireUninterruptibly?

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

@@ -61,11 +62,10 @@ public final class StreamingEngineWorkCommitter implements WorkCommitter {
Supplier<CloseableStream<CommitWorkStream>> commitWorkStreamFactory,
int numCommitSenders,
Consumer<CompleteCommit> onCommitComplete,
String backendWorkerToken) {
String backendWorkerToken,
WeightedSemaphore<Commit> weigher) {
Copy link
Contributor

Choose a reason for hiding this comment

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

rename weigher throughout to commitByteSemaphore or something capturing it is the limiter not just a weighing function?

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

Thread putThread =
new Thread(
() -> {
try {
Copy link
Contributor

Choose a reason for hiding this comment

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

remove the sleep? the element is already added by queue1 above so the sleep doesn't seem necessary

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 result;
}

/** Returns and removes the next value, or blocks until one is available. */
public @Nullable V take() throws InterruptedException {
V result = queue.take();
limit.release(weigher.apply(result));
weigher.release(result);
return result;
}

/** Returns the current weight of the queue. */
Copy link
Contributor

Choose a reason for hiding this comment

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

can this be removed and just look at the weighted semaphore? It is confusing since it doesn't count just bytes in this queue.

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 final int maxWeight;
private final Semaphore limit;
private final Function<V, Integer> weigher;
private final WeightedSemaphore<V> weigher;
Copy link
Contributor

Choose a reason for hiding this comment

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

name weightedSemaphore? weigher just sounds like it is the weigh function

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

@@ -86,16 +86,19 @@ public final class StreamingEngineWorkCommitter implements WorkCommitter {
public static Builder builder() {
return new AutoBuilder_StreamingEngineWorkCommitter_Builder()
.setBackendWorkerToken(NO_BACKEND_WORKER_TOKEN)
.setWeigher(
WeightedSemaphore.create(
Copy link
Contributor

Choose a reason for hiding this comment

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

maybe require this to be set by caller instead of hiding it? It would be nice for the callers to know about it because then they could display it on their status page

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

});
putThread.start();

// Should only see the first value in the queue, since the queue is at capacity. putThread
Copy link
Contributor

Choose a reason for hiding this comment

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

could put the sleep here if you want to give a little more time for putThread to run and become blocked (doesn't matter if it doesn't or not so not racy but better test if it does block).

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 Nov 2, 2024

back to you @scwhittle thank you

Copy link
Contributor

github-actions bot commented Nov 2, 2024

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

R: @Abacn 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).

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.

minor cleanups

public static <V> WeightedBoundedQueue<V> create(int maxWeight, Function<V, Integer> weigherFn) {
return new WeightedBoundedQueue<>(
new LinkedBlockingQueue<>(), maxWeight, new Semaphore(maxWeight, true), weigherFn);
public static <V> WeightedBoundedQueue<V> create(WeightedSemaphore<V> weigher) {
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: weightedSemaphore

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

}

public void acquireUninterruptibly(V value) {
limit.acquireUninterruptibly(weigher.apply(value));
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 capping the calculated weight to not be more than the max so it will be the single item available to process?

Otherwise it seems that we coudl block forever. this is currently done in the specific weigher logic but seems safer to do here and in release.

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


public static WeightedSemaphore<Commit> maxCommitByteSemaphore() {
return WeightedSemaphore.create(
MAX_QUEUED_COMMITS_BYTES, commit -> Math.min(MAX_QUEUED_COMMITS_BYTES, 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.

see other comment, remove the min here.

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

MAX_COMMIT_QUEUE_BYTES, commit -> Math.min(MAX_COMMIT_QUEUE_BYTES, commit.getSize()));
WeightedSemaphore.create(
MAX_COMMIT_QUEUE_BYTES,
commit -> Math.min(MAX_COMMIT_QUEUE_BYTES, 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.

remove min.

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

Thread.sleep(100);
assertEquals(MAX_WEIGHT, weigher.currentWeight());
assertEquals(1, queue1.size());
assertEquals(MAX_WEIGHT, weigher.currentWeight());
Copy link
Contributor

Choose a reason for hiding this comment

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

rm, doing this twice.

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

putThread.join();

assertEquals(MAX_WEIGHT, weigher.currentWeight());
assertEquals(1, queue2.size());
Copy link
Contributor

Choose a reason for hiding this comment

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

poll from queue2 and verify the weight goes to zero.

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 Nov 5, 2024

back to you thanks! @scwhittle

@scwhittle
Copy link
Contributor

Run Java PreCommit

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