scwhittle commented on code in PR #27767:
URL: https://github.com/apache/beam/pull/27767#discussion_r1312770471


##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetWorkerMetadataStream.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.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 WorkerMetadataRequest workerMetadataRequest;
+  private final ThrottleTimer getWorkerMetadataThrottleTimer;
+  private final Consumer<WindmillEndpoints> serverMappingConsumer;
+  private long metadataVersion;
+  private WorkerMetadataResponse latestResponse;
+
+  private GrpcGetWorkerMetadataStream(
+      Function<StreamObserver<WorkerMetadataResponse>, 
StreamObserver<WorkerMetadataRequest>>
+          startGetWorkerMetadataRpcFn,
+      BackOff backoff,
+      StreamObserverFactory streamObserverFactory,
+      Set<AbstractWindmillStream<?, ?>> streamRegistry,
+      int logEveryNStreamFailures,
+      JobHeader jobHeader,
+      long metadataVersion,
+      ThrottleTimer getWorkerMetadataThrottleTimer,
+      Consumer<WindmillEndpoints> serverMappingConsumer) {
+    super(
+        startGetWorkerMetadataRpcFn,
+        backoff,
+        streamObserverFactory,
+        streamRegistry,
+        logEveryNStreamFailures);
+    this.workerMetadataRequest = 
WorkerMetadataRequest.newBuilder().setHeader(jobHeader).build();
+    this.metadataVersion = metadataVersion;
+    this.getWorkerMetadataThrottleTimer = getWorkerMetadataThrottleTimer;
+    this.serverMappingConsumer = serverMappingConsumer;
+    this.latestResponse = WorkerMetadataResponse.getDefaultInstance();
+  }
+
+  public static GrpcGetWorkerMetadataStream create(
+      Function<StreamObserver<WorkerMetadataResponse>, 
StreamObserver<WorkerMetadataRequest>>
+          startGetWorkerMetadataRpcFn,
+      BackOff backoff,
+      StreamObserverFactory streamObserverFactory,
+      Set<AbstractWindmillStream<?, ?>> streamRegistry,
+      int logEveryNStreamFailures,
+      JobHeader jobHeader,
+      int 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,
+            metadataVersion,
+            new ThrottleTimer(),
+            serverMappingUpdater);
+    testStream.startStream();
+    return testStream;
+  }
+
+  /**
+   * Each instance of {@link AbstractWindmillStream} owns its own 
responseObserver that calls
+   * onResponse(). This method does not and should not ever have any 
concurrent callers.
+   */
+  @Override
+  protected void onResponse(WorkerMetadataResponse response) {
+    if (response.getMetadataVersion() > metadataVersion) {
+      metadataVersion = response.getMetadataVersion();
+      serverMappingConsumer.accept(WindmillEndpoints.from(response));
+    } else {
+      // 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",
+          response.getMetadataVersion(),
+          metadataVersion);
+    }
+
+    this.latestResponse = response;

Review Comment:
   don't update lastResponse in the case it is stale



##########
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:
   Looking again, the AbstractWindmillStream are used for longer than the 
