Skip to content
Open
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 @@ -221,11 +221,12 @@ private DoFnRunner<InputT, KV<K, Iterable<V>>> createRunner() {
if (hasStreamingSideInput) {
return new StreamingSideInputDoFnRunner<>(
basicRunner,
new StreamingSideInputFetcher<>(
sideInputViews,
inputCoder,
windowingStrategy,
(StreamingModeExecutionContext.StreamingModeStepContext) stepContext));
() ->
new StreamingSideInputFetcher<>(
sideInputViews,
inputCoder,
windowingStrategy,
(StreamingModeExecutionContext.StreamingModeStepContext) stepContext));
} else {
return basicRunner;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import com.google.api.services.dataflow.model.SideInputInfo;
import java.util.List;
import java.util.Set;
import java.util.function.Supplier;
import org.apache.beam.runners.core.GlobalCombineFnRunner;
import org.apache.beam.runners.core.GlobalCombineFnRunners;
import org.apache.beam.runners.core.NullSideInputReader;
Expand All @@ -48,6 +49,7 @@
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.WindowedValue;
import org.apache.beam.sdk.values.WindowedValues;
import org.apache.beam.sdk.values.WindowingStrategy;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.ByteStreams;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.CountingOutputStream;
Expand All @@ -68,7 +70,7 @@ public static <K, InputT, AccumT> ParDoFn create(
DataflowExecutionContext<?> executionContext,
DataflowOperationContext operationContext)
throws Exception {
AppliedCombineFn<K, InputT, AccumT, ?> combineFn;
AppliedCombineFn<K, InputT, AccumT, BoundedWindow> combineFn;
SideInputReader sideInputReader;
StepContext stepContext;
if (cloudUserFn == null) {
Expand All @@ -80,8 +82,8 @@ public static <K, InputT, AccumT> ParDoFn create(
SerializableUtils.deserializeFromByteArray(
getBytes(cloudUserFn, PropertyNames.SERIALIZED_FN), "serialized combine fn");
@SuppressWarnings("unchecked")
AppliedCombineFn<K, InputT, AccumT, ?> combineFnUnchecked =
((AppliedCombineFn<K, InputT, AccumT, ?>) deserializedFn);
AppliedCombineFn<K, InputT, AccumT, BoundedWindow> combineFnUnchecked =
((AppliedCombineFn<K, InputT, AccumT, BoundedWindow>) deserializedFn);
combineFn = combineFnUnchecked;

sideInputReader =
Expand Down Expand Up @@ -136,13 +138,15 @@ static <K, InputT, AccumT> ParDoFn create(
if (sideInputReader.isEmpty()) {
return new SimplePartialGroupByKeyParDoFn<>(groupingTable, receiver);
} else if (options.as(StreamingOptions.class).isStreaming()) {
StreamingSideInputFetcher<KV<K, InputT>, ?> sideInputFetcher =
new StreamingSideInputFetcher<>(
combineFn.getSideInputViews(),
combineFn.getKvCoder(),
combineFn.getWindowingStrategy(),
(StreamingModeExecutionContext.StreamingModeStepContext) stepContext);
return new StreamingSideInputPGBKParDoFn<>(groupingTable, receiver, sideInputFetcher);
Supplier<StreamingSideInputFetcher<KV<K, InputT>, BoundedWindow>> sideInputFetcherSupplier =
() ->
new StreamingSideInputFetcher<>(
combineFn.getSideInputViews(),
combineFn.getKvCoder(),
(WindowingStrategy<?, BoundedWindow>) combineFn.getWindowingStrategy(),
(StreamingModeExecutionContext.StreamingModeStepContext) stepContext);
return new StreamingSideInputPGBKParDoFn<>(
groupingTable, receiver, sideInputFetcherSupplier);
} else {
return new BatchSideInputPGBKParDoFn<>(groupingTable, receiver);
}
Expand Down Expand Up @@ -240,7 +244,7 @@ public WindowingCoderGroupingKeyCreator(Coder<K> coder) {
}

@Override
public Object createGroupingKey(WindowedValue<K> key) throws Exception {
public Object createGroupingKey(WindowedValue<K> key) {
// Ignore timestamp for grouping purposes.
// The PGBK output will inherit the timestamp of one of its inputs.
return WindowedValues.builder(key)
Expand Down Expand Up @@ -333,19 +337,21 @@ static class StreamingSideInputPGBKParDoFn<K, InputT, AccumT, W extends BoundedW
implements ParDoFn {
private final GroupingTable<WindowedValue<K>, InputT, AccumT> groupingTable;
private final Receiver receiver;
private final StreamingSideInputFetcher<KV<K, InputT>, W> sideInputFetcher;
private final Supplier<StreamingSideInputFetcher<KV<K, InputT>, W>> sideInputFetcherSupplier;
private StreamingSideInputFetcher<KV<K, InputT>, W> sideInputFetcher = null;
private boolean activeKey = false;

StreamingSideInputPGBKParDoFn(
GroupingTable<WindowedValue<K>, InputT, AccumT> groupingTable,
Receiver receiver,
StreamingSideInputFetcher<KV<K, InputT>, W> sideInputFetcher) {
Supplier<StreamingSideInputFetcher<KV<K, InputT>, W>> sideInputFetcherSupplier) {
this.groupingTable = groupingTable;
this.receiver = receiver;
this.sideInputFetcher = sideInputFetcher;
this.sideInputFetcherSupplier = sideInputFetcherSupplier;
}

@Override
public void startBundle(Receiver... receivers) throws Exception {
private void onStartKey() throws Exception {
this.sideInputFetcher = sideInputFetcherSupplier.get();
// Find the set of ready windows.
Set<W> readyWindows = sideInputFetcher.getReadyWindows();

Expand All @@ -361,12 +367,23 @@ public void startBundle(Receiver... receivers) throws Exception {
elementsBag.clear();
}
sideInputFetcher.releaseBlockedWindows(readyWindows);
this.activeKey = true;
}

@Override
public void startBundle(Receiver... receivers) throws Exception {
this.activeKey = false;
}

@Override
public void processElement(Object elem) throws Exception {
@SuppressWarnings({"unchecked"})
WindowedValue<KV<K, InputT>> input = (WindowedValue<KV<K, InputT>>) elem;

if (!activeKey) {
onStartKey();
}

for (BoundedWindow w : input.getWindows()) {
WindowedValue<KV<K, InputT>> windowsExpandedInput =
WindowedValues.of(input.getValue(), input.getTimestamp(), w, input.getPaneInfo());
Expand All @@ -378,15 +395,28 @@ public void processElement(Object elem) throws Exception {
}

@Override
public void processTimers() {}
public void processTimers() throws Exception {
if (!activeKey) {
onStartKey();
}
}

@Override
public void finishKey(Object key) throws Exception {}
public void finishKey(Object key) throws Exception {
if (!activeKey) {
onStartKey();
}
sideInputFetcher.persist();
sideInputFetcher = null;
this.activeKey = false;
}

@Override
public void finishBundle() throws Exception {
groupingTable.flush(receiver);
sideInputFetcher.persist();
if (sideInputFetcher != null) {
sideInputFetcher.persist();
}
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@
*/
package org.apache.beam.runners.dataflow.worker;

import java.io.Closeable;
import java.util.Collection;
import java.util.Collections;
import java.util.Iterator;
Expand Down Expand Up @@ -50,7 +49,7 @@
"nullness" // TODO(https://github.com/apache/beam/issues/20497)
})
public class SimpleParDoFn<InputT, OutputT, W extends BoundedWindow> implements ParDoFn {
private final SimpleParDoFnHelpers<InputT, OutputT, W> helpers;
private final SimpleParDoFnHelpers<Object, InputT, OutputT, W> helpers;
private @Nullable StreamingSideInputProcessor<InputT, W> sideInputProcessor;

/** Creates a {@link SimpleParDoFn} using basic information about the step being executed. */
Expand All @@ -76,7 +75,8 @@ public class SimpleParDoFn<InputT, OutputT, W extends BoundedWindow> implements
operationContext,
doFnSchemaInformation,
sideInputMapping,
runnerFactory);
runnerFactory,
this::onStartKey);
}

@Override
Expand All @@ -86,17 +86,14 @@ public void startBundle(Receiver... receivers) throws Exception {
// There is non-trivial setup that needs to be performed for watermark propagation
// even on empty bundles.
helpers.reallyStartBundle();
onStartKey();
}
}

protected void onStartKey() {
// TODO(relax): This assumes single-key bundles, which will change! Refactor this to not make
// this assumption.
protected void onStartKey(Object key) {
if (helpers.hasStreamingSideInput) {
sideInputProcessor =
new StreamingSideInputProcessor<>(
new StreamingSideInputFetcher<InputT, W>(
new StreamingSideInputFetcher<>(
helpers.fnInfo.getSideInputViews(),
helpers.fnInfo.getInputCoder(),
(WindowingStrategy<?, W>) helpers.fnInfo.getWindowingStrategy(),
Expand All @@ -123,20 +120,7 @@ protected void onStartKey() {
@Override
@SuppressWarnings("unchecked")
public void processElement(Object untypedElem) throws Exception {
if (helpers.fnRunner == null) {
// If we need to run reallyStartBundle in here, we need to make sure to switch the state
// sampler into the start state.
try (Closeable start = helpers.operationContext.enterStart()) {
helpers.reallyStartBundle();
onStartKey();
}
}
helpers.outputsPerElementTracker.onProcessElement();

WindowedValue<InputT> elem = (WindowedValue<InputT>) untypedElem;
onProcessWindowedValue(elem);

helpers.outputsPerElementTracker.onProcessElementSuccess();
helpers.processElement(null, (WindowedValue<InputT>) untypedElem, this::onProcessWindowedValue);
}

protected void onProcessWindowedValue(WindowedValue<InputT> elem) {
Expand Down Expand Up @@ -169,7 +153,6 @@ protected void onProcessWindowedValue(WindowedValue<InputT> elem) {

@Override
public void processTimers() throws Exception {

// Note: We need to get windowCoder to decode the timers. If we haven't already deserialized
// the fnInfo, we peek at a new instance to retrieve that. If this extra deserialization becomes
// excessively costly, we could either (1) have the DoFnInstanceManager remember the associated
Expand All @@ -185,18 +168,18 @@ public void processTimers() throws Exception {
SimpleParDoFnHelpers.TimerType.USER,
helpers.userStepContext,
windowCoder,
this::onStartKey,
() -> sideInputProcessor);
helpers.processTimers(
SimpleParDoFnHelpers.TimerType.SYSTEM,
helpers.stepContext,
windowCoder,
this::onStartKey,
() -> sideInputProcessor);
}

@Override
public void finishKey(Object key) throws Exception {}
public void finishKey(Object key) throws Exception {
helpers.finishKey(sideInputProcessor);
}

@Override
public void finishBundle() throws Exception {
Expand Down
Loading
Loading