arunpandianp commented on code in PR #31784:
URL: https://github.com/apache/beam/pull/31784#discussion_r1703666762
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresher.java:
##########
@@ -103,5 +126,67 @@ private void invalidateStuckCommits() {
}
}
- protected abstract void refreshActiveWork();
+ private void refreshActiveWork() {
+ Instant refreshDeadline =
clock.get().minus(Duration.millis(activeWorkRefreshPeriodMillis));
+ Map<HeartbeatSender, Heartbeats> heartbeatsBySender =
+ aggregateHeartbeatsBySender(refreshDeadline);
+
+ List<CompletableFuture<Void>> fanOutRefreshActiveWork = new ArrayList<>();
+
+ // Send the first heartbeat on the calling thread, and fan out the rest
via the
+ // fanOutActiveWorkRefreshExecutor.
+ @Nullable Map.Entry<HeartbeatSender, Heartbeats> firstHeartbeat = null;
+ for (Map.Entry<HeartbeatSender, Heartbeats> heartbeat :
heartbeatsBySender.entrySet()) {
+ if (firstHeartbeat == null) {
+ firstHeartbeat = heartbeat;
+ } else {
+ fanOutRefreshActiveWork.add(
+ CompletableFuture.runAsync(
+ () -> sendHeartbeatSafely(heartbeat),
fanOutActiveWorkRefreshExecutor));
+ }
+ }
+
+ sendHeartbeatSafely(firstHeartbeat);
Review Comment:
firstHeartbeat here will be null, if heartbeatsBySender is empty
I see exceptions like
```
java.lang.NullPointerException: Cannot invoke
"java.util.Map$Entry.getValue()" because "heartbeat" is null
at
org.apache.beam.runners.dataflow.worker.windmill.work.refresh.ActiveWorkRefresher.sendHeartbeatSafely(ActiveWorkRefresher.java:182)
at
org.apache.beam.runners.dataflow.worker.windmill.work.refresh.ActiveWorkRefresher.refreshActiveWork(ActiveWorkRefresher.java:149)
at
org.apache.beam.runners.dataflow.worker.windmill.work.refresh.ActiveWorkRefresher.lambda$start$0(ActiveWorkRefresher.java:94)
at
java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:572)
at
java.base/java.util.concurrent.FutureTask.runAndReset(FutureTask.java:358)
at
java.base/java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:305)
at
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1144)
at
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:642)
at java.base/java.lang.Thread.run(Thread.java:1583)
```
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresher.java:
##########
@@ -103,5 +126,67 @@ private void invalidateStuckCommits() {
}
}
- protected abstract void refreshActiveWork();
+ private void refreshActiveWork() {
+ Instant refreshDeadline =
clock.get().minus(Duration.millis(activeWorkRefreshPeriodMillis));
+ Map<HeartbeatSender, Heartbeats> heartbeatsBySender =
+ aggregateHeartbeatsBySender(refreshDeadline);
+
+ List<CompletableFuture<Void>> fanOutRefreshActiveWork = new ArrayList<>();
+
+ // Send the first heartbeat on the calling thread, and fan out the rest
via the
+ // fanOutActiveWorkRefreshExecutor.
+ @Nullable Map.Entry<HeartbeatSender, Heartbeats> firstHeartbeat = null;
+ for (Map.Entry<HeartbeatSender, Heartbeats> heartbeat :
heartbeatsBySender.entrySet()) {
+ if (firstHeartbeat == null) {
+ firstHeartbeat = heartbeat;
+ } else {
+ fanOutRefreshActiveWork.add(
+ CompletableFuture.runAsync(
+ () -> sendHeartbeatSafely(heartbeat),
fanOutActiveWorkRefreshExecutor));
+ }
+ }
+
+ sendHeartbeatSafely(firstHeartbeat);
Review Comment:
@m-trieu
--
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]