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


##########
runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClientTest.java:
##########
@@ -333,32 +346,35 @@ public void testOnNewWorkerMetadata_redistributesBudget() 
throws InterruptedExce
             .putAllGlobalDataEndpoints(DEFAULT)
             .build();
 
+    List<WorkerMetadataResponse> workerMetadataResponses =
+        Lists.newArrayList(firstWorkerMetadata, secondWorkerMetadata, 
thirdWorkerMetadata);
+
+    TestGetWorkBudgetDistributor getWorkBudgetDistributor =
+        spy(new TestGetWorkBudgetDistributor(workerMetadataResponses.size()));
+    streamingEngineClient =
+        newStreamingEngineClient(
+            GetWorkBudget.builder().setItems(1).setBytes(1).build(),
+            getWorkBudgetDistributor,
+            noOpProcessWorkItemFn());
+
     getWorkerMetadataReady.await();
-    fakeGetWorkerMetadataStub.injectWorkerMetadata(firstWorkerMetadata);
-    Thread.sleep(50);
-    fakeGetWorkerMetadataStub.injectWorkerMetadata(secondWorkerMetadata);
-    Thread.sleep(50);
-    fakeGetWorkerMetadataStub.injectWorkerMetadata(thirdWorkerMetadata);
-    Thread.sleep(50);
-    verify(getWorkBudgetDistributor, atLeast(3)).distributeBudget(any(), 
any());
+
+    // Make sure we are injecting the metadata from smallest to largest.
+    workerMetadataResponses.stream()
+        
.sorted(Comparator.comparingLong(WorkerMetadataResponse::getMetadataVersion).reversed())
+        .forEach(fakeGetWorkerMetadataStub::injectWorkerMetadata);
+
+    waitForWorkerMetadataToBeConsumed(getWorkBudgetDistributor);
+    verify(getWorkBudgetDistributor, atLeast(workerMetadataResponses.size()))
+        .distributeBudget(any(), any());
   }
 
-  private StreamingEngineConnectionState waitForWorkerMetadataToBeConsumed(
-      int expectedMetadataConsumed) throws InterruptedException {
-    int currentMetadataConsumed = 0;
-    StreamingEngineConnectionState currentConsumedMetadata = 
StreamingEngineConnectionState.EMPTY;
-    while (true) {
-      if (!connections.get().equals(currentConsumedMetadata)) {
-        ++currentMetadataConsumed;
-        if (currentMetadataConsumed == expectedMetadataConsumed) {
-          break;
-        }
-        currentConsumedMetadata = connections.get();
-      }
+  private void waitForWorkerMetadataToBeConsumed(
+      TestGetWorkBudgetDistributor getWorkBudgetDistributor) throws 
InterruptedException {
+    getWorkBudgetDistributor.waitForBudgetDistribution();
+    while (isBudgetRefreshPaused.get()) {
+      // wait for budget refresh til budget refresh is unpaused.

Review Comment:
   add a little sleep to not burn cpu and slow test down



##########
runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClientTest.java:
##########
@@ -333,32 +346,35 @@ public void testOnNewWorkerMetadata_redistributesBudget() 
throws InterruptedExce
             .putAllGlobalDataEndpoints(DEFAULT)
             .build();
 
+    List<WorkerMetadataResponse> workerMetadataResponses =
+        Lists.newArrayList(firstWorkerMetadata, secondWorkerMetadata, 
thirdWorkerMetadata);
+
+    TestGetWorkBudgetDistributor getWorkBudgetDistributor =
+        spy(new TestGetWorkBudgetDistributor(workerMetadataResponses.size()));
+    streamingEngineClient =
+        newStreamingEngineClient(
+            GetWorkBudget.builder().setItems(1).setBytes(1).build(),
+            getWorkBudgetDistributor,
+            noOpProcessWorkItemFn());
+
     getWorkerMetadataReady.await();
-    fakeGetWorkerMetadataStub.injectWorkerMetadata(firstWorkerMetadata);
-    Thread.sleep(50);
-    fakeGetWorkerMetadataStub.injectWorkerMetadata(secondWorkerMetadata);
-    Thread.sleep(50);
-    fakeGetWorkerMetadataStub.injectWorkerMetadata(thirdWorkerMetadata);
-    Thread.sleep(50);
-    verify(getWorkBudgetDistributor, atLeast(3)).distributeBudget(any(), 
any());
+
+    // Make sure we are injecting the metadata from smallest to largest.
+    workerMetadataResponses.stream()
+        
.sorted(Comparator.comparingLong(WorkerMetadataResponse::getMetadataVersion).reversed())
+        .forEach(fakeGetWorkerMetadataStub::injectWorkerMetadata);
+
+    waitForWorkerMetadataToBeConsumed(getWorkBudgetDistributor);
+    verify(getWorkBudgetDistributor, atLeast(workerMetadataResponses.size()))
+        .distributeBudget(any(), any());
   }
 
-  private StreamingEngineConnectionState waitForWorkerMetadataToBeConsumed(
-      int expectedMetadataConsumed) throws InterruptedException {
-    int currentMetadataConsumed = 0;
-    StreamingEngineConnectionState currentConsumedMetadata = 
StreamingEngineConnectionState.EMPTY;
-    while (true) {
-      if (!connections.get().equals(currentConsumedMetadata)) {
-        ++currentMetadataConsumed;
-        if (currentMetadataConsumed == expectedMetadataConsumed) {
-          break;
-        }
-        currentConsumedMetadata = connections.get();
-      }
+  private void waitForWorkerMetadataToBeConsumed(
+      TestGetWorkBudgetDistributor getWorkBudgetDistributor) throws 
InterruptedException {
+    getWorkBudgetDistributor.waitForBudgetDistribution();
+    while (isBudgetRefreshPaused.get()) {
+      // wait for budget refresh til budget refresh is unpaused.

Review Comment:
   also can you improve comment on why this is necessary?
   It seems we pause during updating the endpoints. But we set that to false 
before we trigger the refresher.



##########
runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClientTest.java:
##########
@@ -412,10 +428,22 @@ private void close() {
   }
 
   private static class TestGetWorkBudgetDistributor implements 
GetWorkBudgetDistributor {
+    private final CountDownLatch getWorkBudgetDistributorTriggered;
+
+    private TestGetWorkBudgetDistributor(int numBudgetDistributionsExpected) {
+      this.getWorkBudgetDistributorTriggered = new 
CountDownLatch(numBudgetDistributionsExpected);
+    }
+
+    @SuppressWarnings("ReturnValueIgnored")
+    private void waitForBudgetDistribution() throws InterruptedException {
+      getWorkBudgetDistributorTriggered.await(5, TimeUnit.SECONDS);
+    }
+
     @Override
     public void distributeBudget(
         ImmutableCollection<WindmillStreamSender> streams, GetWorkBudget 
getWorkBudget) {
       streams.forEach(stream -> stream.adjustBudget(getWorkBudget.items(), 
getWorkBudget.bytes()));
+      getWorkBudgetDistributorTriggered.countDown();

Review Comment:
   does this throw if we decrement too much? if so with the schedled test it 
seems racy if it happens too often.  Maybe could check before decrementing here 
since it is single-threaded distributing.



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