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 @@ -88,7 +88,6 @@
import org.apache.beam.sdk.transforms.reflect.DoFnSignatures;
import org.apache.beam.sdk.transforms.resourcehints.ResourceHint;
import org.apache.beam.sdk.transforms.resourcehints.ResourceHints;
import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
import org.apache.beam.sdk.transforms.windowing.Window;
import org.apache.beam.sdk.util.AppliedCombineFn;
import org.apache.beam.sdk.util.CoderUtils;
Expand Down Expand Up @@ -970,8 +969,8 @@ private <K, V> void groupByKeyHelper(
&& windowingStrategy.getWindowFn().assignsToOneWindow();
if (isStreaming) {
allowCombinerLifting &= transform.fewKeys();
// TODO: Allow combiner lifting on the non-default trigger, as appropriate.
allowCombinerLifting &= (windowingStrategy.getTrigger() instanceof DefaultTrigger);
allowCombinerLifting &=
windowingStrategy.getTrigger().isCompatibleWithCombinerLifting();
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this method should be part of the DataflowRunner not the trigger. Whether the needed metadata is available is a property of how the runner executes the trigger.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Making it part of the Trigger does ensure that we don't accidentally miss a trigger (in case one gets added). I guess we could check this at runtime or we could create a TriggerVisitor (which will ensure no cases are missed at compile time)?

}
stepContext.addInput(PropertyNames.DISALLOW_COMBINER_LIFTING, !allowCombinerLifting);
stepContext.addInput(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,11 @@ public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
return deadline;
}

@Override
public boolean isCompatibleWithCombinerLifting() {
return subTriggers.stream().allMatch(Trigger::isCompatibleWithCombinerLifting);
}

@Override
protected OnceTrigger getContinuationTrigger(List<Trigger> continuationTriggers) {
return new AfterAll(continuationTriggers);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -67,14 +67,19 @@ public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {

@Override
public boolean mayFinish() {
return subTriggers.stream().allMatch(trigger -> trigger.mayFinish());
return subTriggers.stream().allMatch(Trigger::mayFinish);
}

@Override
protected Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
return Repeatedly.forever(new AfterFirst(continuationTriggers));
}

@Override
public boolean isCompatibleWithCombinerLifting() {
return subTriggers.stream().allMatch(Trigger::isCompatibleWithCombinerLifting);
}

@Override
public String toString() {
StringBuilder builder = new StringBuilder("AfterEach.inOrder(");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,11 @@ public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
return deadline;
}

@Override
public boolean isCompatibleWithCombinerLifting() {
return subTriggers.stream().allMatch(Trigger::isCompatibleWithCombinerLifting);
}

@Override
protected OnceTrigger getContinuationTrigger(List<Trigger> continuationTriggers) {
return new AfterFirst(continuationTriggers);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,12 @@ public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
return BoundedWindow.TIMESTAMP_MAX_VALUE;
}

@Override
public boolean isCompatibleWithCombinerLifting() {
// Combiner lifting not supported for count triggers.
return false;
}

@Override
protected OnceTrigger getContinuationTrigger(List<Trigger> continuationTriggers) {
return AfterPane.elementCountAtLeast(1);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -107,6 +107,11 @@ public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
return BoundedWindow.TIMESTAMP_MAX_VALUE;
}

@Override
public boolean isCompatibleWithCombinerLifting() {
return true;
}

@Override
protected Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
return AfterSynchronizedProcessingTime.ofFirstElement();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,11 @@ public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
return BoundedWindow.TIMESTAMP_MAX_VALUE;
}

@Override
public boolean isCompatibleWithCombinerLifting() {
return true;
}

@Override
protected Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
return this;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -126,6 +126,12 @@ public boolean mayFinish() {
return lateTrigger == null;
}

@Override
public boolean isCompatibleWithCombinerLifting() {
return earlyTrigger.isCompatibleWithCombinerLifting()
&& (lateTrigger == null || lateTrigger.isCompatibleWithCombinerLifting());
}
Comment on lines 130 to 133
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

high

This implementation can cause a NullPointerException because lateTrigger can be null. A more robust and concise approach, consistent with other composite triggers, is to check all subTriggers.

    public boolean isCompatibleWithCombinerLifting() {
      return subTriggers.stream().allMatch(Trigger::isCompatibleWithCombinerLifting);
    }

Comment on lines +129 to +133
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

medium

For consistency with other composite triggers like AfterAll and AfterFirst, consider using a stream over subTriggers to check for compatibility. This would make the implementation more uniform across different trigger types.

Suggested change
@Override
public boolean isCompatibleWithCombinerLifting() {
return earlyTrigger.isCompatibleWithCombinerLifting()
&& (lateTrigger == null || lateTrigger.isCompatibleWithCombinerLifting());
}
@Override
public boolean isCompatibleWithCombinerLifting() {
return subTriggers.stream().allMatch(Trigger::isCompatibleWithCombinerLifting);
}


@Override
public String toString() {
StringBuilder builder = new StringBuilder(TO_STRING);
Expand Down Expand Up @@ -177,6 +183,11 @@ protected FromEndOfWindow getContinuationTrigger(List<Trigger> continuationTrigg
return this;
}

@Override
public boolean isCompatibleWithCombinerLifting() {
return true;
}

@Override
public String toString() {
return TO_STRING;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,11 @@ public boolean isCompatible(Trigger other) {
return other instanceof DefaultTrigger;
}

@Override
public boolean isCompatibleWithCombinerLifting() {
return true;
}

@Override
protected Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
return this;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -56,5 +56,10 @@ protected Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
return BoundedWindow.TIMESTAMP_MAX_VALUE;
}

@Override
public boolean isCompatibleWithCombinerLifting() {
return true;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -64,6 +64,12 @@ public boolean mayFinish() {
return subTriggers.get(ACTUAL).mayFinish() || subTriggers.get(UNTIL).mayFinish();
}

@Override
public boolean isCompatibleWithCombinerLifting() {
return subTriggers.get(ACTUAL).isCompatibleWithCombinerLifting()
&& subTriggers.get(UNTIL).isCompatibleWithCombinerLifting();
}
Comment on lines +67 to +71
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

medium

For consistency with other composite triggers, you could use a stream over subTriggers here as well. This would make the code more uniform and align with the pattern used in AfterAll, AfterFirst, and AfterEach.

Suggested change
@Override
public boolean isCompatibleWithCombinerLifting() {
return subTriggers.get(ACTUAL).isCompatibleWithCombinerLifting()
&& subTriggers.get(UNTIL).isCompatibleWithCombinerLifting();
}
@Override
public boolean isCompatibleWithCombinerLifting() {
return subTriggers.stream().allMatch(Trigger::isCompatibleWithCombinerLifting);
}


@Override
protected Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
// Use OrFinallyTrigger instead of AfterFirst because the continuation of ACTUAL
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -73,6 +73,11 @@ public boolean mayFinish() {
return false;
}

@Override
public boolean isCompatibleWithCombinerLifting() {
return subTriggers.get(REPEATED).isCompatibleWithCombinerLifting();
}

@Override
protected Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
return new Repeatedly(continuationTriggers.get(REPEATED));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,11 @@ public boolean mayFinish() {
return false;
}

@Override
public boolean isCompatibleWithCombinerLifting() {
return false;
}

@Override
public String toString() {
return "ReshuffleTrigger()";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -109,6 +109,9 @@ public Trigger getContinuationTrigger() {
return getContinuationTrigger(subTriggerContinuations);
}

/** Returns whether this trigger is compatible with combiner lifting. */
public abstract boolean isCompatibleWithCombinerLifting();

/**
* Subclasses should override this to return the {@link #getContinuationTrigger} of this {@link
* Trigger}. For convenience, this is provided the continuation trigger of each of the
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,4 +48,9 @@ protected Trigger getContinuationTrigger(List<Trigger> continuationTriggers) {
public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
return null;
}

@Override
public boolean isCompatibleWithCombinerLifting() {
return false;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -73,6 +73,11 @@ public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
public boolean mayFinish() {
return false;
}

@Override
public boolean isCompatibleWithCombinerLifting() {
return false;
}
}

private static class Trigger2 extends Trigger {
Expand All @@ -95,5 +100,10 @@ public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) {
public boolean mayFinish() {
return false;
}

@Override
public boolean isCompatibleWithCombinerLifting() {
return false;
}
}
}
Loading