m-trieu commented on code in PR #27767:
URL: https://github.com/apache/beam/pull/27767#discussion_r1280088296


##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetWorkerMetadataStream.java:
##########
@@ -0,0 +1,285 @@
+/*
+ * 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 static 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList.toImmutableList;
+import static 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap.toImmutableMap;
+
+import java.io.PrintWriter;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+import javax.annotation.Nullable;
+import org.apache.beam.runners.dataflow.worker.windmill.AbstractWindmillStream;
+import 
org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc;
+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.WindmillServerStub.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.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class GrpcGetWorkerMetadataStream
+    extends AbstractWindmillStream<WorkerMetadataRequest, 
WorkerMetadataResponse>
+    implements GetWorkerMetadataStream {
+  private static final Logger LOG = 
LoggerFactory.getLogger(GrpcGetWorkerMetadataStream.class);
+
+  private final AtomicLong metadataVersion;
+  private final WorkerMetadataRequest workerMetadataRequest;
+  private final ThrottleTimer getWorkerMetadataThrottleTimer;
+  private final GetWorkMetadataStreamRequestObserver 
getWorkMetadataStreamRequestObserver;
+  private final Consumer<WindmillEndpoints> serverMappingUpdater;
+
+  private GrpcGetWorkerMetadataStream(
+      CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1Stub stub,
+      BackOff backoff,
+      StreamObserverFactory streamObserverFactory,
+      Set<AbstractWindmillStream<?, ?>> streamRegistry,
+      int logEveryNStreamFailures,
+      JobHeader jobHeader,
+      AtomicLong metadataVersion,
+      ThrottleTimer getWorkerMetadataThrottleTimer,
+      Consumer<WindmillEndpoints> serverMappingUpdater) {
+    super(
+        // This is a client unary request, server stream response RPC. There 
is no need to
+        // maintain the client side stream.
+        responseObserver -> null,
+        backoff,
+        streamObserverFactory,
+        streamRegistry,
+        logEveryNStreamFailures);
+    this.getWorkMetadataStreamRequestObserver =
+        GetWorkMetadataStreamRequestObserver.newUnopenedStream(stub);
+    this.workerMetadataRequest = 
WorkerMetadataRequest.newBuilder().setHeader(jobHeader).build();
+    this.metadataVersion = metadataVersion;
+    this.getWorkerMetadataThrottleTimer = getWorkerMetadataThrottleTimer;
+    this.serverMappingUpdater = serverMappingUpdater;
+  }
+
+  public static GrpcGetWorkerMetadataStream create(
+      CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1Stub stub,
+      BackOff backoff,
+      StreamObserverFactory streamObserverFactory,
+      Set<AbstractWindmillStream<?, ?>> streamRegistry,
+      int logEveryNStreamFailures,
+      JobHeader jobHeader,
+      AtomicLong metadataVersion,
+      ThrottleTimer getWorkerMetadataThrottleTimer,
+      Consumer<WindmillEndpoints> serverMappingUpdater) {
+    GrpcGetWorkerMetadataStream getWorkerMetadataStream =
+        new GrpcGetWorkerMetadataStream(
+            stub,
+            backoff,
+            streamObserverFactory,
+            streamRegistry,
+            logEveryNStreamFailures,
+            jobHeader,
+            metadataVersion,
+            getWorkerMetadataThrottleTimer,
+            serverMappingUpdater);
+    getWorkerMetadataStream.fullyInitializeRequestObserver();
+    getWorkerMetadataStream.startStream();
+    return getWorkerMetadataStream;
+  }
+
+  @VisibleForTesting
+  static GrpcGetWorkerMetadataStream forTesting(
+      CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1Stub stub,
+      AtomicLong metadataVersion,
+      JobHeader jobHeader,
+      Set<AbstractWindmillStream<?, ?>> streamRegistry,
+      Consumer<WindmillEndpoints> serverMappingUpdater) {
+    GrpcGetWorkerMetadataStream testStream =
+        new GrpcGetWorkerMetadataStream(
+            stub,
+            FluentBackoff.DEFAULT.backoff(),
+            
StreamObserverFactory.serverStream(DEFAULT_STREAM_RPC_DEADLINE_SECONDS * 2),
+            streamRegistry,
+            1, // logEveryNStreamFailures
+            jobHeader,
+            metadataVersion,
+            new ThrottleTimer(),
+            serverMappingUpdater);
+    testStream.fullyInitializeRequestObserver();
+    testStream.startStream();
+    return testStream;
+  }
+
+  /**
+   * Because {@link AbstractWindmillStream.ResponseObserver} is a non-static 
inner class, it (and
+   * its members) cannot be referenced until the constructor finishes 
executing and the object is
+   * completely instantiated. We gate this from being null by making the 
constructor private and
+   * exposing a static method so that an instance of {@link 
GrpcGetWorkerMetadataStream} will be
+   * created in the correct state.
+   */
+  private void fullyInitializeRequestObserver() {
+    getWorkMetadataStreamRequestObserver.init(
+        AbstractWindmillStream.ResponseObserver::new,
+        () -> {
+          synchronized (GrpcGetWorkerMetadataStream.this) {
+            streamClosed.set(true);
+          }
+        });
+  }
+
+  @Override
+  protected StreamObserver<WorkerMetadataRequest> createNewRequestObserver() {
+    // we do not need to re-initiate the requestObserver here, since every
+    // GetWorkMetadataStreamRequestObserver.onNext() starts a new server 
stream.
+    return getWorkMetadataStreamRequestObserver;
+  }
+
+  @Override
+  protected void onResponse(WorkerMetadataResponse response) {
+    synchronized (this) {
+      // If the metadataVersion newer or the same as the one in the response, 
the response data is
+      // stale, and we do not want to do anything.
+      long currentMetadataVersion = metadataVersion.get();
+      if (currentMetadataVersion >= response.getMetadataVersion()) {
+        LOG.info(
+            "Received metadata version={}; Current metadata version={}. "
+                + "Skipping update because received stale metadata",
+            response.getMetadataVersion(),
+            currentMetadataVersion);
+        return;
+      }
+      metadataVersion.set(response.getMetadataVersion());
+    }
+    ImmutableMap<String, String> globalDataServers =
+        response.getGlobalDataEndpointsMap().entrySet().stream()
+            .collect(
+                toImmutableMap(
+                    Map.Entry::getKey, // global data key
+                    endpoint -> endpoint.getValue().getEndpoint()));
+
+    ImmutableList<String> windmillServers =
+        response.getDirectPathEndpointsList().stream()
+            .map(WorkerMetadataResponse.Endpoint::getEndpoint)
+            .collect(toImmutableList());
+
+    serverMappingUpdater.accept(
+        WindmillEndpoints.builder()
+            .setGlobalDataServers(globalDataServers)
+            .setWindmillServers(windmillServers)
+            .build());
+  }
+
+  @Override
+  protected synchronized void onNewStream() {
+    send(workerMetadataRequest);
+  }
+
+  @Override
+  protected void startThrottleTimer() {
+    getWorkerMetadataThrottleTimer.start();

Review Comment:
   @scwhittle are these throttle timers for client side throttling? if yes, i 
don't think we need them here.



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