scwhittle commented on code in PR #27767: URL: https://github.com/apache/beam/pull/27767#discussion_r1307199364
########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetWorkerMetadataStream.java: ########## @@ -0,0 +1,177 @@ +/* + * 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.grpcclient; + +import java.io.PrintWriter; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; +import java.util.function.Function; +import org.apache.beam.runners.dataflow.worker.windmill.AbstractWindmillStream; +import org.apache.beam.runners.dataflow.worker.windmill.StreamObserverFactory; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.JobHeader; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkerMetadataRequest; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkerMetadataResponse; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillEndpoints; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetWorkerMetadataStream; +import org.apache.beam.sdk.util.BackOff; +import org.apache.beam.sdk.util.FluentBackoff; +import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +final class GrpcGetWorkerMetadataStream + extends AbstractWindmillStream<WorkerMetadataRequest, WorkerMetadataResponse> + implements GetWorkerMetadataStream { + private static final Logger LOG = LoggerFactory.getLogger(GrpcGetWorkerMetadataStream.class); + private static final WorkerMetadataRequest HEALTH_CHECK_REQUEST = + WorkerMetadataRequest.getDefaultInstance(); + + private final AtomicLong metadataVersion; + private final WorkerMetadataRequest workerMetadataRequest; + private final ThrottleTimer getWorkerMetadataThrottleTimer; + private final Consumer<WindmillEndpoints> serverMappingUpdater; + + private GrpcGetWorkerMetadataStream( + Function<StreamObserver<WorkerMetadataResponse>, StreamObserver<WorkerMetadataRequest>> + startGetWorkerMetadataRpcFn, + BackOff backoff, + StreamObserverFactory streamObserverFactory, + Set<AbstractWindmillStream<?, ?>> streamRegistry, + int logEveryNStreamFailures, + JobHeader jobHeader, + AtomicLong metadataVersion, + ThrottleTimer getWorkerMetadataThrottleTimer, + Consumer<WindmillEndpoints> serverMappingUpdater) { + super( + startGetWorkerMetadataRpcFn, + backoff, + streamObserverFactory, + streamRegistry, + logEveryNStreamFailures); + this.workerMetadataRequest = WorkerMetadataRequest.newBuilder().setHeader(jobHeader).build(); + this.metadataVersion = metadataVersion; + this.getWorkerMetadataThrottleTimer = getWorkerMetadataThrottleTimer; + this.serverMappingUpdater = serverMappingUpdater; + } + + public static GrpcGetWorkerMetadataStream create( + Function<StreamObserver<WorkerMetadataResponse>, StreamObserver<WorkerMetadataRequest>> + startGetWorkerMetadataRpcFn, + BackOff backoff, + StreamObserverFactory streamObserverFactory, + Set<AbstractWindmillStream<?, ?>> streamRegistry, + int logEveryNStreamFailures, + JobHeader jobHeader, + AtomicLong metadataVersion, + ThrottleTimer getWorkerMetadataThrottleTimer, + Consumer<WindmillEndpoints> serverMappingUpdater) { + GrpcGetWorkerMetadataStream getWorkerMetadataStream = + new GrpcGetWorkerMetadataStream( + startGetWorkerMetadataRpcFn, + backoff, + streamObserverFactory, + streamRegistry, + logEveryNStreamFailures, + jobHeader, + metadataVersion, + getWorkerMetadataThrottleTimer, + serverMappingUpdater); + getWorkerMetadataStream.startStream(); + return getWorkerMetadataStream; + } + + @VisibleForTesting + static GrpcGetWorkerMetadataStream forTesting( + Function<StreamObserver<WorkerMetadataResponse>, StreamObserver<WorkerMetadataRequest>> + getWorkerMetadataRpc, + int metadataVersion, + JobHeader jobHeader, + Set<AbstractWindmillStream<?, ?>> streamRegistry, + Consumer<WindmillEndpoints> serverMappingUpdater) { + GrpcGetWorkerMetadataStream testStream = + new GrpcGetWorkerMetadataStream( + getWorkerMetadataRpc, + FluentBackoff.DEFAULT.backoff(), + StreamObserverFactory.direct(DEFAULT_STREAM_RPC_DEADLINE_SECONDS * 2, 1), + streamRegistry, + 1, // logEveryNStreamFailures + jobHeader, + new AtomicLong(metadataVersion), + new ThrottleTimer(), + serverMappingUpdater); + testStream.startStream(); + return testStream; + } + + private static long updateMetadataVersion( + long currentMetadataVersion, long responseMetadataVersion) { + if (currentMetadataVersion < responseMetadataVersion) { + return responseMetadataVersion; + } + + // If the currentMetadataVersion is greater than or equal to one in the response, the response + // data is stale, and we do not want to do anything. + LOG.info( + "Received metadata version={}; Current metadata version={}. " + + "Skipping update because received stale metadata", + responseMetadataVersion, + currentMetadataVersion); + return currentMetadataVersion; + } + + @Override + protected void onResponse(WorkerMetadataResponse response) { + long currentMetadataVersion = metadataVersion.get(); + long updatedMetadataVersion = + metadataVersion.updateAndGet( + current -> updateMetadataVersion(current, response.getMetadataVersion())); Review Comment: how about inlining the non-logging part of updateMetadataVersion here, and moving the logging below to else? Seems clearer than logging in the separate method. ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetWorkerMetadataStream.java: ########## @@ -0,0 +1,177 @@ +/* + * 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.grpcclient; + +import java.io.PrintWriter; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; +import java.util.function.Function; +import org.apache.beam.runners.dataflow.worker.windmill.AbstractWindmillStream; +import org.apache.beam.runners.dataflow.worker.windmill.StreamObserverFactory; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.JobHeader; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkerMetadataRequest; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkerMetadataResponse; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillEndpoints; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetWorkerMetadataStream; +import org.apache.beam.sdk.util.BackOff; +import org.apache.beam.sdk.util.FluentBackoff; +import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +final class GrpcGetWorkerMetadataStream + extends AbstractWindmillStream<WorkerMetadataRequest, WorkerMetadataResponse> + implements GetWorkerMetadataStream { + private static final Logger LOG = LoggerFactory.getLogger(GrpcGetWorkerMetadataStream.class); + private static final WorkerMetadataRequest HEALTH_CHECK_REQUEST = + WorkerMetadataRequest.getDefaultInstance(); + + private final AtomicLong metadataVersion; + private final WorkerMetadataRequest workerMetadataRequest; + private final ThrottleTimer getWorkerMetadataThrottleTimer; + private final Consumer<WindmillEndpoints> serverMappingUpdater; + + private GrpcGetWorkerMetadataStream( + Function<StreamObserver<WorkerMetadataResponse>, StreamObserver<WorkerMetadataRequest>> + startGetWorkerMetadataRpcFn, + BackOff backoff, + StreamObserverFactory streamObserverFactory, + Set<AbstractWindmillStream<?, ?>> streamRegistry, + int logEveryNStreamFailures, + JobHeader jobHeader, + AtomicLong metadataVersion, + ThrottleTimer getWorkerMetadataThrottleTimer, + Consumer<WindmillEndpoints> serverMappingUpdater) { + super( + startGetWorkerMetadataRpcFn, + backoff, + streamObserverFactory, + streamRegistry, + logEveryNStreamFailures); + this.workerMetadataRequest = WorkerMetadataRequest.newBuilder().setHeader(jobHeader).build(); + this.metadataVersion = metadataVersion; + this.getWorkerMetadataThrottleTimer = getWorkerMetadataThrottleTimer; + this.serverMappingUpdater = serverMappingUpdater; + } + + public static GrpcGetWorkerMetadataStream create( + Function<StreamObserver<WorkerMetadataResponse>, StreamObserver<WorkerMetadataRequest>> + startGetWorkerMetadataRpcFn, + BackOff backoff, + StreamObserverFactory streamObserverFactory, + Set<AbstractWindmillStream<?, ?>> streamRegistry, + int logEveryNStreamFailures, + JobHeader jobHeader, + AtomicLong metadataVersion, + ThrottleTimer getWorkerMetadataThrottleTimer, + Consumer<WindmillEndpoints> serverMappingUpdater) { + GrpcGetWorkerMetadataStream getWorkerMetadataStream = + new GrpcGetWorkerMetadataStream( + startGetWorkerMetadataRpcFn, + backoff, + streamObserverFactory, + streamRegistry, + logEveryNStreamFailures, + jobHeader, + metadataVersion, + getWorkerMetadataThrottleTimer, + serverMappingUpdater); + getWorkerMetadataStream.startStream(); + return getWorkerMetadataStream; + } + + @VisibleForTesting + static GrpcGetWorkerMetadataStream forTesting( + Function<StreamObserver<WorkerMetadataResponse>, StreamObserver<WorkerMetadataRequest>> + getWorkerMetadataRpc, + int metadataVersion, + JobHeader jobHeader, + Set<AbstractWindmillStream<?, ?>> streamRegistry, + Consumer<WindmillEndpoints> serverMappingUpdater) { + GrpcGetWorkerMetadataStream testStream = + new GrpcGetWorkerMetadataStream( + getWorkerMetadataRpc, + FluentBackoff.DEFAULT.backoff(), + StreamObserverFactory.direct(DEFAULT_STREAM_RPC_DEADLINE_SECONDS * 2, 1), + streamRegistry, + 1, // logEveryNStreamFailures + jobHeader, + new AtomicLong(metadataVersion), + new ThrottleTimer(), + serverMappingUpdater); + testStream.startStream(); + return testStream; + } + + private static long updateMetadataVersion( + long currentMetadataVersion, long responseMetadataVersion) { + if (currentMetadataVersion < responseMetadataVersion) { + return responseMetadataVersion; + } + + // If the currentMetadataVersion is greater than or equal to one in the response, the response + // data is stale, and we do not want to do anything. + LOG.info( + "Received metadata version={}; Current metadata version={}. " + + "Skipping update because received stale metadata", + responseMetadataVersion, + currentMetadataVersion); + return currentMetadataVersion; + } + + @Override + protected void onResponse(WorkerMetadataResponse response) { + long currentMetadataVersion = metadataVersion.get(); + long updatedMetadataVersion = + metadataVersion.updateAndGet( + current -> updateMetadataVersion(current, response.getMetadataVersion())); + + if (updatedMetadataVersion > currentMetadataVersion) { + serverMappingUpdater.accept(WindmillEndpoints.from(response)); + } + } + + @Override + protected synchronized void onNewStream() { + send(workerMetadataRequest); + } + + @Override + protected boolean hasPendingRequests() { + return false; + } + + @Override + protected void startThrottleTimer() { + getWorkerMetadataThrottleTimer.start(); + } + + @Override + protected void sendHealthCheck() { + send(HEALTH_CHECK_REQUEST); + } + + @Override + protected void appendSpecificHtml(PrintWriter writer) { + writer.format( + "GetWorkerMetadataStream: version=[%d] , job_header=[%s]", Review Comment: could show the latest response? could be useful. ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetWorkerMetadataStream.java: ########## @@ -0,0 +1,177 @@ +/* + * 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.grpcclient; + +import java.io.PrintWriter; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; +import java.util.function.Function; +import org.apache.beam.runners.dataflow.worker.windmill.AbstractWindmillStream; +import org.apache.beam.runners.dataflow.worker.windmill.StreamObserverFactory; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.JobHeader; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkerMetadataRequest; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkerMetadataResponse; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillEndpoints; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetWorkerMetadataStream; +import org.apache.beam.sdk.util.BackOff; +import org.apache.beam.sdk.util.FluentBackoff; +import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +final class GrpcGetWorkerMetadataStream + extends AbstractWindmillStream<WorkerMetadataRequest, WorkerMetadataResponse> + implements GetWorkerMetadataStream { + private static final Logger LOG = LoggerFactory.getLogger(GrpcGetWorkerMetadataStream.class); + private static final WorkerMetadataRequest HEALTH_CHECK_REQUEST = + WorkerMetadataRequest.getDefaultInstance(); + + private final AtomicLong metadataVersion; + private final WorkerMetadataRequest workerMetadataRequest; + private final ThrottleTimer getWorkerMetadataThrottleTimer; + private final Consumer<WindmillEndpoints> serverMappingUpdater; + + private GrpcGetWorkerMetadataStream( + Function<StreamObserver<WorkerMetadataResponse>, StreamObserver<WorkerMetadataRequest>> + startGetWorkerMetadataRpcFn, + BackOff backoff, + StreamObserverFactory streamObserverFactory, + Set<AbstractWindmillStream<?, ?>> streamRegistry, + int logEveryNStreamFailures, + JobHeader jobHeader, + AtomicLong metadataVersion, + ThrottleTimer getWorkerMetadataThrottleTimer, + Consumer<WindmillEndpoints> serverMappingUpdater) { + super( + startGetWorkerMetadataRpcFn, + backoff, + streamObserverFactory, + streamRegistry, + logEveryNStreamFailures); + this.workerMetadataRequest = WorkerMetadataRequest.newBuilder().setHeader(jobHeader).build(); + this.metadataVersion = metadataVersion; + this.getWorkerMetadataThrottleTimer = getWorkerMetadataThrottleTimer; + this.serverMappingUpdater = serverMappingUpdater; + } + + public static GrpcGetWorkerMetadataStream create( + Function<StreamObserver<WorkerMetadataResponse>, StreamObserver<WorkerMetadataRequest>> + startGetWorkerMetadataRpcFn, + BackOff backoff, + StreamObserverFactory streamObserverFactory, + Set<AbstractWindmillStream<?, ?>> streamRegistry, + int logEveryNStreamFailures, + JobHeader jobHeader, + AtomicLong metadataVersion, + ThrottleTimer getWorkerMetadataThrottleTimer, + Consumer<WindmillEndpoints> serverMappingUpdater) { + GrpcGetWorkerMetadataStream getWorkerMetadataStream = + new GrpcGetWorkerMetadataStream( + startGetWorkerMetadataRpcFn, + backoff, + streamObserverFactory, + streamRegistry, + logEveryNStreamFailures, + jobHeader, + metadataVersion, + getWorkerMetadataThrottleTimer, + serverMappingUpdater); + getWorkerMetadataStream.startStream(); + return getWorkerMetadataStream; + } + + @VisibleForTesting + static GrpcGetWorkerMetadataStream forTesting( + Function<StreamObserver<WorkerMetadataResponse>, StreamObserver<WorkerMetadataRequest>> + getWorkerMetadataRpc, + int metadataVersion, + JobHeader jobHeader, + Set<AbstractWindmillStream<?, ?>> streamRegistry, + Consumer<WindmillEndpoints> serverMappingUpdater) { + GrpcGetWorkerMetadataStream testStream = + new GrpcGetWorkerMetadataStream( + getWorkerMetadataRpc, + FluentBackoff.DEFAULT.backoff(), + StreamObserverFactory.direct(DEFAULT_STREAM_RPC_DEADLINE_SECONDS * 2, 1), + streamRegistry, + 1, // logEveryNStreamFailures + jobHeader, + new AtomicLong(metadataVersion), + new ThrottleTimer(), + serverMappingUpdater); + testStream.startStream(); + return testStream; + } + + private static long updateMetadataVersion( + long currentMetadataVersion, long responseMetadataVersion) { + if (currentMetadataVersion < responseMetadataVersion) { + return responseMetadataVersion; + } + + // If the currentMetadataVersion is greater than or equal to one in the response, the response + // data is stale, and we do not want to do anything. + LOG.info( + "Received metadata version={}; Current metadata version={}. " + + "Skipping update because received stale metadata", + responseMetadataVersion, + currentMetadataVersion); + return currentMetadataVersion; + } + + @Override + protected void onResponse(WorkerMetadataResponse response) { + long currentMetadataVersion = metadataVersion.get(); + long updatedMetadataVersion = + metadataVersion.updateAndGet( + current -> updateMetadataVersion(current, response.getMetadataVersion())); + + if (updatedMetadataVersion > currentMetadataVersion) { + serverMappingUpdater.accept(WindmillEndpoints.from(response)); Review Comment: Do we need the atomic? I think that there is only one stream calling onResponse and thus those should be ordered. If we did have concurrent callers, this could be racy because you could have current version is 0 T1: atomically updates to 1 T2: atomically updates to 2, calls accept T1: resumes and calls accept then the consumer has last observed the payload for 1. So either: - remove the atomic if it isn't needed and document the thread-safety - or ensure that the ordering is maintained (ie could use synchronized and update version and call accept under it) ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillEndpoints.java: ########## @@ -0,0 +1,112 @@ +/* + * 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.auto.value.AutoValue; +import java.util.Map; +import java.util.Optional; +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; + +/** Value class for holding endpoints used for communicating with Windmill service. */ +@AutoValue +public abstract class WindmillEndpoints { + public static WindmillEndpoints from( + Windmill.WorkerMetadataResponse workerMetadataResponseProto) { + ImmutableMap<String, WindmillEndpoints.Endpoint> globalDataServers = + workerMetadataResponseProto.getGlobalDataEndpointsMap().entrySet().stream() + .collect( + toImmutableMap( + Map.Entry::getKey, // global data key + endpoint -> WindmillEndpoints.Endpoint.from(endpoint.getValue()))); + + ImmutableList<WindmillEndpoints.Endpoint> windmillServers = + workerMetadataResponseProto.getWorkEndpointsList().stream() + .map(WindmillEndpoints.Endpoint::from) + .collect(toImmutableList()); + + return WindmillEndpoints.builder() + .setGlobalDataServers(globalDataServers) + .setWindmillServers(windmillServers) + .build(); + } + + public static WindmillEndpoints.Builder builder() { + return new AutoValue_WindmillEndpoints.Builder(); + } + + /** + * 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 + * data tag resides. + * + * @see <a href="https://beam.apache.org/documentation/programming-guide/#side-inputs">Beam Side + * Inputs</a> + */ + public abstract ImmutableMap<String, Endpoint> globalDataServers(); + + /** + * Used by GetWork/GetData/CommitWork calls to send, receive, and commit work directly to/from + * Windmill servers. Returns a list of endpoints used to communicate with the corresponding + * Windmill servers. + */ + public abstract ImmutableList<Endpoint> windmillServers(); + + @AutoValue + public abstract static class Endpoint { + public static Endpoint.Builder builder() { + return new AutoValue_WindmillEndpoints_Endpoint.Builder(); + } + + public static Endpoint from(Windmill.WorkerMetadataResponse.Endpoint endpointProto) { + Endpoint.Builder endpointBuilder = Endpoint.builder(); + if (endpointProto.hasDirectEndpoint() && !endpointProto.getDirectEndpoint().isEmpty()) { + endpointBuilder.setDirectEndpoint(endpointProto.getDirectEndpoint()); + } + if (endpointProto.hasWorkerToken() && !endpointProto.getWorkerToken().isEmpty()) { + endpointBuilder.setWorkerToken(endpointProto.getWorkerToken()); + } + return endpointBuilder.build(); + } + + public abstract Optional<String> directEndpoint(); Review Comment: is there a better way to represent ipv6 address than String? ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetWorkerMetadataStream.java: ########## @@ -0,0 +1,177 @@ +/* + * 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.grpcclient; + +import java.io.PrintWriter; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; +import java.util.function.Function; +import org.apache.beam.runners.dataflow.worker.windmill.AbstractWindmillStream; +import org.apache.beam.runners.dataflow.worker.windmill.StreamObserverFactory; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.JobHeader; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkerMetadataRequest; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkerMetadataResponse; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillEndpoints; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetWorkerMetadataStream; +import org.apache.beam.sdk.util.BackOff; +import org.apache.beam.sdk.util.FluentBackoff; +import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +final class GrpcGetWorkerMetadataStream + extends AbstractWindmillStream<WorkerMetadataRequest, WorkerMetadataResponse> + implements GetWorkerMetadataStream { + private static final Logger LOG = LoggerFactory.getLogger(GrpcGetWorkerMetadataStream.class); + private static final WorkerMetadataRequest HEALTH_CHECK_REQUEST = + WorkerMetadataRequest.getDefaultInstance(); + + private final AtomicLong metadataVersion; + private final WorkerMetadataRequest workerMetadataRequest; + private final ThrottleTimer getWorkerMetadataThrottleTimer; + private final Consumer<WindmillEndpoints> serverMappingUpdater; + + private GrpcGetWorkerMetadataStream( + Function<StreamObserver<WorkerMetadataResponse>, StreamObserver<WorkerMetadataRequest>> + startGetWorkerMetadataRpcFn, + BackOff backoff, + StreamObserverFactory streamObserverFactory, + Set<AbstractWindmillStream<?, ?>> streamRegistry, + int logEveryNStreamFailures, + JobHeader jobHeader, + AtomicLong metadataVersion, + ThrottleTimer getWorkerMetadataThrottleTimer, + Consumer<WindmillEndpoints> serverMappingUpdater) { + super( + startGetWorkerMetadataRpcFn, + backoff, + streamObserverFactory, + streamRegistry, + logEveryNStreamFailures); + this.workerMetadataRequest = WorkerMetadataRequest.newBuilder().setHeader(jobHeader).build(); + this.metadataVersion = metadataVersion; + this.getWorkerMetadataThrottleTimer = getWorkerMetadataThrottleTimer; + this.serverMappingUpdater = serverMappingUpdater; + } + + public static GrpcGetWorkerMetadataStream create( + Function<StreamObserver<WorkerMetadataResponse>, StreamObserver<WorkerMetadataRequest>> + startGetWorkerMetadataRpcFn, + BackOff backoff, + StreamObserverFactory streamObserverFactory, + Set<AbstractWindmillStream<?, ?>> streamRegistry, + int logEveryNStreamFailures, + JobHeader jobHeader, + AtomicLong metadataVersion, Review Comment: still AtomicLong here ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcWindmillServer.java: ########## @@ -295,10 +294,12 @@ private synchronized CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alph if (stubList.isEmpty()) { throw new RuntimeException("windmillServiceEndpoint has not been set"); } - if (stubList.size() == 1) { - return stubList.get(0); - } - return stubList.get(rand.nextInt(stubList.size())); + + CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1Stub stub = + (stubList.size() == 1 ? stubList.get(0) : stubList.get(rand.nextInt(stubList.size()))); + + return stub.withDeadlineAfter( + AbstractWindmillStream.DEFAULT_STREAM_RPC_DEADLINE_SECONDS, TimeUnit.SECONDS); Review Comment: could name stubWithAbsoluteDeadlineSet to make it obvious to callers instead of a comment. ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetWorkerMetadataStream.java: ########## @@ -0,0 +1,177 @@ +/* + * 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.grpcclient; + +import java.io.PrintWriter; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; +import java.util.function.Function; +import org.apache.beam.runners.dataflow.worker.windmill.AbstractWindmillStream; +import org.apache.beam.runners.dataflow.worker.windmill.StreamObserverFactory; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.JobHeader; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkerMetadataRequest; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkerMetadataResponse; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillEndpoints; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetWorkerMetadataStream; +import org.apache.beam.sdk.util.BackOff; +import org.apache.beam.sdk.util.FluentBackoff; +import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +final class GrpcGetWorkerMetadataStream + extends AbstractWindmillStream<WorkerMetadataRequest, WorkerMetadataResponse> + implements GetWorkerMetadataStream { + private static final Logger LOG = LoggerFactory.getLogger(GrpcGetWorkerMetadataStream.class); + private static final WorkerMetadataRequest HEALTH_CHECK_REQUEST = + WorkerMetadataRequest.getDefaultInstance(); + + private final AtomicLong metadataVersion; + private final WorkerMetadataRequest workerMetadataRequest; + private final ThrottleTimer getWorkerMetadataThrottleTimer; + private final Consumer<WindmillEndpoints> serverMappingUpdater; Review Comment: nit: serverMappingConsumer or serverMappingObserver? Updater makes it sound a little like it is the other direction to me ########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcWindmillServer.java: ########## @@ -295,10 +294,12 @@ private synchronized CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alph if (stubList.isEmpty()) { throw new RuntimeException("windmillServiceEndpoint has not been set"); } - if (stubList.size() == 1) { - return stubList.get(0); - } - return stubList.get(rand.nextInt(stubList.size())); + + CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1Stub stub = + (stubList.size() == 1 ? stubList.get(0) : stubList.get(rand.nextInt(stubList.size()))); + + return stub.withDeadlineAfter( + AbstractWindmillStream.DEFAULT_STREAM_RPC_DEADLINE_SECONDS, TimeUnit.SECONDS); Review Comment: see other cl about withDeadlineAfter either use some other mechanism or document that result of stub() should just be used and not cached. -- 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]
