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


##########
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:
   done. added locking



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