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

Migrate portable runners to second version of data channel code #25104

Merged
merged 1 commit into from
Jan 24, 2023
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 @@ -20,10 +20,12 @@
import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState;

import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.CompletionStage;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.Phaser;
Expand All @@ -46,12 +48,16 @@
import org.apache.beam.runners.fnexecution.data.RemoteInputDestination;
import org.apache.beam.runners.fnexecution.state.StateDelegator;
import org.apache.beam.runners.fnexecution.state.StateRequestHandler;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.fn.IdGenerator;
import org.apache.beam.sdk.fn.IdGenerators;
import org.apache.beam.sdk.fn.data.BeamFnDataInboundObserver2;
import org.apache.beam.sdk.fn.data.BeamFnDataOutboundAggregator;
import org.apache.beam.sdk.fn.data.CloseableFnDataReceiver;
import org.apache.beam.sdk.fn.data.DataEndpoint;
import org.apache.beam.sdk.fn.data.FnDataReceiver;
import org.apache.beam.sdk.fn.data.InboundDataClient;
import org.apache.beam.sdk.fn.data.LogicalEndpoint;
import org.apache.beam.sdk.fn.data.TimerEndpoint;
import org.apache.beam.sdk.util.MoreFutures;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
Expand Down Expand Up @@ -262,62 +268,78 @@ public ActiveBundle newBundle(

CompletionStage<BeamFnApi.ProcessBundleResponse> specificResponse =
genericResponse.thenApply(InstructionResponse::getProcessBundle);
Map<LogicalEndpoint, InboundDataClient> outputClients = new HashMap<>();
for (Map.Entry<String, RemoteOutputReceiver<?>> receiver : outputReceivers.entrySet()) {
LogicalEndpoint endpoint = LogicalEndpoint.data(bundleId, receiver.getKey());
InboundDataClient outputClient =
attachReceiver(endpoint, (RemoteOutputReceiver) receiver.getValue());
outputClients.put(endpoint, outputClient);
}
for (Map.Entry<KV<String, String>, RemoteOutputReceiver<Timer<?>>> timerReceiver :
timerReceivers.entrySet()) {
LogicalEndpoint endpoint =
LogicalEndpoint.timer(
bundleId, timerReceiver.getKey().getKey(), timerReceiver.getKey().getValue());
InboundDataClient outputClient = attachReceiver(endpoint, timerReceiver.getValue());
outputClients.put(endpoint, outputClient);
Optional<BeamFnDataInboundObserver2> beamFnDataInboundObserver;
if (outputReceivers.isEmpty() && timerReceivers.isEmpty()) {
beamFnDataInboundObserver = Optional.empty();
} else {
List<DataEndpoint<?>> dataEndpoints = new ArrayList<>(outputReceivers.size());
for (Map.Entry<String, RemoteOutputReceiver<?>> receiver : outputReceivers.entrySet()) {
dataEndpoints.add(
DataEndpoint.create(
receiver.getKey(),
(Coder<Object>) receiver.getValue().getCoder(),
(FnDataReceiver<Object>) receiver.getValue().getReceiver()));
}
List<TimerEndpoint<?>> timerEndpoints = new ArrayList<>(timerReceivers.size());
for (Map.Entry<KV<String, String>, RemoteOutputReceiver<Timer<?>>> timerReceiver :
timerReceivers.entrySet()) {
timerEndpoints.add(
TimerEndpoint.create(
timerReceiver.getKey().getKey(),
timerReceiver.getKey().getValue(),
timerReceiver.getValue().getCoder(),
timerReceiver.getValue().getReceiver()));
}
beamFnDataInboundObserver =
Optional.of(BeamFnDataInboundObserver2.forConsumers(dataEndpoints, timerEndpoints));
fnApiDataService.registerReceiver(bundleId, beamFnDataInboundObserver.get());
}

ImmutableMap.Builder<LogicalEndpoint, CloseableFnDataReceiver> receiverBuilder =
ImmutableMap.Builder<LogicalEndpoint, FnDataReceiver<?>> receiverBuilder =
ImmutableMap.builder();
BeamFnDataOutboundAggregator beamFnDataOutboundAggregator =
fnApiDataService.createOutboundAggregator(() -> bundleId, false);
for (RemoteInputDestination remoteInput : remoteInputs) {
LogicalEndpoint endpoint = LogicalEndpoint.data(bundleId, remoteInput.getPTransformId());
receiverBuilder.put(
endpoint,
new CountingFnDataReceiver(fnApiDataService.send(endpoint, remoteInput.getCoder())));
new CountingFnDataReceiver(
beamFnDataOutboundAggregator.registerOutputDataLocation(
remoteInput.getPTransformId(), remoteInput.getCoder())));
}

for (Map.Entry<String, Map<String, TimerSpec>> entry : timerSpecs.entrySet()) {
for (TimerSpec timerSpec : entry.getValue().values()) {
LogicalEndpoint endpoint =
LogicalEndpoint.timer(bundleId, timerSpec.transformId(), timerSpec.timerId());
receiverBuilder.put(endpoint, fnApiDataService.send(endpoint, timerSpec.coder()));
receiverBuilder.put(
endpoint,
beamFnDataOutboundAggregator.registerOutputTimersLocation(
timerSpec.transformId(), timerSpec.timerId(), timerSpec.coder()));
}
}
beamFnDataOutboundAggregator.start();

return new ActiveBundle(
bundleId,
specificResponse,
beamFnDataOutboundAggregator,
receiverBuilder.build(),
outputClients,
beamFnDataInboundObserver,
stateDelegator.registerForProcessBundleInstructionId(bundleId, stateRequestHandler),
progressHandler,
splitHandler,
checkpointHandler,
finalizationHandler);
}

