Skip to content
This repository has been archived by the owner on Mar 3, 2023. It is now read-only.

Commit

Permalink
changing watermark generator to be triggered by tick tuples for event…
Browse files Browse the repository at this point in the history
… time based windowing (#2370)

* changing watermark generator to be triggered by tick tuples for event time based windowing

* fixing unit test
  • Loading branch information
jerrypeng authored and srkukarni committed Oct 1, 2017
1 parent f8cd589 commit bf9fe93
Show file tree
Hide file tree
Showing 3 changed files with 16 additions and 45 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -201,7 +201,9 @@ private WindowManager<Tuple> initWindowManager(WindowLifecycleListener<Tuple>
} else {
watermarkInterval = DEFAULT_WATERMARK_EVENT_INTERVAL_MS;
}
waterMarkEventGenerator = new WaterMarkEventGenerator<>(manager, watermarkInterval,
// Use tick tuple to perodically generate watermarks
Config.setTickTupleFrequencyMs(topoConf, watermarkInterval);
waterMarkEventGenerator = new WaterMarkEventGenerator<>(manager,
maxLagMs, getComponentStreams(context));
} else {
if (topoConf.containsKey(WindowingConfigs.TOPOLOGY_BOLTS_LATE_TUPLE_STREAM)) {
Expand Down Expand Up @@ -310,8 +312,10 @@ protected void doPrepare(Map<String, Object> topoConf, TopologyContext context,
public void execute(Tuple input) {
if (TupleUtils.isTick(input)) {
long currTime = System.currentTimeMillis();
triggerPolicy.track(new TimerEvent<>(input, currTime));
evictionPolicy.track(new TimerEvent<>(input, currTime));
windowManager.add(new TimerEvent<>(input, currTime));
if (isTupleTs()) {
waterMarkEventGenerator.run();
}
} else {
if (isTupleTs()) {
long ts = timestampExtractor.extractTimestamp(input);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,14 +17,7 @@
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.logging.Logger;

import com.twitter.heron.api.exception.FailedException;
import com.twitter.heron.api.generated.TopologyAPI;

/**
Expand All @@ -33,33 +26,27 @@
* across all the input streams (minus the lag). Once a watermark event is emitted
* any tuple coming with an earlier timestamp can be considered as late events.
*/
public class WaterMarkEventGenerator<T> implements Runnable {
private static final Logger LOG = Logger.getLogger(WaterMarkEventGenerator.class.getName());
public class WaterMarkEventGenerator<T> {
private final WindowManager<T> windowManager;
private final int eventTsLag;
private final Set<TopologyAPI.StreamId> inputStreams;
private final Map<TopologyAPI.StreamId, Long> streamToTs;
private final ScheduledExecutorService executorService;
private final int interval;
private ScheduledFuture<?> executorFuture;
private volatile long lastWaterMarkTs;
private boolean started = false;

/**
* Creates a new WatermarkEventGenerator.
*
* @param windowManager The window manager this generator will submit watermark events to
* @param intervalMs The generator will check if it should generate a watermark event with this
* interval
* @param eventTsLagMs The max allowed lag behind the last watermark event before an event is
* considered late
* @param inputStreams The input streams this generator is expected to handle
*/
public WaterMarkEventGenerator(WindowManager<T> windowManager, int intervalMs, int
eventTsLagMs, Set<TopologyAPI.StreamId> inputStreams) {
public WaterMarkEventGenerator(WindowManager<T> windowManager, int eventTsLagMs,
Set<TopologyAPI.StreamId> inputStreams) {
this.windowManager = windowManager;
streamToTs = new ConcurrentHashMap<>();
executorService = Executors.newSingleThreadScheduledExecutor();
this.interval = intervalMs;
this.eventTsLag = eventTsLagMs;
this.inputStreams = inputStreams;
}
Expand All @@ -74,22 +61,16 @@ public boolean track(TopologyAPI.StreamId stream, long ts) {
if (currentVal == null || ts > currentVal) {
streamToTs.put(stream, ts);
}
checkFailures();
return ts >= lastWaterMarkTs;
}

@Override
@SuppressWarnings("IllegalCatch")
public void run() {
try {
if (started) {
long waterMarkTs = computeWaterMarkTs();
if (waterMarkTs > lastWaterMarkTs) {
this.windowManager.add(new WaterMarkEvent<>(waterMarkTs));
lastWaterMarkTs = waterMarkTs;
}
} catch (Throwable th) {
LOG.severe(String.format("Failed while processing watermark event\n%s", th));
throw th;
}
}

Expand All @@ -108,22 +89,7 @@ private long computeWaterMarkTs() {
return ts - eventTsLag;
}

private void checkFailures() {
if (executorFuture != null && executorFuture.isDone()) {
try {
executorFuture.get();
} catch (InterruptedException ex) {
LOG.severe(String.format("Got exception:\n%s", ex));
throw new FailedException(ex);
} catch (ExecutionException ex) {
LOG.severe(String.format("Got exception:\n%s", ex));
throw new FailedException(ex.getCause());
}
}
}

public void start() {
this.executorFuture = executorService.scheduleAtFixedRate(this, interval, interval, TimeUnit
.MILLISECONDS);
started = true;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@ public void add(Event<Integer> event) {
}
};
// set watermark interval to a high value and trigger manually to fix timing issues
waterMarkEventGenerator = new WaterMarkEventGenerator<>(windowManager, 100000, 5, Collections
waterMarkEventGenerator = new WaterMarkEventGenerator<>(windowManager, 5, Collections
.singleton(streamId("s1")));
waterMarkEventGenerator.start();
}
Expand Down Expand Up @@ -77,7 +77,8 @@ public void testTrackTwoStreams() throws Exception {
Set<TopologyAPI.StreamId> streams = new HashSet<>();
streams.add(streamId("s1"));
streams.add(streamId("s2"));
waterMarkEventGenerator = new WaterMarkEventGenerator<>(windowManager, 100000, 5, streams);
waterMarkEventGenerator = new WaterMarkEventGenerator<>(windowManager, 5, streams);
waterMarkEventGenerator.start();

waterMarkEventGenerator.track(streamId("s1"), 100);
waterMarkEventGenerator.track(streamId("s1"), 110);
Expand Down

0 comments on commit bf9fe93

Please sign in to comment.