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

[BEAM-1038] Allow stateful DoFn in DataflowRunner #1523

Closed
wants to merge 1 commit into from
Closed
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
1 change: 0 additions & 1 deletion runners/google-cloud-dataflow-java/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -78,7 +78,6 @@
<id>runnable-on-service-tests</id>
<configuration>
<excludedGroups>
org.apache.beam.sdk.testing.UsesStatefulParDo,
org.apache.beam.sdk.testing.UsesSplittableParDo
</excludedGroups>
<excludes>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -77,6 +77,7 @@
import org.apache.beam.sdk.transforms.View;
import org.apache.beam.sdk.transforms.display.DisplayData;
import org.apache.beam.sdk.transforms.display.HasDisplayData;
import org.apache.beam.sdk.transforms.reflect.DoFnSignature;
import org.apache.beam.sdk.transforms.reflect.DoFnSignatures;
import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
import org.apache.beam.sdk.transforms.windowing.Window;
Expand Down Expand Up @@ -960,7 +961,6 @@ public void translate(
private <InputT, OutputT> void translateMultiHelper(
ParDo.BoundMulti<InputT, OutputT> transform,
TranslationContext context) {
rejectStatefulDoFn(transform.getNewFn());

context.addStep(transform, "ParallelDo");
translateInputs(context.getInput(transform), transform.getSideInputs(), context);
Expand Down Expand Up @@ -990,7 +990,6 @@ public void translate(
private <InputT, OutputT> void translateSingleHelper(
ParDo.Bound<InputT, OutputT> transform,
TranslationContext context) {
rejectStatefulDoFn(transform.getNewFn());

context.addStep(transform, "ParallelDo");
translateInputs(context.getInput(transform), transform.getSideInputs(), context);
Expand Down Expand Up @@ -1038,18 +1037,6 @@ private <T> void translateHelper(
registerTransformTranslator(Read.Bounded.class, new ReadTranslator());
}

private static void rejectStatefulDoFn(DoFn<?, ?> doFn) {
if (DoFnSignatures.getSignature(doFn.getClass()).isStateful()) {
throw new UnsupportedOperationException(
String.format(
"Found %s annotations on %s, but %s cannot yet be used with state in the %s.",
DoFn.StateId.class.getSimpleName(),
doFn.getClass().getName(),
DoFn.class.getSimpleName(),
DataflowRunner.class.getSimpleName()));
}
}

private static void translateInputs(
PCollection<?> input,
List<PCollectionView<?>> sideInputs,
Expand Down Expand Up @@ -1081,13 +1068,20 @@ private static void translateFn(
TranslationContext context,
long mainOutput,
Map<Long, TupleTag<?>> outputMap) {

DoFnSignature signature = DoFnSignatures.getSignature(fn.getClass());

context.addInput(PropertyNames.USER_FN, fn.getClass().getName());
context.addInput(
PropertyNames.SERIALIZED_FN,
byteArrayToJsonString(
serializeToByteArray(
DoFnInfo.forFn(
fn, windowingStrategy, sideInputs, inputCoder, mainOutput, outputMap))));

if (signature.isStateful()) {
context.addInput(PropertyNames.USES_KEYED_STATE, "true");
}
}

private static BiMap<Long, TupleTag<?>> translateOutputs(
Expand Down