private <OutputT> InboundDataClient attachReceiver(
LogicalEndpoint endpoint, RemoteOutputReceiver<OutputT> receiver) {
return fnApiDataService.receive(endpoint, receiver.getCoder(), receiver.getReceiver());
}

/** An active bundle for a particular {@link BeamFnApi.ProcessBundleDescriptor}. */
public class ActiveBundle implements RemoteBundle {
private final String bundleId;
private final CompletionStage<BeamFnApi.ProcessBundleResponse> response;
private final Map<LogicalEndpoint, CloseableFnDataReceiver> inputReceivers;
private final Map<LogicalEndpoint, InboundDataClient> outputClients;
private final BeamFnDataOutboundAggregator beamFnDataOutboundAggregator;
private final Map<LogicalEndpoint, FnDataReceiver<?>> inputReceivers;
private final Optional<BeamFnDataInboundObserver2> beamFnDataInboundObserver;
private final StateDelegator.Registration stateRegistration;
private final BundleProgressHandler progressHandler;
private final BundleSplitHandler splitHandler;
Expand All @@ -330,17 +352,19 @@ public class ActiveBundle implements RemoteBundle {
private ActiveBundle(
String bundleId,
CompletionStage<ProcessBundleResponse> response,
Map<LogicalEndpoint, CloseableFnDataReceiver> inputReceivers,
Map<LogicalEndpoint, InboundDataClient> outputClients,
BeamFnDataOutboundAggregator beamFnDataOutboundAggregator,
Map<LogicalEndpoint, FnDataReceiver<?>> inputReceivers,
Optional<BeamFnDataInboundObserver2> beamFnDataInboundObserver,
StateDelegator.Registration stateRegistration,
BundleProgressHandler progressHandler,
BundleSplitHandler splitHandler,
BundleCheckpointHandler checkpointHandler,
BundleFinalizationHandler finalizationHandler) {
this.bundleId = bundleId;
this.response = response;
this.beamFnDataOutboundAggregator = beamFnDataOutboundAggregator;
this.inputReceivers = inputReceivers;
this.outputClients = outputClients;
this.beamFnDataInboundObserver = beamFnDataInboundObserver;
this.stateRegistration = stateRegistration;
this.progressHandler = progressHandler;
this.splitHandler = splitHandler;
Expand Down Expand Up @@ -371,8 +395,7 @@ public String getId() {
@Override
public Map<String, FnDataReceiver> getInputReceivers() {
ImmutableMap.Builder<String, FnDataReceiver> rval = ImmutableMap.builder();
for (Map.Entry<LogicalEndpoint, CloseableFnDataReceiver> entry :
inputReceivers.entrySet()) {
for (Map.Entry<LogicalEndpoint, FnDataReceiver<?>> entry : inputReceivers.entrySet()) {
if (!entry.getKey().isTimer()) {
rval.put(entry.getKey().getTransformId(), entry.getValue());
}
Expand All @@ -384,12 +407,11 @@ public Map<String, FnDataReceiver> getInputReceivers() {
public Map<KV<String, String>, FnDataReceiver<Timer>> getTimerReceivers() {
ImmutableMap.Builder<KV<String, String>, FnDataReceiver<Timer>> rval =
ImmutableMap.builder();
for (Map.Entry<LogicalEndpoint, CloseableFnDataReceiver> entry :
inputReceivers.entrySet()) {
for (Map.Entry<LogicalEndpoint, FnDataReceiver<?>> entry : inputReceivers.entrySet()) {
if (entry.getKey().isTimer()) {
rval.put(
KV.of(entry.getKey().getTransformId(), entry.getKey().getTimerFamilyId()),
entry.getValue());
(FnDataReceiver<Timer>) entry.getValue());
}
}
return rval.build();
Expand Down Expand Up @@ -432,7 +454,7 @@ public void split(double fractionOfRemainder) {
outstandingRequests.register();
}
Map<String, DesiredSplit> splits = new HashMap<>();
for (Map.Entry<LogicalEndpoint, CloseableFnDataReceiver> ptransformToInput :
for (Map.Entry<LogicalEndpoint, FnDataReceiver<?>> ptransformToInput :
inputReceivers.entrySet()) {
if (!ptransformToInput.getKey().isTimer()) {
splits.put(
Expand Down Expand Up @@ -487,16 +509,12 @@ public void close() throws Exception {
}

Exception exception = null;
for (CloseableFnDataReceiver<?> inputReceiver : inputReceivers.values()) {
try {
inputReceiver.close();
} catch (Exception e) {
if (exception == null) {
exception = e;
} else {
exception.addSuppressed(e);
}
}
try {
beamFnDataOutboundAggregator.sendOrCollectBufferedDataAndFinishOutboundStreams();
} catch (Exception e) {
exception = e;
} finally {
beamFnDataOutboundAggregator.discard();
}
try {
// We don't have to worry about the completion stage.
Expand Down Expand Up @@ -537,12 +555,13 @@ public void close() throws Exception {
exception.addSuppressed(e);
}
}
for (InboundDataClient outputClient : outputClients.values()) {
if (beamFnDataInboundObserver.isPresent()) {
try {
if (exception == null) {
outputClient.awaitCompletion();
beamFnDataInboundObserver.get().awaitCompletion();
fnApiDataService.unregisterReceiver(bundleId);
} else {
outputClient.cancel();
beamFnDataInboundObserver.get().close();
}
} catch (Exception e) {
if (exception == null) {
Expand Down Expand Up @@ -696,14 +715,12 @@ public void abort() {}
}
}

/**
* A {@link CloseableFnDataReceiver} which counts the number of elements that have been accepted.
*/
private static class CountingFnDataReceiver<T> implements CloseableFnDataReceiver<T> {
private final CloseableFnDataReceiver delegate;
/** A {@link FnDataReceiver} which counts the number of elements that have been accepted. */
private static class CountingFnDataReceiver<T> implements FnDataReceiver<T> {
private final FnDataReceiver<T> delegate;
private long count;

private CountingFnDataReceiver(CloseableFnDataReceiver delegate) {
private CountingFnDataReceiver(FnDataReceiver<T> delegate) {
this.delegate = delegate;
}

Expand All @@ -716,16 +733,6 @@ public void accept(T input) throws Exception {
delegate.accept(input);
count += 1;
}

@Override
public void flush() throws Exception {
delegate.flush();
}

@Override
public void close() throws Exception {
delegate.close();
}
}

/** Registers a {@link BeamFnApi.ProcessBundleDescriptor} for future processing. */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,12 +17,11 @@
*/
package org.apache.beam.runners.fnexecution.data;

import org.apache.beam.sdk.coders.Coder;
import java.util.function.Supplier;
import org.apache.beam.model.fnexecution.v1.BeamFnApi.Elements;
import org.apache.beam.sdk.fn.data.BeamFnDataOutboundAggregator;
import org.apache.beam.sdk.fn.data.CloseableFnDataReceiver;
import org.apache.beam.sdk.fn.data.FnDataReceiver;
import org.apache.beam.sdk.fn.data.InboundDataClient;
import org.apache.beam.sdk.fn.data.LogicalEndpoint;

/**
* The {@link FnDataService} is able to forward inbound elements to a consumer and is also a
Expand All @@ -32,33 +31,43 @@
public interface FnDataService {

/**
* Registers a receiver to be notified upon any incoming elements.
* Registers a receiver for the provided instruction id.
*
* <p>The provided coder is used to decode inbound elements. The decoded elements are passed to
* the provided receiver.
* <p>The receiver is not required to be thread safe.
*
* <p>Any failure during decoding or processing of the element will put the {@link
* InboundDataClient} into an error state such that {@link InboundDataClient#awaitCompletion()}
* will throw an exception.
* <p>Receivers for successfully processed bundles must be unregistered. See {@link
* #unregisterReceiver} for details.
*
* <p>The provided receiver is not required to be thread safe.
* <p>Any failure during {@link FnDataReceiver#accept} will mark the provided {@code
* instructionId} as invalid and will ignore any future data. It is expected that if a bundle
* fails during processing then the failure will become visible to the {@link FnDataService}
* during a future {@link FnDataReceiver#accept} invocation.
*/
<T> InboundDataClient receive(
LogicalEndpoint inputLocation, Coder<T> coder, FnDataReceiver<T> listener);
void registerReceiver(String instructionId, CloseableFnDataReceiver<Elements> observer);

/**
* Creates a receiver to which you can write data values and have them sent over this data plane
* service.
* Receivers are only expected to be unregistered when bundle processing has completed
* successfully.
*
* <p>The provided coder is used to encode elements on the outbound stream.
*
* <p>Closing the returned receiver signals the end of the stream.
* <p>It is expected that if a bundle fails during processing then the failure will become visible
* to the {@link FnDataService} during a future {@link FnDataReceiver#accept} invocation.
*/
void unregisterReceiver(String instructionId);

/**
* Creates a {@link BeamFnDataOutboundAggregator} for buffering and sending outbound data and
* timers over the data plane. It is important that {@link
* BeamFnDataOutboundAggregator#sendOrCollectBufferedDataAndFinishOutboundStreams()} is called on
* the returned BeamFnDataOutboundAggregator at the end of each bundle. If
* collectElementsIfNoFlushes is set to true, {@link
* BeamFnDataOutboundAggregator#sendOrCollectBufferedDataAndFinishOutboundStreams()} returns the
* buffered elements instead of sending it through the outbound StreamObserver if there's no
* previous flush.
*
* <p>The returned receiver is not thread safe.
* <p>Closing the returned aggregator signals the end of the streams.
*
* @deprecated Migrate to use {@link BeamFnDataOutboundAggregator} directly for sending outbound
* data.
* <p>The returned aggregator is not thread safe.
*/
@Deprecated
<T> CloseableFnDataReceiver<T> send(LogicalEndpoint outputLocation, Coder<T> coder);
BeamFnDataOutboundAggregator createOutboundAggregator(
Supplier<String> processBundleRequestIdSupplier, boolean collectElementsIfNoFlushes);
}
Loading