-
Notifications
You must be signed in to change notification settings - Fork 4.3k
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
Changes from 2 commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,49 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
package org.apache.beam.runners.dataflow.worker.streaming; | ||
|
||
import java.util.concurrent.Semaphore; | ||
import java.util.function.Function; | ||
|
||
public final class WeightedSemaphore<V> { | ||
private final int maxWeight; | ||
private final Semaphore limit; | ||
private final Function<V, Integer> weigher; | ||
|
||
private WeightedSemaphore(int maxWeight, Semaphore limit, Function<V, Integer> weigher) { | ||
this.maxWeight = maxWeight; | ||
this.limit = limit; | ||
this.weigher = weigher; | ||
} | ||
|
||
public static <V> WeightedSemaphore<V> create(int maxWeight, Function<V, Integer> weigherFn) { | ||
return new WeightedSemaphore<>(maxWeight, new Semaphore(maxWeight, true), weigherFn); | ||
} | ||
|
||
public void acquireUninterruptibly(V value) { | ||
limit.acquireUninterruptibly(weigher.apply(value)); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. done |
||
} | ||
|
||
public void release(V value) { | ||
limit.release(weigher.apply(value)); | ||
} | ||
|
||
public int currentWeight() { | ||
return maxWeight - limit.availablePermits(); | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,37 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
package org.apache.beam.runners.dataflow.worker.windmill.client.commits; | ||
|
||
import org.apache.beam.runners.dataflow.worker.streaming.WeightedSemaphore; | ||
import org.apache.beam.sdk.annotations.Internal; | ||
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; | ||
|
||
/** Utility class for commits. */ | ||
@Internal | ||
public final class Commits { | ||
|
||
/** Max bytes of commits queued on the user worker. */ | ||
@VisibleForTesting static final int MAX_QUEUED_COMMITS_BYTES = 500 << 20; // 500MB | ||
|
||
private Commits() {} | ||
|
||
public static WeightedSemaphore<Commit> maxCommitByteSemaphore() { | ||
return WeightedSemaphore.create( | ||
MAX_QUEUED_COMMITS_BYTES, commit -> Math.min(MAX_QUEUED_COMMITS_BYTES, commit.getSize())); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. see other comment, remove the min here. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. done |
||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -27,6 +27,7 @@ | |
import org.apache.beam.runners.dataflow.worker.streaming.ComputationState; | ||
import org.apache.beam.runners.dataflow.worker.streaming.ShardedKey; | ||
import org.apache.beam.runners.dataflow.worker.streaming.WeightedBoundedQueue; | ||
import org.apache.beam.runners.dataflow.worker.streaming.WeightedSemaphore; | ||
import org.apache.beam.runners.dataflow.worker.streaming.Work; | ||
import org.apache.beam.runners.dataflow.worker.streaming.WorkId; | ||
import org.apache.beam.runners.dataflow.worker.windmill.Windmill; | ||
|
@@ -55,7 +56,9 @@ private StreamingApplianceWorkCommitter( | |
this.commitWorkFn = commitWorkFn; | ||
this.commitQueue = | ||
WeightedBoundedQueue.create( | ||
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()))); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. remove min. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. done |
||
this.commitWorkers = | ||
Executors.newSingleThreadExecutor( | ||
new ThreadFactoryBuilder() | ||
|
@@ -73,10 +76,9 @@ public static StreamingApplianceWorkCommitter create( | |
} | ||
|
||
@Override | ||
@SuppressWarnings("FutureReturnValueIgnored") | ||
public void start() { | ||
if (!commitWorkers.isShutdown()) { | ||
commitWorkers.submit(this::commitLoop); | ||
commitWorkers.execute(this::commitLoop); | ||
} | ||
} | ||
|
||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: weightedSemaphore
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
done