arne-alex commented on code in PR #23333:
URL: https://github.com/apache/beam/pull/23333#discussion_r997818696
##########
runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java:
##########
@@ -3081,6 +3146,283 @@ public void testActiveWorkRefresh() throws Exception {
assertThat(server.numGetDataRequests(), greaterThan(0));
}
+ static class FakeClock implements Supplier<Instant> {
+ private static final FakeClock DEFAULT = new FakeClock();
+
+ private Instant now = Instant.now();
+
+ @Override
+ public synchronized Instant get() {
+ return now;
+ }
+
+ synchronized void sleep(Duration duration) {
+ this.now = this.now.plus(duration);
+ }
+ }
+
+ @Test
+ public void testLatencyAttributionProtobufsPopulated() throws Exception {
+ StreamingDataflowWorker.Work work =
+ new StreamingDataflowWorker.Work(null, FakeClock.DEFAULT) {
+ @Override
+ public void run() {}
+ };
+
+ FakeClock.DEFAULT.sleep(Duration.millis(10));
+ work.setState(StreamingDataflowWorker.Work.State.PROCESSING);
+ FakeClock.DEFAULT.sleep(Duration.millis(20));
+ work.setState(StreamingDataflowWorker.Work.State.COMMIT_QUEUED);
+ FakeClock.DEFAULT.sleep(Duration.millis(30));
+
+ Iterator<LatencyAttribution> it =
work.getLatencyAttributionList().iterator();
+ assertTrue(it.hasNext());
+ LatencyAttribution lat = it.next();
+ assertTrue(lat.getState() == LatencyAttribution.State.QUEUED);
+ assertTrue(lat.getTotalDurationMillis() == 10);
+ assertTrue(it.hasNext());
+ lat = it.next();
+ assertTrue(lat.getState() == LatencyAttribution.State.ACTIVE);
+ assertTrue(lat.getTotalDurationMillis() == 20);
+ assertTrue(it.hasNext());
+ lat = it.next();
+ assertTrue(lat.getState() == LatencyAttribution.State.COMMITTING);
+ assertTrue(lat.getTotalDurationMillis() == 30);
+ assertTrue(!it.hasNext());
+ }
+
+ // Aggregates LatencyAttribution data from active work refresh requests.
+ static class ActiveWorkRefreshSink {
+ private final Function<GetDataRequest, GetDataResponse> responder;
+ private final Map<Long, EnumMap<LatencyAttribution.State, Duration>>
totalDurations =
+ new HashMap<>();
+
+ ActiveWorkRefreshSink(Function<GetDataRequest, GetDataResponse> responder)
{
+ this.responder = responder;
+ }
+
+ Duration getLatencyAttributionDuration(long workToken,
LatencyAttribution.State state) {
+ EnumMap<LatencyAttribution.State, Duration> durations =
totalDurations.get(workToken);
+ return durations == null ? Duration.ZERO : durations.getOrDefault(state,
Duration.ZERO);
+ }
+
+ boolean isActiveWorkRefresh(GetDataRequest request) {
+ for (ComputationGetDataRequest computationRequest :
request.getRequestsList()) {
+ if
(!computationRequest.getComputationId().equals(DEFAULT_COMPUTATION_ID)) {
+ return false;
+ }
+ for (KeyedGetDataRequest keyedRequest :
computationRequest.getRequestsList()) {
+ if (keyedRequest.getWorkToken() == 0
+ || keyedRequest.getShardingKey() != DEFAULT_SHARDING_KEY
+ || keyedRequest.getValuesToFetchCount() != 0
+ || keyedRequest.getBagsToFetchCount() != 0
+ || keyedRequest.getTagValuePrefixesToFetchCount() != 0
+ || keyedRequest.getWatermarkHoldsToFetchCount() != 0) {
+ return false;
+ }
+ }
+ }
+ return true;
+ }
+
+ GetDataResponse getData(GetDataRequest request) {
+ if (!isActiveWorkRefresh(request)) {
+ return responder.apply(request);
+ }
+ for (ComputationGetDataRequest computationRequest :
request.getRequestsList()) {
+ for (KeyedGetDataRequest keyedRequest :
computationRequest.getRequestsList()) {
+ for (LatencyAttribution la :
keyedRequest.getLatencyAttributionList()) {
+ EnumMap<LatencyAttribution.State, Duration> durations =
+ totalDurations.computeIfAbsent(
+ keyedRequest.getWorkToken(),
+ (Long workToken) ->
+ new EnumMap<LatencyAttribution.State, Duration>(
+ LatencyAttribution.State.class));
+ Duration cur = Duration.millis(la.getTotalDurationMillis());
+ durations.compute(la.getState(), (s, d) -> d == null ||
d.isShorterThan(cur) ? cur : d);
+ }
+ }
+ }
+ return EMPTY_DATA_RESPONDER.apply(request);
+ }
+ }
+
+ @Test
+ public void testLatencyAttributionToQueuedState() throws Exception {
+ final int workToken = 3232; // A unique id makes it easier to search logs.
+
+ List<ParallelInstruction> instructions =
+ Arrays.asList(
+ makeSourceInstruction(StringUtf8Coder.of()),
+ makeDoFnInstruction(new SlowDoFn(Duration.millis(1000)), 0,
StringUtf8Coder.of()),
+ makeSinkInstruction(StringUtf8Coder.of(), 0));
+
+ FakeWindmillServer server = new FakeWindmillServer(errorCollector);
+ StreamingDataflowWorkerOptions options =
createTestingPipelineOptions(server);
+ options.setActiveWorkRefreshPeriodMillis(100);
Review Comment:
The latest commit adds a fake ScheduledExecutorService, so that new unit
tests no longer depend on the wall-time clock.
--
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]