Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[Dataflow Streaming] Optimizing work submit loop #33736

Merged
merged 4 commits into from
Jan 24, 2025
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@

import com.google.auto.value.AutoValue;
import java.util.Arrays;
import java.util.Collection;
import java.util.EnumMap;
import java.util.IntSummaryStatistics;
import java.util.Map;
Expand All @@ -41,6 +40,7 @@
import org.apache.beam.runners.dataflow.worker.windmill.Windmill.LatencyAttribution.ActiveLatencyBreakdown;
import org.apache.beam.runners.dataflow.worker.windmill.Windmill.LatencyAttribution.ActiveLatencyBreakdown.ActiveElementMetadata;
import org.apache.beam.runners.dataflow.worker.windmill.Windmill.LatencyAttribution.ActiveLatencyBreakdown.Distribution;
import org.apache.beam.runners.dataflow.worker.windmill.Windmill.LatencyAttribution.State;
import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItem;
import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItemCommitRequest;
import org.apache.beam.runners.dataflow.worker.windmill.client.commits.Commit;
Expand All @@ -49,6 +49,7 @@
import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateReader;
import org.apache.beam.runners.dataflow.worker.windmill.work.refresh.HeartbeatSender;
import org.apache.beam.sdk.annotations.Internal;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
import org.joda.time.Duration;
import org.joda.time.Instant;
Expand All @@ -61,6 +62,9 @@
@NotThreadSafe
@Internal
public final class Work implements RefreshableWork {

private static final EnumMap<LatencyAttribution.State, Duration> EMPTY_ENUM_MAP =
new EnumMap<>(LatencyAttribution.State.class);
private final ShardedKey shardedKey;
private final WorkItem workItem;
private final ProcessingContext processingContext;
Expand Down Expand Up @@ -88,7 +92,10 @@ private Work(
this.watermarks = watermarks;
this.clock = clock;
this.startTime = clock.get();
this.totalDurationPerState = new EnumMap<>(LatencyAttribution.State.class);
Preconditions.checkState(EMPTY_ENUM_MAP.isEmpty());
// Create by passing EMPTY_ENUM_MAP to avoid recreating
// keyUniverse inside EnumMap every time.
this.totalDurationPerState = new EnumMap<>(EMPTY_ENUM_MAP);
this.id = WorkId.of(workItem);
this.latencyTrackingId =
Long.toHexString(workItem.getShardingKey())
Expand All @@ -103,11 +110,8 @@ public static Work create(
long serializedWorkItemSize,
Watermarks watermarks,
ProcessingContext processingContext,
Supplier<Instant> clock,
Collection<LatencyAttribution> getWorkStreamLatencies) {
Work work = new Work(workItem, serializedWorkItemSize, watermarks, processingContext, clock);
work.recordGetWorkStreamLatencies(getWorkStreamLatencies);
return work;
Supplier<Instant> clock) {
return new Work(workItem, serializedWorkItemSize, watermarks, processingContext, clock);
}

public static ProcessingContext createProcessingContext(
Expand Down Expand Up @@ -256,7 +260,8 @@ public WorkId id() {
return id;
}

private void recordGetWorkStreamLatencies(Collection<LatencyAttribution> getWorkStreamLatencies) {
public void recordGetWorkStreamLatencies(
ImmutableList<LatencyAttribution> getWorkStreamLatencies) {
for (LatencyAttribution latency : getWorkStreamLatencies) {
totalDurationPerState.put(
latency.getState(), Duration.millis(latency.getTotalDurationMillis()));
Expand Down Expand Up @@ -350,6 +355,7 @@ LatencyAttribution.State toLatencyAttributionState() {
*/
@AutoValue
abstract static class TimedState {

private static TimedState create(State state, Instant startTime) {
return new AutoValue_Work_TimedState(state, startTime);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@

import com.google.auto.value.AutoBuilder;
import com.google.auto.value.AutoOneOf;
import java.util.Collections;
import java.util.Optional;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
Expand All @@ -44,6 +43,7 @@
import org.apache.beam.sdk.annotations.Internal;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.joda.time.Instant;
import org.slf4j.Logger;
Expand Down Expand Up @@ -243,7 +243,7 @@ private void applianceDispatchLoop(Supplier<Windmill.GetWorkResponse> getWorkFn)
watermarks.setOutputDataWatermark(workItem.getOutputDataWatermark()).build(),
Work.createProcessingContext(
computationId, getDataClient, workCommitter::commit, heartbeatSender),
/* getWorkStreamLatencies= */ Collections.emptyList());
/* getWorkStreamLatencies= */ ImmutableList.of());
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

import java.util.ArrayList;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;

/**
Expand All @@ -27,7 +28,7 @@
*/
public class OutputReceiver implements Receiver {
private final List<Receiver> outputs = new ArrayList<>();
private final HashMap<String, ElementCounter> outputCounters = new HashMap<>();
private final HashMap<String, ElementCounter> outputCounters = new LinkedHashMap<>();
arunpandianp marked this conversation as resolved.
Show resolved Hide resolved

/** Adds a new receiver that this OutputReceiver forwards to. */
public void addOutput(Receiver receiver) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,9 +26,11 @@
import javax.annotation.concurrent.NotThreadSafe;
import org.apache.beam.runners.dataflow.worker.WindmillTimeUtils;
import org.apache.beam.runners.dataflow.worker.windmill.Windmill;
import org.apache.beam.runners.dataflow.worker.windmill.Windmill.LatencyAttribution;
import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItem;
import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
import org.joda.time.Instant;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -134,7 +136,7 @@ abstract static class AssembledWorkItem {
private static AssembledWorkItem create(
WorkItem workItem,
ComputationMetadata computationMetadata,
List<Windmill.LatencyAttribution> latencyAttributions,
ImmutableList<LatencyAttribution> latencyAttributions,
long size) {
return new AutoValue_GetWorkResponseChunkAssembler_AssembledWorkItem(
workItem, computationMetadata, latencyAttributions, size);
Expand All @@ -144,7 +146,7 @@ private static AssembledWorkItem create(

abstract ComputationMetadata computationMetadata();

abstract List<Windmill.LatencyAttribution> latencyAttributions();
abstract ImmutableList<LatencyAttribution> latencyAttributions();

abstract long bufferedSize();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,18 +17,16 @@
*/
package org.apache.beam.runners.dataflow.worker.windmill.client.grpc;

import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.EnumMap;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import javax.annotation.Nullable;
import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetWorkStreamTimingInfo;
import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetWorkStreamTimingInfo.Event;
import org.apache.beam.runners.dataflow.worker.windmill.Windmill.LatencyAttribution;
import org.apache.beam.runners.dataflow.worker.windmill.Windmill.LatencyAttribution.State;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
import org.joda.time.DateTimeUtils.MillisProvider;
import org.joda.time.Duration;
import org.joda.time.Instant;
Expand Down Expand Up @@ -125,12 +123,12 @@ void addTimingInfo(Collection<GetWorkStreamTimingInfo> infos) {
workItemLastChunkReceivedByWorkerTime = now;
}

List<LatencyAttribution> getLatencyAttributions() {
ImmutableList<LatencyAttribution> getLatencyAttributions() {
if (workItemCreationLatency == null && aggregatedGetWorkStreamLatencies.isEmpty()) {
return Collections.emptyList();
return ImmutableList.of();
}
List<LatencyAttribution> latencyAttributions =
new ArrayList<>(aggregatedGetWorkStreamLatencies.size() + 1);
ImmutableList.Builder<LatencyAttribution> latencyAttributions =
ImmutableList.builderWithExpectedSize(aggregatedGetWorkStreamLatencies.size() + 1);
if (workItemCreationLatency != null) {
latencyAttributions.add(workItemCreationLatency);
}
Expand Down Expand Up @@ -167,7 +165,7 @@ List<LatencyAttribution> getLatencyAttributions() {
.setTotalDurationMillis(durationMills)
.build());
});
return latencyAttributions;
return latencyAttributions.build();
}

void reset() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,9 @@
*/
package org.apache.beam.runners.dataflow.worker.windmill.work;

import java.util.Collection;
import org.apache.beam.runners.dataflow.worker.windmill.Windmill;
import org.apache.beam.runners.dataflow.worker.windmill.Windmill.LatencyAttribution;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
import org.checkerframework.checker.nullness.qual.Nullable;
import org.joda.time.Instant;

Expand All @@ -31,5 +32,5 @@ void receiveWork(
@Nullable Instant synchronizedProcessingTime,
Windmill.WorkItem workItem,
long serializedWorkItemSize,
Collection<Windmill.LatencyAttribution> getWorkStreamLatencies);
ImmutableList<LatencyAttribution> getWorkStreamLatencies);
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,13 +17,13 @@
*/
package org.apache.beam.runners.dataflow.worker.windmill.work;

import java.util.Collection;
import javax.annotation.CheckReturnValue;
import org.apache.beam.runners.dataflow.worker.streaming.Watermarks;
import org.apache.beam.runners.dataflow.worker.streaming.Work;
import org.apache.beam.runners.dataflow.worker.windmill.Windmill.LatencyAttribution;
import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItem;
import org.apache.beam.sdk.annotations.Internal;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;

@FunctionalInterface
@CheckReturnValue
Expand All @@ -43,5 +43,5 @@ void scheduleWork(
long serializedWorkItemSize,
Watermarks watermarks,
Work.ProcessingContext processingContext,
Collection<LatencyAttribution> getWorkStreamLatencies);
ImmutableList<LatencyAttribution> getWorkStreamLatencies);
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@

import com.google.api.services.dataflow.model.MapTask;
import com.google.auto.value.AutoValue;
import java.util.Collection;
import java.util.Optional;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.TimeUnit;
Expand All @@ -48,6 +47,7 @@
import org.apache.beam.runners.dataflow.worker.streaming.sideinput.SideInputStateFetcherFactory;
import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor;
import org.apache.beam.runners.dataflow.worker.windmill.Windmill;
import org.apache.beam.runners.dataflow.worker.windmill.Windmill.LatencyAttribution;
import org.apache.beam.runners.dataflow.worker.windmill.client.commits.Commit;
import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache;
import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateReader;
Expand All @@ -57,6 +57,7 @@
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.fn.IdGenerator;
import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
import org.checkerframework.checker.nullness.qual.Nullable;
import org.joda.time.Duration;
import org.joda.time.Instant;
Expand All @@ -71,6 +72,7 @@
@Internal
@ThreadSafe
public class StreamingWorkScheduler {

private static final Logger LOG = LoggerFactory.getLogger(StreamingWorkScheduler.class);

private final DataflowWorkerHarnessOptions options;
Expand Down Expand Up @@ -208,17 +210,11 @@ public void scheduleWork(
long serializedWorkItemSize,
Watermarks watermarks,
Work.ProcessingContext processingContext,
Collection<Windmill.LatencyAttribution> getWorkStreamLatencies) {
ImmutableList<LatencyAttribution> getWorkStreamLatencies) {
computationState.activateWork(
ExecutableWork.create(
Work.create(
workItem,
serializedWorkItemSize,
watermarks,
processingContext,
clock,
getWorkStreamLatencies),
work -> processWork(computationState, work)));
Work.create(workItem, serializedWorkItemSize, watermarks, processingContext, clock),
work -> processWork(computationState, work, getWorkStreamLatencies)));
}

/**
Expand All @@ -228,6 +224,14 @@ public void scheduleWork(
*
* @implNote This will block the calling thread during execution of user DoFns.
*/
private void processWork(
ComputationState computationState,
Work work,
ImmutableList<LatencyAttribution> getWorkStreamLatencies) {
work.recordGetWorkStreamLatencies(getWorkStreamLatencies);
processWork(computationState, work);
}

private void processWork(ComputationState computationState, Work work) {
Windmill.WorkItem workItem = work.getWorkItem();
String computationId = computationState.getComputationId();
Expand Down Expand Up @@ -429,6 +433,7 @@ private ExecuteWorkResult executeWork(

@AutoValue
abstract static class ExecuteWorkResult {

private static ExecuteWorkResult create(
Windmill.WorkItemCommitRequest.Builder commitWorkRequest, long stateBytesRead) {
return new AutoValue_StreamingWorkScheduler_ExecuteWorkResult(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,6 @@

import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
Expand Down Expand Up @@ -67,6 +66,7 @@
import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetWorkStream;
import org.apache.beam.runners.dataflow.worker.windmill.work.WorkItemReceiver;
import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudget;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Uninterruptibles;
Expand Down Expand Up @@ -276,7 +276,7 @@ public boolean awaitTermination(int time, TimeUnit unit) throws InterruptedExcep
Instant.now(),
workItem,
workItem.getSerializedSize(),
Collections.singletonList(
ImmutableList.of(
LatencyAttribution.newBuilder()
.setState(State.GET_WORK_IN_TRANSIT_TO_USER_WORKER)
.setTotalDurationMillis(1000)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -358,8 +358,7 @@ private static ExecutableWork createMockWork(
Watermarks.builder().setInputDataWatermark(Instant.EPOCH).build(),
Work.createProcessingContext(
computationId, new FakeGetDataClient(), ignored -> {}, mock(HeartbeatSender.class)),
Instant::now,
Collections.emptyList()),
Instant::now),
processWorkFn);
}

Expand Down Expand Up @@ -3523,8 +3522,7 @@ public void testLatencyAttributionProtobufsPopulated() {
new FakeGetDataClient(),
ignored -> {},
mock(HeartbeatSender.class)),
clock,
Collections.emptyList());
clock);

clock.sleep(Duration.millis(10));
work.setState(Work.State.PROCESSING);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,6 @@
import com.google.api.services.dataflow.model.CounterStructuredNameAndMetadata;
import com.google.api.services.dataflow.model.CounterUpdate;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
Expand Down Expand Up @@ -144,8 +143,7 @@ private static Work createMockWork(Windmill.WorkItem workItem, Watermarks waterm
watermarks,
Work.createProcessingContext(
COMPUTATION_ID, new FakeGetDataClient(), ignored -> {}, mock(HeartbeatSender.class)),
Instant::now,
Collections.emptyList());
Instant::now);
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -204,8 +204,7 @@ private static Work createMockWork(Windmill.WorkItem workItem, Watermarks waterm
watermarks,
Work.createProcessingContext(
COMPUTATION_ID, new FakeGetDataClient(), ignored -> {}, mock(HeartbeatSender.class)),
Instant::now,
Collections.emptyList());
Instant::now);
}

private static class SourceProducingSubSourcesInSplit extends MockSource {
Expand Down Expand Up @@ -1015,8 +1014,7 @@ public void testFailedWorkItemsAbort() throws Exception {
new FakeGetDataClient(),
ignored -> {},
mock(HeartbeatSender.class)),
Instant::now,
Collections.emptyList());
Instant::now);
context.start(
"key",
dummyWork,
Expand Down
Loading
Loading