deadline, they reconnect grpc streams internally.  So I think you need to use 
the observer approach and not a fixed deadline.



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillEndpoints.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.net.Inet6Address;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+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;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Value class for holding endpoints used for communicating with Windmill 
service. */
+@AutoValue
+public abstract class WindmillEndpoints {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(WindmillEndpoints.class);
+
+  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()) {
+        parseDirectEndpoint(endpointProto.getDirectEndpoint())
+            .ifPresent(endpointBuilder::setDirectEndpoint);
+      }
+      if (endpointProto.hasWorkerToken() && 
!endpointProto.getWorkerToken().isEmpty()) {
+        endpointBuilder.setWorkerToken(endpointProto.getWorkerToken());
+      }
+
+      Endpoint endpoint = endpointBuilder.build();
+
+      if (!endpoint.directEndpoint().isPresent() && 
!endpoint.workerToken().isPresent()) {
+        throw new IllegalArgumentException(
+            String.format(
+                "direct_endpoint=[%s] not present or could not be parsed, and 
worker_token"
+                    + " not present. At least one of these fields is 
required.",
+                endpointProto.getDirectEndpoint()));
+      }
+
+      return endpoint;
+    }
+
+    private static Optional<Inet6Address> parseDirectEndpoint(String 
directEndpoint) {
+      InetAddress directEndpointAddress = null;
+      try {
+        directEndpointAddress = Inet6Address.getByName(directEndpoint);
+      } catch (UnknownHostException e) {
+        LOG.warn(
+            "Error occurred trying to parse direct_endpoint={} into IPv6 
address. Exception={}",
+            directEndpoint,
+            e.toString());
+      }
+
+      // Inet6Address.getByAddress returns either an IPv4 or an IPv6 address 
depending on the format
+      // of the direct_endpoint string.
+      if (!(directEndpointAddress instanceof Inet6Address)) {
+        LOG.warn(
+            "{} is not an IPv6 address. Direct endpoints are expected to be in 
IPv6 format.",
+            directEndpoint);
+      }
+
+      return directEndpointAddress == null
+          ? Optional.empty()
+          : Optional.of((Inet6Address) directEndpointAddress);

Review Comment:
   this cast seems wrong if the above instanceof fails.
   seems we should return empty in that case
   
   since null will be false for instanceof you could just do a single check 
using that.



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetWorkerMetadataStream.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.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 WorkerMetadataRequest workerMetadataRequest;
+  private final ThrottleTimer getWorkerMetadataThrottleTimer;
+  private final Consumer<WindmillEndpoints> serverMappingConsumer;
+  private long metadataVersion;
+  private WorkerMetadataResponse latestResponse;
+
+  private GrpcGetWorkerMetadataStream(
+      Function<StreamObserver<WorkerMetadataResponse>, 
StreamObserver<WorkerMetadataRequest>>
+          startGetWorkerMetadataRpcFn,
+      BackOff backoff,
+      StreamObserverFactory streamObserverFactory,
+      Set<AbstractWindmillStream<?, ?>> streamRegistry,
+      int logEveryNStreamFailures,
+      JobHeader jobHeader,
+      long metadataVersion,
+      ThrottleTimer getWorkerMetadataThrottleTimer,
+      Consumer<WindmillEndpoints> serverMappingConsumer) {
+    super(
+        startGetWorkerMetadataRpcFn,
+        backoff,
+        streamObserverFactory,
+        streamRegistry,
+        logEveryNStreamFailures);
+    this.workerMetadataRequest = 
WorkerMetadataRequest.newBuilder().setHeader(jobHeader).build();
+    this.metadataVersion = metadataVersion;
+    this.getWorkerMetadataThrottleTimer = getWorkerMetadataThrottleTimer;
+    this.serverMappingConsumer = serverMappingConsumer;
+    this.latestResponse = WorkerMetadataResponse.getDefaultInstance();
+  }
+
+  public static GrpcGetWorkerMetadataStream create(
+      Function<StreamObserver<WorkerMetadataResponse>, 
StreamObserver<WorkerMetadataRequest>>
+          startGetWorkerMetadataRpcFn,
+      BackOff backoff,
+      StreamObserverFactory streamObserverFactory,
+      Set<AbstractWindmillStream<?, ?>> streamRegistry,
+      int logEveryNStreamFailures,
+      JobHeader jobHeader,
+      int 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,
+            metadataVersion,
+            new ThrottleTimer(),
+            serverMappingUpdater);
+    testStream.startStream();
+    return testStream;
+  }
+
+  /**
+   * Each instance of {@link AbstractWindmillStream} owns its own 
responseObserver that calls
+   * onResponse(). This method does not and should not ever have any 
concurrent callers.
+   */
+  @Override
+  protected void onResponse(WorkerMetadataResponse response) {
+    if (response.getMetadataVersion() > metadataVersion) {
+      metadataVersion = response.getMetadataVersion();
+      serverMappingConsumer.accept(WindmillEndpoints.from(response));
+    } else {
+      // 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",
+          response.getMetadataVersion(),
+          metadataVersion);
+    }
+
+    this.latestResponse = 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], 
latest_response=[%s]",
+        metadataVersion, workerMetadataRequest.getHeader(), latestResponse);

Review Comment:
   I think you need some synchronization for latestResponse etc since I believe 
