Skip to content
Merged
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 @@ -26,7 +26,6 @@
import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
import org.apache.flink.streaming.api.operators.InternalTimer;
import org.apache.flink.streaming.api.operators.InternalTimerService;
import org.apache.flink.streaming.api.operators.MailboxWatermarkProcessor;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.operators.SimpleOperatorFactory;
import org.apache.flink.streaming.api.operators.Triggerable;
Expand Down Expand Up @@ -170,7 +169,6 @@ private static class MultipleTimersAtTheSameTimestamp extends AbstractStreamOper

private final Map<Instant, Integer> timersToRegister;
private transient @Nullable MailboxExecutor mailboxExecutor;
private transient @Nullable MailboxWatermarkProcessor watermarkProcessor;

MultipleTimersAtTheSameTimestamp() {
this(Collections.emptyMap());
Expand All @@ -181,18 +179,14 @@ private static class MultipleTimersAtTheSameTimestamp extends AbstractStreamOper
}

@Override
public void setMailboxExecutor(MailboxExecutor mailboxExecutor) {
this.mailboxExecutor = mailboxExecutor;
public boolean useInterruptibleTimers() {
return true;
}

@Override
public void open() throws Exception {
super.open();
if (getTimeServiceManager().isPresent()) {
this.watermarkProcessor =
new MailboxWatermarkProcessor(
output, mailboxExecutor, getTimeServiceManager().get());
}
public void setMailboxExecutor(MailboxExecutor mailboxExecutor) {
super.setMailboxExecutor(mailboxExecutor);
this.mailboxExecutor = mailboxExecutor;
}

@Override
Expand All @@ -212,15 +206,6 @@ public void processElement(StreamRecord<String> element) {
}
}

@Override
public void processWatermark(Watermark mark) throws Exception {
if (watermarkProcessor == null) {
super.processWatermark(mark);
} else {
watermarkProcessor.emitWatermarkInsideMailbox(mark);
}
}

@Override
public void onEventTime(InternalTimer<String, String> timer) throws Exception {
mailboxExecutor.execute(
Expand Down