-
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
Simplify budget distribution logic and new worker metadata consumption #32775
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Large diffs are not rendered by default.
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,63 @@ | ||
/* | ||
* 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.harness; | ||
|
||
import java.io.Closeable; | ||
import java.util.function.Supplier; | ||
import javax.annotation.concurrent.ThreadSafe; | ||
import org.apache.beam.runners.dataflow.worker.windmill.WindmillEndpoints.Endpoint; | ||
import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetDataStream; | ||
import org.apache.beam.sdk.annotations.Internal; | ||
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Suppliers; | ||
|
||
@Internal | ||
@ThreadSafe | ||
// TODO (m-trieu): replace Supplier<Stream> with Stream after github.com/apache/beam/pull/32774/ is | ||
// merged | ||
final class GlobalDataStreamSender implements Closeable, Supplier<GetDataStream> { | ||
private final Endpoint endpoint; | ||
private final Supplier<GetDataStream> delegate; | ||
private volatile boolean started; | ||
|
||
GlobalDataStreamSender(Supplier<GetDataStream> delegate, Endpoint endpoint) { | ||
// Ensures that the Supplier is thread-safe | ||
this.delegate = Suppliers.memoize(delegate::get); | ||
this.started = false; | ||
this.endpoint = endpoint; | ||
} | ||
|
||
@Override | ||
public GetDataStream get() { | ||
if (!started) { | ||
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. This is not thread safe, the reads and writes to 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. started is volatile but it also isn't controlling anything useful at the moment as far as I can tell. Should started be removed for now instead? Or should close() be changed to do something only if started? 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. the change i wanted is actually here https://github.com/apache/beam/pull/32774/files#diff-561fe80cd3d4e69975cab3d41268f5eb6cda8f583f9d1e5dfac91334efb351e0 but that depends on the other PR ill guard just close for now |
||
started = true; | ||
} | ||
|
||
return delegate.get(); | ||
} | ||
|
||
@Override | ||
public void close() { | ||
if (started) { | ||
delegate.get().shutdown(); | ||
} | ||
} | ||
|
||
Endpoint endpoint() { | ||
return endpoint; | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -17,6 +17,7 @@ | |
*/ | ||
package org.apache.beam.runners.dataflow.worker.streaming.harness; | ||
|
||
import java.io.Closeable; | ||
import java.util.concurrent.atomic.AtomicBoolean; | ||
import java.util.concurrent.atomic.AtomicReference; | ||
import java.util.function.Function; | ||
|
@@ -49,7 +50,7 @@ | |
* {@link GetWorkBudget} is set. | ||
* | ||
* <p>Once started, the underlying streams are "alive" until they are manually closed via {@link | ||
* #closeAllStreams()}. | ||
* #close()} ()}. | ||
* | ||
* <p>If closed, it means that the backend endpoint is no longer in the worker set. Once closed, | ||
* these instances are not reused. | ||
|
@@ -59,7 +60,7 @@ | |
*/ | ||
@Internal | ||
@ThreadSafe | ||
final class WindmillStreamSender implements GetWorkBudgetSpender { | ||
final class WindmillStreamSender implements GetWorkBudgetSpender, Closeable { | ||
private final AtomicBoolean started; | ||
private final AtomicReference<GetWorkBudget> getWorkBudget; | ||
private final Supplier<GetWorkStream> getWorkStream; | ||
|
@@ -103,9 +104,9 @@ private WindmillStreamSender( | |
connection, | ||
withRequestBudget(getWorkRequest, getWorkBudget.get()), | ||
streamingEngineThrottleTimers.getWorkThrottleTimer(), | ||
() -> FixedStreamHeartbeatSender.create(getDataStream.get()), | ||
() -> getDataClientFactory.apply(getDataStream.get()), | ||
workCommitter, | ||
FixedStreamHeartbeatSender.create(getDataStream.get()), | ||
getDataClientFactory.apply(getDataStream.get()), | ||
workCommitter.get(), | ||
workItemScheduler)); | ||
} | ||
|
||
|
@@ -141,7 +142,8 @@ void startStreams() { | |
started.set(true); | ||
} | ||
|
||
void closeAllStreams() { | ||
@Override | ||
public void close() { | ||
// Supplier<Stream>.get() starts the stream which is an expensive operation as it initiates the | ||
// streaming RPCs by possibly making calls over the network. Do not close the streams unless | ||
// they have already been started. | ||
|
@@ -154,18 +156,13 @@ void closeAllStreams() { | |
} | ||
|
||
@Override | ||
public void adjustBudget(long itemsDelta, long bytesDelta) { | ||
getWorkBudget.set(getWorkBudget.get().apply(itemsDelta, bytesDelta)); | ||
public void setBudget(long items, long bytes) { | ||
getWorkBudget.set(getWorkBudget.get().apply(items, bytes)); | ||
if (started.get()) { | ||
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. Since 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. that is in this PR #32774 and i added synchronization there |
||
getWorkStream.get().adjustBudget(itemsDelta, bytesDelta); | ||
getWorkStream.get().setBudget(items, bytes); | ||
} | ||
} | ||
|
||
@Override | ||
public GetWorkBudget remainingBudget() { | ||
return started.get() ? getWorkStream.get().remainingBudget() : getWorkBudget.get(); | ||
} | ||
|
||
long getAndResetThrottleTime() { | ||
return streamingEngineThrottleTimers.getAndResetThrottleTime(); | ||
} | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -17,8 +17,8 @@ | |
*/ | ||
package org.apache.beam.runners.dataflow.worker.windmill; | ||
|
||
import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList.toImmutableList; | ||
import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap.toImmutableMap; | ||
import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet.toImmutableSet; | ||
|
||
import com.google.auto.value.AutoValue; | ||
import java.net.Inet6Address; | ||
|
@@ -27,8 +27,8 @@ | |
import java.util.Map; | ||
import java.util.Optional; | ||
import org.apache.beam.runners.dataflow.worker.windmill.WindmillServiceAddress.AuthenticatedGcpServiceAddress; | ||
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; | ||
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; | ||
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; | ||
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
@@ -41,6 +41,14 @@ | |
public abstract class WindmillEndpoints { | ||
private static final Logger LOG = LoggerFactory.getLogger(WindmillEndpoints.class); | ||
|
||
public static WindmillEndpoints none() { | ||
return WindmillEndpoints.builder() | ||
.setVersion(Long.MAX_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. min seems safer. Otherwise if somehow none() was observed the logic to ensure version is increasing mean's we'd never process another endpoint set 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 |
||
.setWindmillEndpoints(ImmutableSet.of()) | ||
.setGlobalDataEndpoints(ImmutableMap.of()) | ||
.build(); | ||
} | ||
|
||
public static WindmillEndpoints from( | ||
Windmill.WorkerMetadataResponse workerMetadataResponseProto) { | ||
ImmutableMap<String, WindmillEndpoints.Endpoint> globalDataServers = | ||
|
@@ -53,14 +61,15 @@ public static WindmillEndpoints from( | |
endpoint.getValue(), | ||
workerMetadataResponseProto.getExternalEndpoint()))); | ||
|
||
ImmutableList<WindmillEndpoints.Endpoint> windmillServers = | ||
ImmutableSet<WindmillEndpoints.Endpoint> windmillServers = | ||
workerMetadataResponseProto.getWorkEndpointsList().stream() | ||
.map( | ||
endpointProto -> | ||
Endpoint.from(endpointProto, workerMetadataResponseProto.getExternalEndpoint())) | ||
.collect(toImmutableList()); | ||
.collect(toImmutableSet()); | ||
|
||
return WindmillEndpoints.builder() | ||
.setVersion(workerMetadataResponseProto.getMetadataVersion()) | ||
.setGlobalDataEndpoints(globalDataServers) | ||
.setWindmillEndpoints(windmillServers) | ||
.build(); | ||
|
@@ -123,6 +132,9 @@ private static Optional<HostAndPort> tryParseDirectEndpointIntoIpV6Address( | |
directEndpointAddress.getHostAddress(), (int) endpointProto.getPort())); | ||
} | ||
|
||
/** Version of the endpoints which increases with every modification. */ | ||
public abstract long version(); | ||
|
||
/** | ||
* Used by GetData GlobalDataRequest(s) to support Beam side inputs. Returns a map where the key | ||
* is a global data tag and the value is the endpoint where the data associated with the global | ||
|
@@ -138,7 +150,7 @@ private static Optional<HostAndPort> tryParseDirectEndpointIntoIpV6Address( | |
* Windmill servers. Returns a list of endpoints used to communicate with the corresponding | ||
* Windmill servers. | ||
*/ | ||
public abstract ImmutableList<Endpoint> windmillEndpoints(); | ||
public abstract ImmutableSet<Endpoint> windmillEndpoints(); | ||
|
||
/** | ||
* Representation of an endpoint in {@link Windmill.WorkerMetadataResponse.Endpoint} proto with | ||
|
@@ -204,13 +216,15 @@ public abstract static class Builder { | |
|
||
@AutoValue.Builder | ||
public abstract static class Builder { | ||
public abstract Builder setVersion(long version); | ||
|
||
public abstract Builder setGlobalDataEndpoints( | ||
ImmutableMap<String, WindmillEndpoints.Endpoint> globalDataServers); | ||
|
||
public abstract Builder setWindmillEndpoints( | ||
ImmutableList<WindmillEndpoints.Endpoint> windmillServers); | ||
ImmutableSet<WindmillEndpoints.Endpoint> windmillServers); | ||
|
||
abstract ImmutableList.Builder<WindmillEndpoints.Endpoint> windmillEndpointsBuilder(); | ||
abstract ImmutableSet.Builder<WindmillEndpoints.Endpoint> windmillEndpointsBuilder(); | ||
|
||
public final Builder addWindmillEndpoint(WindmillEndpoints.Endpoint endpoint) { | ||
windmillEndpointsBuilder().add(endpoint); | ||
|
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.
For the class to be threadsafe the provided supplier needs to be thread safe, can we add a comment on the supplier?
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 will be changed in https://github.com/apache/beam/pull/32774/files#diff-561fe80cd3d4e69975cab3d41268f5eb6cda8f583f9d1e5dfac91334efb351e0