robertwb commented on code in PR #24025:
URL: https://github.com/apache/beam/pull/24025#discussion_r1035233601
##########
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/FlatMapElements.java:
##########
@@ -45,16 +45,13 @@
extends PTransform<PCollection<? extends InputT>, PCollection<OutputT>> {
private final transient @Nullable TypeDescriptor<InputT> inputType;
private final transient @Nullable TypeDescriptor<OutputT> outputType;
- private final transient @Nullable Object originalFnForDisplayData;
- private final @Nullable Contextful<Fn<InputT, Iterable<OutputT>>> fn;
+ private final @Nullable Object fn;
private FlatMapElements(
- @Nullable Contextful<Fn<InputT, Iterable<OutputT>>> fn,
- @Nullable Object originalFnForDisplayData,
+ @Nullable Object fn,
Review Comment:
Shouldn't this still be an InferableFunction? Or is the goal to accept
either InferableFunction or Contextful? (If the latter, is this too painful to
encode in the type system?)
##########
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/FlatMapElements.java:
##########
@@ -137,57 +132,90 @@ public <NewInputT> FlatMapElements<NewInputT, OutputT>
via(
@Experimental(Kind.CONTEXTFUL)
public <NewInputT> FlatMapElements<NewInputT, OutputT> via(
Contextful<Fn<NewInputT, Iterable<OutputT>>> fn) {
- return new FlatMapElements<>(
- fn, fn.getClosure(), TypeDescriptors.inputOf(fn.getClosure()),
outputType);
+ return new FlatMapElements<>(fn, TypeDescriptors.inputOf(fn.getClosure()),
outputType);
}
@Override
public PCollection<OutputT> expand(PCollection<? extends InputT> input) {
checkArgument(fn != null, ".via() is required");
- return input.apply(
- "FlatMap",
- ParDo.of(
- new DoFn<InputT, OutputT>() {
- @ProcessElement
- public void processElement(ProcessContext c) throws
Exception {
- Iterable<OutputT> res =
- fn.getClosure().apply(c.element(),
Fn.Context.wrapProcessContext(c));
- for (OutputT output : res) {
- c.output(output);
+ if (fn instanceof Contextful) {
Review Comment:
Do I understand correctly that doing this branch every time is cheaper than
the indirection of an "empty" Contextful?
Would it make sense to create a subclass based on the Fn type rather than
branch here (and below).
--
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]