the html rendering can be concurrent



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetWorkerMetadataStream.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.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 WorkerMetadataRequest workerMetadataRequest;
+  private final ThrottleTimer getWorkerMetadataThrottleTimer;
+  private final Consumer<WindmillEndpoints> serverMappingConsumer;
+  private long metadataVersion;
+  private WorkerMetadataResponse latestResponse;
+
+  private GrpcGetWorkerMetadataStream(
+      Function<StreamObserver<WorkerMetadataResponse>, 
StreamObserver<WorkerMetadataRequest>>
+          startGetWorkerMetadataRpcFn,
+      BackOff backoff,
+      StreamObserverFactory streamObserverFactory,
+      Set<AbstractWindmillStream<?, ?>> streamRegistry,
+      int logEveryNStreamFailures,
+      JobHeader jobHeader,
+      long metadataVersion,
+      ThrottleTimer getWorkerMetadataThrottleTimer,
+      Consumer<WindmillEndpoints> serverMappingConsumer) {
+    super(
+        startGetWorkerMetadataRpcFn,
+        backoff,
+        streamObserverFactory,
+        streamRegistry,
+        logEveryNStreamFailures);
+    this.workerMetadataRequest = 
WorkerMetadataRequest.newBuilder().setHeader(jobHeader).build();
+    this.metadataVersion = metadataVersion;
+    this.getWorkerMetadataThrottleTimer = getWorkerMetadataThrottleTimer;
+    this.serverMappingConsumer = serverMappingConsumer;
+    this.latestResponse = WorkerMetadataResponse.getDefaultInstance();
+  }
+
+  public static GrpcGetWorkerMetadataStream create(
+      Function<StreamObserver<WorkerMetadataResponse>, 
StreamObserver<WorkerMetadataRequest>>
+          startGetWorkerMetadataRpcFn,
+      BackOff backoff,
+      StreamObserverFactory streamObserverFactory,
+      Set<AbstractWindmillStream<?, ?>> streamRegistry,
+      int logEveryNStreamFailures,
+      JobHeader jobHeader,
+      int 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,
+            metadataVersion,
+            new ThrottleTimer(),
+            serverMappingUpdater);
+    testStream.startStream();
+    return testStream;
+  }
+
+  /**
+   * Each instance of {@link AbstractWindmillStream} owns its own 
responseObserver that calls
+   * onResponse(). This method does not and should not ever have any 
concurrent callers.
+   */
+  @Override
+  protected void onResponse(WorkerMetadataResponse response) {
+    if (response.getMetadataVersion() > metadataVersion) {
+      metadataVersion = response.getMetadataVersion();
+      serverMappingConsumer.accept(WindmillEndpoints.from(response));

Review Comment:
   if you add sychronization for html rendering (see below), it would be good 
to keep the accept call out of the synchronized block.



##########
runners/google-cloud-dataflow-java/worker/windmill/src/main/proto/windmill.proto:
##########
@@ -758,18 +758,15 @@ message WorkerMetadataResponse {
   // CommitWorkStream. Each response on this stream replaces the previous, and
   // connections to endpoints that are no longer present should be closed.
   message Endpoint {
-    optional string endpoint = 1;
+    // IPv6 address of a streaming engine windmill worker.
+    optional string direct_endpoint = 1;
+    optional string worker_token = 2;
   }
   repeated Endpoint work_endpoints = 2;
 
   // Maps from GlobalData tag to the endpoint that should be used for GetData
   // calls to retrieve that global data.
   map<string, Endpoint> global_data_endpoints = 3;
-
-  // DirectPath endpoints to be used by user workers for streaming engine jobs.
-  // DirectPath endpoints here are virtual IPv6 addresses of the windmill
-  // workers.
-  repeated Endpoint direct_path_endpoints = 4;

Review Comment:
   reserve the tag # so it is not reused
   
   reserved 4;



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetWorkerMetadataStream.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.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 WorkerMetadataRequest workerMetadataRequest;
+  private final ThrottleTimer getWorkerMetadataThrottleTimer;
+  private final Consumer<WindmillEndpoints> serverMappingConsumer;
+  private long metadataVersion;
+  private WorkerMetadataResponse latestResponse;
+
+  private GrpcGetWorkerMetadataStream(
+      Function<StreamObserver<WorkerMetadataResponse>, 
StreamObserver<WorkerMetadataRequest>>
+          startGetWorkerMetadataRpcFn,
+      BackOff backoff,
+      StreamObserverFactory streamObserverFactory,
+      Set<AbstractWindmillStream<?, ?>> streamRegistry,
+      int logEveryNStreamFailures,
+      JobHeader jobHeader,
+      long metadataVersion,
+      ThrottleTimer getWorkerMetadataThrottleTimer,
+      Consumer<WindmillEndpoints> serverMappingConsumer) {
+    super(
+        startGetWorkerMetadataRpcFn,
+        backoff,
+        streamObserverFactory,
+        streamRegistry,
+        logEveryNStreamFailures);
+    this.workerMetadataRequest = 
WorkerMetadataRequest.newBuilder().setHeader(jobHeader).build();
+    this.metadataVersion = metadataVersion;
+    this.getWorkerMetadataThrottleTimer = getWorkerMetadataThrottleTimer;
+    this.serverMappingConsumer = serverMappingConsumer;
+    this.latestResponse = WorkerMetadataResponse.getDefaultInstance();
+  }
+
+  public static GrpcGetWorkerMetadataStream create(
+      Function<StreamObserver<WorkerMetadataResponse>, 
StreamObserver<WorkerMetadataRequest>>
+          startGetWorkerMetadataRpcFn,
+      BackOff backoff,
+      StreamObserverFactory streamObserverFactory,
+      Set<AbstractWindmillStream<?, ?>> streamRegistry,
+      int logEveryNStreamFailures,
+      JobHeader jobHeader,
+      int 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,
+            metadataVersion,
+            new ThrottleTimer(),
+            serverMappingUpdater);
+    testStream.startStream();
+    return testStream;
+  }
+
+  /**
+   * Each instance of {@link AbstractWindmillStream} owns its own 
responseObserver that calls
+   * onResponse(). This method does not and should not ever have any 
concurrent callers.
+   */
+  @Override
+  protected void onResponse(WorkerMetadataResponse response) {
+    if (response.getMetadataVersion() > metadataVersion) {
+      metadataVersion = response.getMetadataVersion();
+      serverMappingConsumer.accept(WindmillEndpoints.from(response));

Review Comment:
   From AbstractStream header it sounds like you should not synchronize on this 
but some other object:
   
    * <p>Synchronization on this is used to synchronize the gRpc stream state 
and internal data
    * structures. Since grpc channel operations may block, synchronization on 
this stream may also
    * block. This is generally not a problem since streams are used in a 
single-threaded manner.
    * However, some accessors used for status page and other debugging need to 
take care not to require
    * synchronizing on this.



-- 
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]


Reply via email to