scwhittle commented on code in PR #28835: URL: https://github.com/apache/beam/pull/28835#discussion_r1370074314
########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillStreamSender.java: ########## @@ -0,0 +1,147 @@ +/* + * 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; + +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Supplier; +import javax.annotation.concurrent.ThreadSafe; +import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1Stub; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetWorkRequest; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.CommitWorkStream; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetDataStream; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetWorkStream; +import org.apache.beam.runners.dataflow.worker.windmill.util.StreamingEngineThrottleTimers; +import org.apache.beam.runners.dataflow.worker.windmill.work.WorkItemReceiver; +import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudget; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Suppliers; + +/** + * Owns and maintains a set of streams used to communicate with a specific Windmill worker. + * Underlying streams are "cached" in a threadsafe manner so that once {@link Supplier#get} is + * called, a stream that is already started is returned. + * + * <p>Holds references to {@link Supplier<WindmillStream>} because initializing the streams + * automatically start them, and we want to do so lazily here once the {@link GetWorkBudget} is set. + * + * <p>Once started, the underlying streams are "alive" until they are manually closed via {@link + * #closeAllStreams()}. + * + * <p>If closed, it means that the backend endpoint is no longer in the worker set. Once closed, + * these instances are not reused. + * + * @implNote Does not manage streams for fetching {@link + * org.apache.beam.runners.dataflow.worker.windmill.Windmill.GlobalData} for side inputs. + */ +@ThreadSafe +public class WindmillStreamSender { + + private final AtomicBoolean started; + private final AtomicReference<GetWorkBudget> getWorkBudget; + private final Supplier<GetWorkStream> getWorkStream; + private final Supplier<GetDataStream> getKeyedDataStream; + private final Supplier<CommitWorkStream> commitWorkStream; + private final StreamingEngineThrottleTimers streamingEngineThrottleTimers; + + private WindmillStreamSender( + CloudWindmillServiceV1Alpha1Stub stub, + GetWorkRequest getWorkRequest, + AtomicReference<GetWorkBudget> getWorkBudget, + StreamingEngineStreamFactory streamingEngineStreamFactory, + WorkItemReceiver workItemReceiver) { + this.started = new AtomicBoolean(false); + this.getWorkBudget = getWorkBudget; + this.streamingEngineThrottleTimers = StreamingEngineThrottleTimers.create(); + + // All streams are memoized/cached since they are expensive to create and some implementations + // perform side effects on construction (i.e. sending initial requests to the stream server to + // initiate the streaming RPC connection). Stream instances connect/reconnect internally so we + // can reuse the same instance through the entire lifecycle of WindmillStreamSender. + this.getWorkStream = + Suppliers.memoize( + () -> + streamingEngineStreamFactory.createGetWorkStream( + stub, + withRequestBudget(getWorkRequest, getWorkBudget.get()), + streamingEngineThrottleTimers.getWorkThrottleTimer(), + workItemReceiver)); + this.getKeyedDataStream = + Suppliers.memoize( + () -> + streamingEngineStreamFactory.createGetDataStream( + stub, streamingEngineThrottleTimers.getDataThrottleTimer())); + this.commitWorkStream = + Suppliers.memoize( + () -> + streamingEngineStreamFactory.createCommitWorkStream( + stub, streamingEngineThrottleTimers.commitWorkThrottleTimer())); + } + + public static WindmillStreamSender create( + CloudWindmillServiceV1Alpha1Stub stub, + GetWorkRequest getWorkRequest, + GetWorkBudget getWorkBudget, + StreamingEngineStreamFactory streamingEngineStreamFactory, + WorkItemReceiver workItemReceiver) { + return new WindmillStreamSender( + stub, + getWorkRequest, + new AtomicReference<>(getWorkBudget), + streamingEngineStreamFactory, + workItemReceiver); + } + + private static GetWorkRequest withRequestBudget(GetWorkRequest request, GetWorkBudget budget) { + return request.toBuilder().setMaxItems(budget.items()).setMaxBytes(budget.bytes()).build(); + } + + @SuppressWarnings("ReturnValueIgnored") + public void startStreams() { + Preconditions.checkState( + !getWorkBudget.get().equals(GetWorkBudget.noBudget()), "Cannot GetWork with no budget."); + getWorkStream.get(); + getKeyedDataStream.get(); + commitWorkStream.get(); + // *stream.get() is all memoized in a threadsafe manner. + started.compareAndSet(false, true); + } + + public void closeAllStreams() { + // Supplier<Stream>.get() starts the stream which is an expensive operation as it initiates the + // streaming RPCs. Do not close the streams unless they have already been started. + if (started.get()) { Review Comment: this is racy if some streams are started and others not closed. An alternative to an atomic would be to have an object you syncrhonize on here and in startStreams so close blocks if start is still ongoing. Or you could document that start/close are not thread-safe and should be sequenced ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillStream.java: ########## @@ -41,15 +40,13 @@ public interface WindmillStream { /** Handle representing a stream of GetWork responses. */ @ThreadSafe interface GetWorkStream extends WindmillStream { - /** Functional interface for receiving WorkItems. */ - @FunctionalInterface - interface WorkItemReceiver { - void receiveWork( - String computation, - @Nullable Instant inputDataWatermark, - @Nullable Instant synchronizedProcessingTime, - Windmill.WorkItem workItem, - Collection<Windmill.LatencyAttribution> getWorkStreamLatencies); + + /** Adjusts the {@link GetWorkBudget} for the stream. */ + default void adjustBudget(long itemsDelta, long bytesDelta) {} Review Comment: these don't seem like great defaults to have, can we just keep it abstract to make sure it is implemented? and put no-op in test impls if desired? ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/util/WindmillChannelFactory.java: ########## @@ -0,0 +1,136 @@ +/* + * 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.util; + +import java.net.Inet6Address; +import java.net.InetSocketAddress; +import java.util.concurrent.TimeUnit; +import javax.net.ssl.SSLException; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillServiceAddress; +import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Channel; +import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessChannelBuilder; +import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.netty.GrpcSslContexts; +import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.netty.NegotiationType; +import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.netty.NettyChannelBuilder; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort; + +/** Utilities for creating {@link Channel} for gRPC stubs. */ +public final class WindmillChannelFactory { Review Comment: Would it be possible to submit first the refactor+update of existing files? I think it would help make the size of this PR smaller and it would also make it clearer which code is being moved/refactored versus the new code. In particular this and StreamingEngineStreamFactory both seem to just be extracted from GrpcWindmillServer. A single PR adding these and updating exisitng usage will be easier to review and ensures we don't have duplicate code sitting around that could possibly drift. ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillStreamSender.java: ########## @@ -0,0 +1,147 @@ +/* + * 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; + +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Supplier; +import javax.annotation.concurrent.ThreadSafe; +import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1Stub; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetWorkRequest; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.CommitWorkStream; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetDataStream; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetWorkStream; +import org.apache.beam.runners.dataflow.worker.windmill.util.StreamingEngineThrottleTimers; +import org.apache.beam.runners.dataflow.worker.windmill.work.WorkItemReceiver; +import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudget; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Suppliers; + +/** + * Owns and maintains a set of streams used to communicate with a specific Windmill worker. + * Underlying streams are "cached" in a threadsafe manner so that once {@link Supplier#get} is + * called, a stream that is already started is returned. + * + * <p>Holds references to {@link Supplier<WindmillStream>} because initializing the streams + * automatically start them, and we want to do so lazily here once the {@link GetWorkBudget} is set. + * + * <p>Once started, the underlying streams are "alive" until they are manually closed via {@link + * #closeAllStreams()}. + * + * <p>If closed, it means that the backend endpoint is no longer in the worker set. Once closed, + * these instances are not reused. + * + * @implNote Does not manage streams for fetching {@link + * org.apache.beam.runners.dataflow.worker.windmill.Windmill.GlobalData} for side inputs. + */ +@ThreadSafe +public class WindmillStreamSender { + + private final AtomicBoolean started; + private final AtomicReference<GetWorkBudget> getWorkBudget; + private final Supplier<GetWorkStream> getWorkStream; + private final Supplier<GetDataStream> getKeyedDataStream; + private final Supplier<CommitWorkStream> commitWorkStream; + private final StreamingEngineThrottleTimers streamingEngineThrottleTimers; + + private WindmillStreamSender( + CloudWindmillServiceV1Alpha1Stub stub, + GetWorkRequest getWorkRequest, + AtomicReference<GetWorkBudget> getWorkBudget, + StreamingEngineStreamFactory streamingEngineStreamFactory, + WorkItemReceiver workItemReceiver) { + this.started = new AtomicBoolean(false); + this.getWorkBudget = getWorkBudget; + this.streamingEngineThrottleTimers = StreamingEngineThrottleTimers.create(); + + // All streams are memoized/cached since they are expensive to create and some implementations + // perform side effects on construction (i.e. sending initial requests to the stream server to + // initiate the streaming RPC connection). Stream instances connect/reconnect internally so we + // can reuse the same instance through the entire lifecycle of WindmillStreamSender. + this.getWorkStream = + Suppliers.memoize( + () -> + streamingEngineStreamFactory.createGetWorkStream( + stub, + withRequestBudget(getWorkRequest, getWorkBudget.get()), + streamingEngineThrottleTimers.getWorkThrottleTimer(), + workItemReceiver)); + this.getKeyedDataStream = + Suppliers.memoize( + () -> + streamingEngineStreamFactory.createGetDataStream( + stub, streamingEngineThrottleTimers.getDataThrottleTimer())); + this.commitWorkStream = + Suppliers.memoize( + () -> + streamingEngineStreamFactory.createCommitWorkStream( + stub, streamingEngineThrottleTimers.commitWorkThrottleTimer())); + } + + public static WindmillStreamSender create( + CloudWindmillServiceV1Alpha1Stub stub, + GetWorkRequest getWorkRequest, + GetWorkBudget getWorkBudget, + StreamingEngineStreamFactory streamingEngineStreamFactory, + WorkItemReceiver workItemReceiver) { + return new WindmillStreamSender( + stub, + getWorkRequest, + new AtomicReference<>(getWorkBudget), + streamingEngineStreamFactory, + workItemReceiver); + } + + private static GetWorkRequest withRequestBudget(GetWorkRequest request, GetWorkBudget budget) { + return request.toBuilder().setMaxItems(budget.items()).setMaxBytes(budget.bytes()).build(); + } + + @SuppressWarnings("ReturnValueIgnored") + public void startStreams() { + Preconditions.checkState( + !getWorkBudget.get().equals(GetWorkBudget.noBudget()), "Cannot GetWork with no budget."); + getWorkStream.get(); + getKeyedDataStream.get(); + commitWorkStream.get(); + // *stream.get() is all memoized in a threadsafe manner. + started.compareAndSet(false, true); + } + + public void closeAllStreams() { + // Supplier<Stream>.get() starts the stream which is an expensive operation as it initiates the + // streaming RPCs. Do not close the streams unless they have already been started. + if (started.get()) { + getWorkStream.get().close(); + getKeyedDataStream.get().close(); + commitWorkStream.get().close(); + } + } + + public synchronized void adjustBudget(long itemsDelta, long bytesDelta) { + getWorkBudget.set(getWorkBudget.get().add(itemsDelta, bytesDelta)); Review Comment: this will create the stream if it was not started ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillStreamSender.java: ########## @@ -0,0 +1,147 @@ +/* + * 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; + +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Supplier; +import javax.annotation.concurrent.ThreadSafe; +import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1Stub; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetWorkRequest; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.CommitWorkStream; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetDataStream; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetWorkStream; +import org.apache.beam.runners.dataflow.worker.windmill.util.StreamingEngineThrottleTimers; +import org.apache.beam.runners.dataflow.worker.windmill.work.WorkItemReceiver; +import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudget; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Suppliers; + +/** + * Owns and maintains a set of streams used to communicate with a specific Windmill worker. + * Underlying streams are "cached" in a threadsafe manner so that once {@link Supplier#get} is + * called, a stream that is already started is returned. + * + * <p>Holds references to {@link Supplier<WindmillStream>} because initializing the streams + * automatically start them, and we want to do so lazily here once the {@link GetWorkBudget} is set. + * + * <p>Once started, the underlying streams are "alive" until they are manually closed via {@link + * #closeAllStreams()}. + * + * <p>If closed, it means that the backend endpoint is no longer in the worker set. Once closed, + * these instances are not reused. + * + * @implNote Does not manage streams for fetching {@link + * org.apache.beam.runners.dataflow.worker.windmill.Windmill.GlobalData} for side inputs. + */ +@ThreadSafe +public class WindmillStreamSender { + + private final AtomicBoolean started; + private final AtomicReference<GetWorkBudget> getWorkBudget; + private final Supplier<GetWorkStream> getWorkStream; + private final Supplier<GetDataStream> getKeyedDataStream; + private final Supplier<CommitWorkStream> commitWorkStream; + private final StreamingEngineThrottleTimers streamingEngineThrottleTimers; + + private WindmillStreamSender( + CloudWindmillServiceV1Alpha1Stub stub, + GetWorkRequest getWorkRequest, + AtomicReference<GetWorkBudget> getWorkBudget, + StreamingEngineStreamFactory streamingEngineStreamFactory, + WorkItemReceiver workItemReceiver) { + this.started = new AtomicBoolean(false); + this.getWorkBudget = getWorkBudget; + this.streamingEngineThrottleTimers = StreamingEngineThrottleTimers.create(); + + // All streams are memoized/cached since they are expensive to create and some implementations + // perform side effects on construction (i.e. sending initial requests to the stream server to + // initiate the streaming RPC connection). Stream instances connect/reconnect internally so we + // can reuse the same instance through the entire lifecycle of WindmillStreamSender. + this.getWorkStream = + Suppliers.memoize( + () -> + streamingEngineStreamFactory.createGetWorkStream( + stub, + withRequestBudget(getWorkRequest, getWorkBudget.get()), + streamingEngineThrottleTimers.getWorkThrottleTimer(), + workItemReceiver)); + this.getKeyedDataStream = + Suppliers.memoize( + () -> + streamingEngineStreamFactory.createGetDataStream( + stub, streamingEngineThrottleTimers.getDataThrottleTimer())); + this.commitWorkStream = + Suppliers.memoize( + () -> + streamingEngineStreamFactory.createCommitWorkStream( + stub, streamingEngineThrottleTimers.commitWorkThrottleTimer())); + } + + public static WindmillStreamSender create( + CloudWindmillServiceV1Alpha1Stub stub, + GetWorkRequest getWorkRequest, Review Comment: similarly this request overlaps with budget and unclear what is used. remove the budget and start with request values or perhaps JobHeader can be used. ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/StreamingEngineClient.java: ########## @@ -0,0 +1,452 @@ +/* + * 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; + +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 com.google.errorprone.annotations.CheckReturnValue; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import javax.annotation.concurrent.ThreadSafe; +import org.apache.beam.repackaged.core.org.apache.commons.lang3.tuple.Pair; +import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1Stub; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetWorkRequest; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillEndpoints.Endpoint; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetDataStream; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetWorkerMetadataStream; +import org.apache.beam.runners.dataflow.worker.windmill.grpcclient.ThrottleTimer; +import org.apache.beam.runners.dataflow.worker.windmill.util.WindmillGrpcStubFactory; +import org.apache.beam.runners.dataflow.worker.windmill.work.WorkItemReceiver; +import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudget; +import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudgetDistributor; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableCollection; +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.util.concurrent.ThreadFactoryBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Client for StreamingEngine. Given a {@link GetWorkBudget}, divides the budget and starts the + * {@link org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetWorkStream}(s). + */ +@CheckReturnValue +@ThreadSafe +public class StreamingEngineClient { + @VisibleForTesting static final int SCHEDULED_BUDGET_REFRESH_MILLIS = 100; + private static final Logger LOG = LoggerFactory.getLogger(StreamingEngineClient.class); + private static final String SCHEDULED_BUDGET_REFRESH_THREAD = "ScheduledBudgetRefreshThread"; + private static final String TRIGGERED_BUDGET_REFRESH_THREAD = "TriggeredBudgetRefreshThread"; + private static final String CONSUMER_WORKER_METADATA_THREAD = "ConsumeWorkerMetadataThread"; + private final AtomicBoolean started; + private final AtomicBoolean running; + private final GetWorkBudget totalGetWorkBudget; + private final StreamingEngineStreamFactory streamingEngineStreamFactory; + private final WorkItemReceiver workItemReceiver; + private final WindmillGrpcStubFactory windmillGrpcStubFactory; + private final GetWorkRequest getWorkRequest; + private final GetWorkBudgetDistributor getWorkBudgetDistributor; + private final DispatcherClient dispatcherClient; + /** + * Reference to {@link GetWorkerMetadataStream} that is lazily initialized, with its initial value + * being null. + */ + private final AtomicReference<GetWorkerMetadataStream> getWorkerMetadataStream; + + private final AtomicBoolean isBudgetRefreshPaused; + /** Writes are guarded by synchronization, reads are lock free. */ + private final AtomicReference<StreamEngineConnectionState> connections; + + private final ExecutorService triggeredBudgetRefreshListener; + /** + * Used to implement publish/subscribe behavior for triggering budget refreshes/redistribution. + * Subscriber {@link #triggeredBudgetRefreshListener} waits for messages to be written to the + * queue, blocking if necessary. + */ + private final SynchronousQueue<Boolean> budgetRefreshTrigger; + + /** Redistributes the budget on a timed cadence. */ + private final ScheduledExecutorService scheduledBudgetRefreshExecutor; Review Comment: It does appear there is a backup time-based trigger in existing internal implemention to recover from cases where budget is lost due to rounding etc. However it is implemented via a single thread waiting for an explicit trigger with a timeout instead of separate threads that may concurrently try to refresh the budget. So perhaps we should keep the timeout too for the same reason but let's also move to the single thread doing refreshing to simplify (as previous comment below was also suggesting). -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
