109
Matt Zimmer QCon San Francisco 14 November | 2017 Custom, Complex Windows at Scale Using Apache Flink @zimmermatt

at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

  • Upload
    others

  • View
    4

  • Download
    0

Embed Size (px)

Citation preview

Page 1: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

Matt ZimmerQCon San Francisco14 November | 2017

Custom, Complex Windowsat Scale Using Apache Flink

@zimmermatt

Page 2: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

Agenda.

● Motivating Use Cases.● Window Requirements.● The Solution (Conceptual).● Event Processing Flow.● Apache Flink Window API Walk-Through.● The Solution (Detail).● Pitfalls to Watch Out For.● Alternative Implementations.● Questions.

@zimmermatt

Page 3: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

● Motivating Use Cases.● Window Requirements.● The Solution (Conceptual).● Event Processing Flow.● Apache Flink Window API Walk-Through.● The Solution in (Detail).● Pitfalls to Watch Out For.● Alternative Implementations.● Questions.

@zimmermatt

Page 4: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Time

User A

User B

User C

Page 5: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Use Case: Most Simple.

Time

User A

User B

User C

t4 t3 t1 t2 t5

t4t3t1 t2t7t5 t6

t3t1 t2 t5 t9 t7t8

Page 6: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Use Case: Most Simple.

Time

User A

User B

User C

t4 t3 t1 t2 t5

t4t3t1 t2t7t5 t6

t3t1 t2 t5 t9 t7t8

Page 7: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Use Case: More Complex.

Time

User A

User B

User C

t4 t3 t1 t2 t5

t4t3t1 t2t7t5 t6

t3t1 t2 t5 t9 t7t8

Page 8: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Use Case: More Complex.

Time

User A

User B

User C

t4 t3 t1 t2 t5

t4t3t1 t2t7t5 t6

t3t1 t2 t5 t9 t7t8

Page 9: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Use Case: Most Complex.

Time

User A

User B

User C

t4 t3 t1 t2 t5

t3t1 t2 t5

Page 10: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Use Case: Most Complex.

Time

User A

User B

User C

t4 t3 t1 t2 t5

t3t1 t2 t5

Page 11: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

● Events

○ Millions per second

○ 100s billions per day

● Data Flowing In

○ 10s of GiB per second

○ Low (single digit) PiB per day

● State

○ 10s of TiB

Targeted Scale.

@zimmermatt

Page 12: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

● Motivating Use Cases.● Window Requirements.● The Solution (Conceptual).● Event Processing Flow.● Apache Flink Window API Walk-Through.● The Solution in (Detail).● Pitfalls to Watch Out For.● Alternative Implementations.● Questions.

@zimmermatt

Page 13: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

● Unaligned windows

● Bounded by event type

● Handle out of order events

● Emit early results

● Capture relevant events; ignore the rest

Window Requirements.

@zimmermatt

Page 14: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

Can we use a standard window type?

@zimmermatt

Page 15: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Tumbling Window?

Time

User A

User B

User C

Page 16: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Sliding Window?

Time

User A

User B

User C

Page 17: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Sliding Window?

Time

User A

Page 18: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Sliding Window?

Time

User A

Page 19: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Sliding Window?

Time

User A

Page 20: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Sliding Window?

Time

User A

Page 21: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Apache Beam Session Window?

Time

User A

User B

User C

Page 22: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Apache Beam Session Window?

Time

User C

Page 23: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Apache Beam Session Window?

Time

User C

Page 24: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Apache Beam Session Window?

Time

User C

Page 25: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Apache Beam Session Window?

Time

User C

Page 26: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Apache Beam Session Window?

Time

User C

Page 27: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Apache Beam Session Window?

Time

User C

Page 28: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

● Motivating Use Cases.● Window Requirements.● The Solution (Conceptual).● Event Processing Flow.● Apache Flink Window API Walk-Through.● The Solution in (Detail).● Pitfalls to Watch Out For.● Alternative Implementations.● Questions.

@zimmermatt

Page 29: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

● Unaligned windows

● Bounded by event type

● Handle out of order events

● Emit early results

● Capture relevant events; ignore the rest

@zimmermatt

Window Requirements Redux.

Page 30: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

The solutionat 10,000 feet.

@zimmermatt

Page 31: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

The Solution (Conceptual).

Time

User A

User B

User C

t4 t3 t1 t2 t5

t3t1 t2 t5

Page 32: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Time

User A

User B

User C

t4 t3 t1 t2 t5

t3t1 t2 t5

The Solution (Conceptual).

Page 33: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

The Solution (Conceptual).

Time

User At4 t3 t1 t2 t5

Page 34: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

The Solution (Conceptual).

Time

User At4 t3 t1 t2 t5

Page 35: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

The Solution (Conceptual).

Time

User At4 t3 t1 t2 t5

Page 36: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

The Solution (Conceptual).

Time

User At4 t3 t1 t2 t5

Watermark

Page 37: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

The Solution (Conceptual).

Time

User At4 t3 t1 t2 t5

Watermark

Page 38: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

The Solution (Conceptual).

Time

User At4 t3 t1 t2 t5

Watermark

Page 39: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

The Solution (Conceptual).

Time

User At4 t3 t1 t2 t5

Watermark

Page 40: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

The Solution (Conceptual).

Time

User At4 t3 t1 t2 t5

Watermark

Page 41: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

The Solution (Conceptual).

Time

User At4 t3 t1 t2 t5

Watermark

Page 42: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

The Solution (Conceptual).

Time

User At4 t3 t1 t2 t5

Watermark

Page 43: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

The Solution (Conceptual).

Time

User At4 t3 t1 t2 t5

Watermark

Page 44: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

The Solution (Conceptual).

Time

User At4 t3 t1 t2 t5

Watermark

Page 45: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

● Motivating Use Cases.● Window Requirements.● The Solution (Conceptual).● Event Processing Flow.● Apache Flink Window API Walk-Through.● The Solution in (Detail).● Pitfalls to Watch Out For.● Alternative Implementations.● Questions.

@zimmermatt

Page 46: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

1. Window assigner.

@zimmermatt

Event processing flow.

Page 47: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

1. Window assigner.

a. Assign Event to Window(s) (assignWindows).

@zimmermatt

Event processing flow.

Page 48: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

1. Window assigner.

a. Assign Event to Window(s) (assignWindows).

b. Merge Windows (mergeWindows).

@zimmermatt

Event processing flow.

Page 49: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

1. Window assigner.

a. Assign Event to Window(s) (assignWindows).

b. Merge Windows (mergeWindows).

2. Trigger Handlers.

@zimmermatt

Event processing flow.

Page 50: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

1. Window assigner.

a. Assign Event to Window(s) (assignWindows).

b. Merge Windows (mergeWindows).

2. Trigger Handlers.

a. Element (onElement).

@zimmermatt

Event processing flow.

Page 51: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

1. Window assigner.

a. Assign Event to Window(s) (assignWindows).

b. Merge Windows (mergeWindows).

2. Trigger Handlers.

a. Element (onElement).

b. Merge (onMerge).

@zimmermatt

Event processing flow.

Page 52: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

1. Window assigner.

a. Assign Event to Window(s) (assignWindows).

b. Merge Windows (mergeWindows).

2. Trigger Handlers.

a. Element (onElement).

b. Merge (onMerge).

3. Trigger Timers.

@zimmermatt

Event processing flow.

Page 53: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

1. Window assigner.

a. Assign Event to Window(s) (assignWindows).

b. Merge Windows (mergeWindows).

2. Trigger Handlers.

a. Element (onElement).

b. Merge (onMerge).

3. Trigger Timers.

a. Processing Time (onProcessingTime).

@zimmermatt

Event processing flow.

Page 54: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

1. Window assigner.

a. Assign Event to Window(s) (assignWindows).

b. Merge Windows (mergeWindows).

2. Trigger Handlers.

a. Element (onElement).

b. Merge (onMerge).

3. Trigger Timers.

a. Processing Time (onProcessingTime).

b. Event Time (onEventTime).

@zimmermatt

Event processing flow.

Page 55: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

1. Window assigner.

a. Assign Event to Window(s) (assignWindows).

b. Merge Windows (mergeWindows).

2. Trigger Handlers.

a. Element (onElement).

b. Merge (onMerge).

3. Trigger Timers.

a. Processing Time (onProcessingTime).

b. Event Time (onEventTime).

4. Evictor.

@zimmermatt

Event processing flow.

Page 56: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

1. Window assigner.

a. Assign Event to Window(s) (assignWindows).

b. Merge Windows (mergeWindows).

2. Trigger Handlers.

a. Element (onElement).

b. Merge (onMerge).

3. Trigger Timers.

a. Processing Time (onProcessingTime).

b. Event Time (onEventTime).

4. Evictor.

a. Before (evictBefore).

@zimmermatt

Event processing flow.

Page 57: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

1. Window assigner.

a. Assign Event to Window(s) (assignWindows).

b. Merge Windows (mergeWindows).

2. Trigger Handlers.

a. Element (onElement).

b. Merge (onMerge).

3. Trigger Timers.

a. Processing Time (onProcessingTime).

b. Event Time (onEventTime).

4. Evictor.

a. Before (evictBefore).

b. Evaluate Window (WindowFunction#apply).

@zimmermatt

Event processing flow.

Page 58: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

1. Window assigner.

a. Assign Event to Window(s) (assignWindows).

b. Merge Windows (mergeWindows).

2. Trigger Handlers.

a. Element (onElement).

b. Merge (onMerge).

3. Trigger Timers.

a. Processing Time (onProcessingTime).

b. Event Time (onEventTime).

4. Evictor.

a. Before (evictBefore).

b. Evaluate Window (WindowFunction#apply).

c. After (evictAfter).

@zimmermatt

Event processing flow.

Page 59: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

● Motivating Use Cases.● Window Requirements.● The Solution (Conceptual).● Event Processing Flow.● Apache Flink Window API Walk-Through.● The Solution in (Detail).● Pitfalls to Watch Out For.● Alternative Implementations.● Questions.

@zimmermatt

Page 60: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Window API: Window.package org.apache.flink.streaming.api.windowing.windows;

public abstract class Window {

public abstract long maxTimestamp();}

* If you implement Window, you’ll need to provide a TypeSerializer implementation for it.

Page 61: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

package org.apache.flink.streaming.api.windowing.assigners;

public abstract class WindowAssigner<T, W extends Window> implements Serializable {

public abstract Collection<W> assignWindows(T element,

long timestamp,

WindowAssignerContext context);

public abstract Trigger<T, W> getDefaultTrigger(StreamExecutionEnvironment env);

public abstract TypeSerializer<W> getWindowSerializer(ExecutionConfig executionConfig);

public abstract boolean isEventTime();

public abstract static class WindowAssignerContext {

public abstract long getCurrentProcessingTime();

}

}

@zimmermatt

Window API: WindowAssigner.

Page 62: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Window API: MergingWindowAssigner.package org.apache.flink.streaming.api.windowing.assigners;

public abstract class MergingWindowAssigner<T, W extends Window>

extends WindowAssigner<T, W> {

public abstract void mergeWindows(Collection<W> windows,

MergeCallback<W> callback);

public interface MergeCallback<W> {

void merge(Collection<W> toBeMerged, W mergeResult);

}

}

Page 63: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Window API: Trigger.package org.apache.flink.streaming.api.windowing.triggers;

public abstract class Trigger<T, W extends Window> implements Serializable {

...

public abstract TriggerResult onElement(T element,

long timestamp,

W window,

TriggerContext ctx) throws Exception;

public boolean canMerge() { return false; }

public void onMerge(W window,

OnMergeContext ctx) throws Exception { throws by default }

...

}

Page 64: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Window API: Trigger.package org.apache.flink.streaming.api.windowing.triggers;

public abstract class Trigger<T, W extends Window> implements Serializable {

...

public abstract TriggerResult onProcessingTime(long time,

W window,

TriggerContext ctx) throws Exception;

public abstract TriggerResult onEventTime(long time,

W window,

TriggerContext ctx) throws Exception;

...

}

Page 65: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Window API: Trigger.package org.apache.flink.streaming.api.windowing.triggers;

public abstract class Trigger<T, W extends Window> implements Serializable {

...

public abstract void clear(W window, TriggerContext ctx) throws Exception;

public interface TriggerContext { ... }

public interface OnMergeContext extends TriggerContext { ... }

...

}

Page 66: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Window API: Trigger.package org.apache.flink.streaming.api.windowing.triggers;

public abstract class Trigger<T, W extends Window> implements Serializable {

...

public interface TriggerContext {

long getCurrentProcessingTime();

MetricGroup getMetricGroup();

long getCurrentWatermark();

void registerProcessingTimeTimer(long time);

void registerEventTimeTimer(long time);

void deleteProcessingTimeTimer(long time);

void deleteEventTimeTimer(long time);

<S extends State> S getPartitionedState(StateDescriptor<S, ?> stateDescriptor);

}

public interface OnMergeContext extends TriggerContext {

<S extends MergingState<?, ?>> void mergePartitionedState(StateDescriptor<S, ?> stateDescriptor);

}

}

Page 67: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

package org.apache.flink.streaming.api.windowing.evictors;

public interface Evictor<T, W extends Window> extends Serializable {

void evictBefore(Iterable<TimestampedValue<T>> elements,

int size,

W window,

EvictorContext evictorContext);

void evictAfter(Iterable<TimestampedValue<T>> elements,

int size,

W window,

EvictorContext evictorContext);

...

}

Window API: Evictor.

Page 68: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

package org.apache.flink.streaming.api.windowing.evictors;

public interface Evictor<T, W extends Window> extends Serializable {

...

interface EvictorContext {

long getCurrentProcessingTime();

MetricGroup getMetricGroup();

long getCurrentWatermark();

}

}

Window API: Evictor.

Page 69: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

The solutionin detail.

@zimmermatt

Page 70: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Custom Window: WindowAssigner.public class CustomWindowAssigner<E extends CustomEvent> extends MergingWindowAssigner<E, CustomWindow<E>> { ... @Override

public Collection<CustomWindow<E>> assignWindows(E element,

long timestamp,

WindowAssignerContext context) {

return Collections.singletonList(new CustomWindow<>(element, timeoutDuration));

}

...

}

Page 71: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Custom Window: Window.public class CustomWindow<E extends CustomEvent> extends Window { ... @Override

public long maxTimestamp() {

return maxTimestamp;

} ...}

Page 72: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Custom Window: Window.public class CustomWindow<E extends CustomEvent> extends Window { ...

@Override

public boolean equals(Object o) {

// important: equals implementation must compare using “value” semantics

}

@Override

public int hashCode() {

// important: same for hashCode implementation

}

...}

Page 73: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Custom Window: Window.public class CustomWindow<E extends CustomEvent> extends Window { ...

public static class Serializer<T extends CustomEvent>

extends TypeSerializer<CustomWindow<T>> {

... }

...}

Page 74: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Custom Window: Window.public class CustomWindow<E extends CustomEvent> extends Window { ... public static class Serializer<T extends CustomEvent>

extends TypeSerializer<CustomWindow<T>> {

@Override

public boolean isImmutableType() { return true; }

... }

...}

Page 75: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

public class CustomWindow<E extends CustomEvent> extends Window { ... public static class Serializer<T extends CustomEvent> extends TypeSerializer<CustomWindow<T>> { ...

@Override

public TypeSerializer<CustomWindow<T>> duplicate() { return this; }

@Override

public CustomWindow<T> createInstance() { return null; }

@Override

public CustomWindow<T> copy(CustomWindow<T> from) { return from; }

@Override

public CustomWindow<T> copy(CustomWindow<T> from, CustomWindow<T> reuse) { return from; }

@Override

public int getLength() { return -1; }

} ...}

@zimmermatt

Custom Window: Window.

Page 76: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

public class CustomWindow<E extends CustomEvent> extends Window { ... public static class Serializer<T extends CustomEvent> extends TypeSerializer<CustomWindow<T>> { ...

public void serialize(CustomWindow<T> record, DataOutputView target)

throws IOException {

serializeStartEvent(record, target);

target.writeLong(record.getDuration().toMillis());

target.writeBoolean(record.evaluate());

final boolean hasEndEventData = record.getEndEventData() != null;

target.writeBoolean(hasEndEventData);

if (hasEndEventData) serializeEndEvent(record, target);

}

} ...}

Custom Window: Window.

Page 77: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

public class CustomWindow<E extends CustomEvent> extends Window { ... public static class Serializer<T extends CustomEvent> extends TypeSerializer<CustomWindow<T>> { ...

@Override

public CustomWindow<T> deserialize(DataInputView source) throws IOException {

final T startEvent = deserializeStartEvent(source);

final Duration duration = Duration.ofMillis(source.readLong());

final boolean evaluate = source.readBoolean();

final boolean hasEndEventData = source.readBoolean();

final T endEvent = hasEndEventData ? deserializeEndEvent(source) : null;

return new CustomWindow<>(startEvent, duration, endEvent, evaluate);

}

} ...}

Custom Window: Window.

Page 78: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

public class CustomWindow<E extends CustomEvent> extends Window { ... public static class Serializer<T extends CustomEvent> extends TypeSerializer<CustomWindow<T>> { ...

@Override

public CustomWindow<T> deserialize(CustomWindow<T> reuse,

DataInputView source) throws IOException {

return reuse != null ? reuse : deserialize(source);

}

} ...}

Custom Window: Window.

Page 79: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

public class CustomWindow<E extends CustomEvent> extends Window { ... public static class Serializer<T extends CustomEvent> extends TypeSerializer<CustomWindow<T>> { ...

@Override

public void copy(DataInputView source, DataOutputView target) throws IOException {

// slightly less efficient, but more maintainable

CustomWindow<T> deserializedWindow = deserialize(source);

serialize(deserializedWindow, target);

}

} ...}

Custom Window: Window.

Page 80: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

public class CustomWindow<E extends CustomEvent> extends Window { ... public static class Serializer<T extends CustomEvent> extends TypeSerializer<CustomWindow<T>> { ...

@Override

public boolean equals(Object obj) { return obj instanceof Serializer; }

@Override

public boolean canEqual(Object obj) { return obj instanceof Serializer; }

@Override

public int hashCode() { return 0; }

} ...}

Custom Window: Window.

Page 81: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

public class CustomWindow<E extends CustomEvent> extends Window { ... public static class Serializer<T extends CustomEvent> extends TypeSerializer<CustomWindow<T>> { ...

@Override

public TypeSerializerConfigSnapshot snapshotConfiguration() { ... }

@Override

public CompatibilityResult<CustomWindow<T>> ensureCompatibility(

TypeSerializerConfigSnapshot configSnapshot) {

return CompatibilityResult.requiresMigration();

}

private static class CustomWindowSerializerConfigSnapshot extends TypeSerializerConfigSnapshot { ...

}

} ...}

Custom Window: Window.

Page 82: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Custom Window: Window.public class CustomWindow<E extends CustomEvent> extends Window { … public CustomWindow(@Nonnull D primaryEventData, @Nonnull Duration timeoutDuration, D endEventData, boolean evaluate) { ...

this.endTimestamp = endEventData != null ? endEventData.getTimestamp() : maxTimestamp; ... } ...

public boolean evaluate() { return evaluate; }

public Instant startTimestamp() { return primaryEventData.getTimestamp(); }

public Instant endTimestamp() { return endTimestamp; }

} ...}

Page 83: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Custom Window: WindowAssigner.public class CustomWindowAssigner<E extends CustomEvent> extends MergingWindowAssigner<E, CustomWindow<E>> { ... @Override

public void mergeWindows(Collection<CustomWindow<E>> mergeCandidates,

MergeCallback<CustomWindow<E>> mergeCallback) {

final CustomWindow<E> sessionWindow = calculateSessionWindow(mergeCandidates);

final Collection<CustomWindow<E>> inWindow = filterWithinWindow(mergeCandidates);

// MergeCallback#merge implementation expects 2 or more.

if (inWindow.size() > 1) {

mergeCallback.merge(inWindow, sessionWindow);

}

} ...}

Page 84: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Custom Window: WindowAssigner.public class CustomWindowAssigner<E extends CustomEvent> extends MergingWindowAssigner<E, CustomWindow<E>> { ... private CustomWindow<E> calculateSessionWindow(Collection<CustomWindow<E>> mergeCandidates) {

CustomWindow<E> startEventWindow = findStartEventWindow(mergeCandidates);

if (startEventWindow != null) { // valid window

… } else { // exploratory window ... } } ...}

Page 85: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Custom Window: WindowAssigner.if (startEventWindow != null) { // valid window

CustomWindow<E> endEvent = findEndEventWindow(mergeCandidates); // can return null

return new CustomWindow<>(startEventWindow.getEvent, timeoutDuration, endEvent,

true); // fire (send this one to the WindowFunction)

} else { // exploratory window ...}

Page 86: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Custom Window: WindowAssigner.if (startEventWindow != null) { // valid window

...

} else { // exploratory window

CustomWindow<E> window = findClosestToMidpointByStartTime(mergeCandidates);

return new CustomWindow(window.getEvent, exploratoryDuration,

false) // just purge without firing

}

Page 87: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Time

User At4 t3 t1 t2 t5

Watermark

Page 88: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Time

User At4 t3 t1 t2 t5

Watermark

Page 89: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Time

User At4 t3 t1 t2 t5

Watermark

Page 90: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

public class CustomWindowTrigger<E extends CustomEvent> extends Trigger<E, CustomWindow<E>> { ...

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

@Override public void onMerge(CustomWindow<E> window, OnMergeContext onMergeContext) throws Exception { onMergeContext.registerEventTimeTimer(window.endTimestamp().toEpochMilli()); } ...}

@zimmermatt

Custom Window: Trigger.

Page 91: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Custom Window: Trigger.public class CustomWindowTrigger<E extends CustomEvent> extends Trigger<E, CustomWindow<E>> { ... @Override public TriggerResult onElement(E element, long timestamp, CustomWindow<E> window, TriggerContext triggerContext) throws Exception { final TriggerResult triggerResult; final ValueState<Boolean> windowClosedState = triggerContext.getPartitionedState(windowClosedDescriptor); final long endTimestamp = window.endTimestamp().toEpochMilli();

if (triggerContext.getCurrentWatermark() >= endTimestamp) {

triggerResult = windowClosedState.value() ? TriggerResult.CONTINUE : triggerWindow(triggerContext, windowClosedState, window); } else { ... } return triggerResult; } ...}

Page 92: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Custom Window: Trigger.public class CustomWindowTrigger<E extends CustomEvent> extends Trigger<E, CustomWindow<E>> { ...

private TriggerResult triggerWindow(TriggerContext triggerContext, ValueState<Boolean> windowClosedState, CustomWindow<E> window) throws IOException {

windowClosedState.update(Boolean.TRUE); removeEarlyFiringTimer(triggerContext); return window.evaluate() ? TriggerResult.FIRE_AND_PURGE : TriggerResult.PURGE; }

private void removeEarlyFiringTimer(TriggerContext triggerContext) throws IOException { final ValueState<Long> earlyFiringState = triggerContext.getPartitionedState(earlyFiringDescriptor);

if (earlyFiringState.value() > 0) { triggerContext.deleteProcessingTimeTimer(earlyFiringState.value()); // set to -1L to differentiate from the default value earlyFiringState.update(-1L); } } ...}

Page 93: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Custom Window: Trigger.public class CustomWindowTrigger<E extends CustomEvent> extends Trigger<E, CustomWindow<E>> { ...

@Override public TriggerResult onElement(E element, long timestamp, CustomWindow<E> window, TriggerContext triggerContext) throws Exception { final TriggerResult triggerResult; final long endTimestamp = window.endTimestamp().toEpochMilli(); final ValueState<Boolean> windowClosedState = triggerContext.getPartitionedState(windowClosedDescriptor); if ...

} else { windowClosedState.update(Boolean.FALSE); triggerResult = TriggerResult.CONTINUE; triggerContext.registerEventTimeTimer(endTimestamp); registerEarlyFiringTimerIfNecessary(window, triggerContext); } return triggerResult; } ...}

Page 94: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Custom Window: Trigger.public class CustomWindowTrigger<E extends CustomEvent> extends Trigger<E, CustomWindow<E>> { ...

private void registerEarlyFiringTimerIfNecessary(CustomWindow<E> window, TriggerContext triggerContext) throws IOException {

if (!window.evaluate() || earlyFiringInterval.toMillis() < 1) return;

final ValueState<Long> earlyFiringState = triggerContext.getPartitionedState(earlyFiringDescriptor);

if (earlyFiringState.value() == Long.MIN_VALUE) { final Long newEarlyFiringTimestamp = System.currentTimeMillis() + earlyFiringInterval.toMillis(); if (newEarlyFiringTimestamp < window.endTimestamp().toEpochMilli()) { triggerContext.registerProcessingTimeTimer(newEarlyFiringTimestamp); earlyFiringState.update(newEarlyFiringTimestamp); } } } ...}

Page 95: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

private void registerEarlyFiringTimerIfNecessary(CustomWindow<E> window, TriggerContext triggerContext) throws IOException { if (!window.evaluate() || earlyFiringInterval.toMillis() < 1) return;

final ValueState<Long> earlyFiringState = triggerContext.getPartitionedState(earlyFiringDescriptor);

if (earlyFiringState.value() == Long.MIN_VALUE) { final Long newEarlyFiringTimestamp = System.currentTimeMillis() + earlyFiringInterval.toMillis();

if (newEarlyFiringTimestamp < window.endTimestamp().toEpochMilli()) { triggerContext.registerProcessingTimeTimer(newEarlyFiringTimestamp); earlyFiringState.update(newEarlyFiringTimestamp); } }

Custom Window: Trigger.

Page 96: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Custom Window: Trigger.public class CustomWindowTrigger<E extends CustomEvent> extends Trigger<E, CustomWindow<E>> { ...

@Override public TriggerResult onEventTime(long time, CustomWindow<E> window, TriggerContext triggerContext) throws Exception {

if (time != window.endTimestamp().toEpochMilli()) { return TriggerResult.CONTINUE; }

final ValueState<Boolean> windowClosedState = triggerContext.getPartitionedState(windowClosedDescriptor); if (windowClosedState.value()) { return TriggerResult.CONTINUE; }

return triggerWindow(triggerContext, windowClosedState, window); } ...}

Page 97: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Custom Window: Trigger.public class CustomWindowTrigger<E extends CustomEvent> extends Trigger<E, CustomWindow<E>> { ...

@Override public TriggerResult onEventTime(long time, CustomWindow<E> window, TriggerContext triggerContext) throws Exception {

if (time != window.endTimestamp().toEpochMilli()) { return TriggerResult.CONTINUE; }

final ValueState<Boolean> windowClosedState = triggerContext.getPartitionedState(windowClosedDescriptor); if (windowClosedState.value()) { return TriggerResult.CONTINUE; }

return triggerWindow(triggerContext, windowClosedState, window); } ...}

Page 98: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Custom Window: Trigger.public class CustomWindowTrigger<E extends CustomEvent> extends Trigger<E, CustomWindow<E>> { ...

@Override public TriggerResult onProcessingTime(long time, CustomWindow<E> window, TriggerContext triggerContext) throws Exception {

TriggerResult triggerResult = TriggerResult.CONTINUE; if (window.evaluate()) { ... } return triggerResult; } ...}

Page 99: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

Custom Window: Trigger.

if (window.evaluate()) { // Update early firing final ValueState<Long> earlyFiringState = triggerContext.getPartitionedState(earlyFiringDescriptor);

final Long newEarlyFiringTimestamp = earlyFiringState.value() + earlyFiringInterval.toMillis();

if (newEarlyFiringTimestamp < window.endTimestamp().toEpochMilli()) { triggerContext.registerProcessingTimeTimer(newEarlyFiringTimestamp); earlyFiringState.update(newEarlyFiringTimestamp); }

triggerResult = TriggerResult.FIRE; } return triggerResult;

Page 100: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

● Motivating Use Cases.● Window Requirements.● The Solution (Conceptual).● Event Processing Flow.● Apache Flink Window API Walk-Through.● The Solution (Detail).● Pitfalls to Watch Out For.● Alternative Implementations.● Questions.

Page 101: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

Pitfall:

@zimmermatt

Window equals / hashCode.

Page 102: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

Pitfall:

@zimmermatt

Metrics and Logs.

Page 103: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

Pitfall:

@zimmermatt

Event Design.

Page 104: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

Pitfall:

@zimmermatt

Large State.

Page 105: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

● Motivating Use Cases.● Window Requirements.● The Solution (Conceptual).● Event Processing Flow.● Apache Flink Window API Walk-Through.● The Solution (Detail).● Pitfalls to Watch Out For.● Alternative Implementations.● Questions.

Page 106: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

Alternative:

@zimmermatt

ProcessFunction

Page 107: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

Alternative:

@zimmermatt

CEP Library

Page 108: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

@zimmermatt

● Motivating Use Cases.● Window Requirements.● The Solution (Conceptual).● Event Processing Flow.● Apache Flink Window API Walk-Through.● The Solution (Detail).● Pitfalls to Watch Out For.● Alternative Implementations.● Questions.

Page 109: at Scale Using Apache Flink Custom, Complex …...package org.apache.flink.streaming.api.windowing.assigners; public abstract class WindowAssigner implements

Thank You!

@zimmermatt