From fd346a240be1069e326df68200a125637f012509 Mon Sep 17 00:00:00 2001 From: vjagadish1989 Date: Tue, 6 Dec 2016 14:19:42 -0800 Subject: [PATCH 1/6] clean up old files --- .../apache/samza/operators/MessageStream.java | 4 + .../samza/operators/StateStoreImpl.java | 56 --------- .../samza/operators/spec/OperatorSpecs.java | 18 +++ .../operators/spec/WindowOperatorSpec.java | 119 ------------------ 4 files changed, 22 insertions(+), 175 deletions(-) delete mode 100644 samza-operator/src/main/java/org/apache/samza/operators/StateStoreImpl.java delete mode 100644 samza-operator/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java diff --git a/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java b/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java index d18536bbbb..d1c0fd80a0 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java +++ b/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java @@ -99,6 +99,10 @@ public interface MessageStream { , WM extends WindowOutput> MessageStream window( Window window); + > MessageStream window( + org.apache.samza.operators.windows.experimental.Window window); + + /** * Joins this {@link MessageStream} with another {@link MessageStream} using the provided pairwise {@link JoinFunction}. *

diff --git a/samza-operator/src/main/java/org/apache/samza/operators/StateStoreImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/StateStoreImpl.java deleted file mode 100644 index 2572f146f7..0000000000 --- a/samza-operator/src/main/java/org/apache/samza/operators/StateStoreImpl.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.samza.operators; - -import org.apache.samza.operators.data.MessageEnvelope; -import org.apache.samza.operators.windows.StoreFunctions; -import org.apache.samza.storage.kv.Entry; -import org.apache.samza.storage.kv.KeyValueStore; -import org.apache.samza.task.TaskContext; - - -/** - * The base class for all state stores - */ -public class StateStoreImpl { - private final String storeName; - private final StoreFunctions storeFunctions; - private KeyValueStore kvStore = null; - - public StateStoreImpl(StoreFunctions store, String storeName) { - this.storeFunctions = store; - this.storeName = storeName; - } - - public void init(TaskContext context) { - this.kvStore = (KeyValueStore) context.getStore(this.storeName); - } - - public Entry getState(M m) { - SK key = this.storeFunctions.getStoreKeyFn().apply(m); - SS state = this.kvStore.get(key); - return new Entry<>(key, state); - } - - public Entry updateState(M m, Entry oldEntry) { - SS newValue = this.storeFunctions.getStateUpdaterFn().apply(m, oldEntry.getValue()); - this.kvStore.put(oldEntry.getKey(), newValue); - return new Entry<>(oldEntry.getKey(), newValue); - } -} \ No newline at end of file diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java index f622b343ea..14fc2db45a 100644 --- a/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java +++ b/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java @@ -26,6 +26,7 @@ import org.apache.samza.operators.windows.WindowState; import org.apache.samza.operators.windows.WindowFn; import org.apache.samza.operators.windows.WindowOutput; +import org.apache.samza.operators.windows.experimental.WindowDefinition; import java.util.ArrayList; import java.util.UUID; @@ -83,6 +84,23 @@ public static SinkOperatorSpec createSinkOperator return new WindowOperatorSpec<>(windowFn, OperatorSpecs.getOperatorId()); } + /** + * Creates a {@link WindowOperatorSpec}. + * + * @param window the {@link WindowDefinition} for the window. + * @param type of input {@link MessageEnvelope} + * @param type of window key + * @param type of value in the window. + * @param type of output {@link WindowOutput} + * @return the {@link WindowOperatorSpec} + */ + + public static > ExperimentalWindowOperatorSpec createWindowOperator(WindowDefinition window) { + return new ExperimentalWindowOperatorSpec<>(window, OperatorSpecs.getOperatorId()); + } + + + /** * Creates a {@link PartialJoinOperatorSpec}. * diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java deleted file mode 100644 index 2f5b1e76f2..0000000000 --- a/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java +++ /dev/null @@ -1,119 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.samza.operators.spec; - -import org.apache.samza.operators.MessageStream; -import org.apache.samza.operators.data.MessageEnvelope; -import org.apache.samza.operators.MessageStreamImpl; -import org.apache.samza.operators.windows.StoreFunctions; -import org.apache.samza.operators.windows.Trigger; -import org.apache.samza.operators.windows.WindowFn; -import org.apache.samza.operators.windows.WindowOutput; -import org.apache.samza.operators.windows.WindowState; -import org.apache.samza.storage.kv.Entry; - -import java.util.function.BiFunction; - - -/** - * Defines a window operator that takes one {@link MessageStreamImpl} as an input, accumulates the window state, - * and generates an output {@link MessageStreamImpl} with output type {@code WM} which extends {@link WindowOutput} - * - * @param the type of input {@link MessageEnvelope} - * @param the type of key in the output {@link MessageEnvelope} from the {@link WindowOperatorSpec} function - * @param the type of window state in the {@link WindowOperatorSpec} function - * @param the type of window output {@link MessageEnvelope} - */ -public class WindowOperatorSpec> implements - OperatorSpec { - - /** - * The output {@link MessageStream}. - */ - private final MessageStreamImpl outputStream; - - /** - * The window transformation function that takes {@link MessageEnvelope}s from one input stream, aggregates with the window - * state(s) from the window state store, and generate output {@link MessageEnvelope}s for the output stream. - */ - private final BiFunction, WM> transformFn; - - /** - * The state store functions for the {@link WindowOperatorSpec}. - */ - private final StoreFunctions storeFns; - - /** - * The window trigger. - */ - private final Trigger trigger; - - /** - * The unique ID of this operator. - */ - private final String operatorId; - - /** - * Constructor for {@link WindowOperatorSpec}. - * - * @param windowFn the window function - * @param operatorId auto-generated unique ID of this operator - */ - WindowOperatorSpec(WindowFn windowFn, String operatorId) { - this.outputStream = new MessageStreamImpl<>(); - this.transformFn = windowFn.getTransformFn(); - this.storeFns = windowFn.getStoreFns(); - this.trigger = windowFn.getTrigger(); - this.operatorId = operatorId; - } - - @Override - public String toString() { - return this.operatorId; - } - - @Override - public MessageStreamImpl getOutputStream() { - return this.outputStream; - } - - public StoreFunctions getStoreFns() { - return this.storeFns; - } - - public BiFunction, WM> getTransformFn() { - return this.transformFn; - } - - public Trigger getTrigger() { - return this.trigger; - } - - /** - * Method to generate the window operator's state store name - * TODO HIGH pmaheshw: should this be here? - * - * @param inputStream the input {@link MessageStreamImpl} to this state store - * @return the persistent store name of the window operator - */ - public String getStoreName(MessageStream inputStream) { - //TODO: need to get the persistent name of ds and the operator in a serialized form - return String.format("input-%s-wndop-%s", inputStream.toString(), this.toString()); - } -} From ddc52b862d086948c14f08650edf4df58bde5c57 Mon Sep 17 00:00:00 2001 From: vjagadish1989 Date: Tue, 6 Dec 2016 14:20:41 -0800 Subject: [PATCH 2/6] APIs for window and triggers --- .../windows/examples/WindowingExamples.java | 109 +++++++++ .../windows/experimental/TriggersBuilder.java | 220 ++++++++++++++++++ .../windows/experimental/Window.java | 43 ++++ .../experimental/WindowDefinition.java | 83 +++++++ .../windows/experimental/WindowKey.java | 27 +++ .../windows/experimental/Windows.java | 115 +++++++++ .../spec/ExperimentalWindowOperatorSpec.java | 42 ++++ .../operators/spec/WindowOperatorSpec.java | 119 ++++++++++ 8 files changed, 758 insertions(+) create mode 100644 samza-api/src/main/java/org/apache/samza/operators/windows/examples/WindowingExamples.java create mode 100644 samza-api/src/main/java/org/apache/samza/operators/windows/experimental/TriggersBuilder.java create mode 100644 samza-api/src/main/java/org/apache/samza/operators/windows/experimental/Window.java create mode 100644 samza-api/src/main/java/org/apache/samza/operators/windows/experimental/WindowDefinition.java create mode 100644 samza-api/src/main/java/org/apache/samza/operators/windows/experimental/WindowKey.java create mode 100644 samza-api/src/main/java/org/apache/samza/operators/windows/experimental/Windows.java create mode 100644 samza-operator/src/main/java/org/apache/samza/operators/spec/ExperimentalWindowOperatorSpec.java create mode 100644 samza-operator/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/examples/WindowingExamples.java b/samza-api/src/main/java/org/apache/samza/operators/windows/examples/WindowingExamples.java new file mode 100644 index 0000000000..e0650b042f --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/operators/windows/examples/WindowingExamples.java @@ -0,0 +1,109 @@ +package org.apache.samza.operators.windows.examples; + +import org.apache.samza.operators.MessageStream; +import org.apache.samza.operators.data.IncomingSystemMessageEnvelope; +import org.apache.samza.operators.data.MessageEnvelope; +import org.apache.samza.operators.windows.WindowOutput; +import org.apache.samza.operators.windows.experimental.TriggersBuilder; +import org.apache.samza.operators.windows.experimental.WindowKey; +import org.apache.samza.operators.windows.experimental.Windows; + +import java.util.Collection; +import java.util.function.BiFunction; +import java.util.function.Function; + +/** + * Examples for programming using the {@link org.apache.samza.operators.windows.experimental.Windows} APIs. + */ +public class WindowingExamples { + + private static final long INTERVAL_MS = 1000; + private static final long SESSION_GAP_MS = 2000; + + private static Integer parseInt(IncomingSystemMessageEnvelope msg) { + //parse an integer from the message + return 0; + } + + private static PercentileMessage compute99Percentile(Collection msg) { + //iterate over values, sort them, and compute percentiles. + return null; + } + + private static class PercentileMessage implements MessageEnvelope { + + @Override + public String getKey() { + return null; + } + + @Override + public Integer getMessage() { + return null; + } + + } + + public static void main(String[] args) { + + MessageStream integerStream = null; + + //GLOBAL TUMBLING WINDOWS WITH CUSTOM AGGREGATES: Demonstrate a custom aggregation (MAX) that returns the max element in a {@link MessageStream} + //Notes: - Aggregations are done with a tumbling window of 1000ms. + // - Early results are emitted once every 50 messages, every 4000ms in processing time. + // - Late results are emitted only when 20 late messages have accumulated. + // - Previously fired window panes are discarded + // - Messages have a maximum allowed lateness of 50000ms. + + BiFunction maxAggregator = (m, c)-> Math.max(parseInt(m), c); + MessageStream, Integer>> window1 = integerStream.window(Windows.tumblingWindow(10000, maxAggregator) + .setTriggers(new TriggersBuilder() + .withEarlyFiringsAfterCountAtleast(50) + .withEarlyFiringsEvery(4000) + .withLateFiringsAfterCountAtleast(20) + .discardFiredPanes() + .build())); + + + //KEYED SESSION WINDOW WITH PERCENTILE COMPUTATION: Demonstrate percentile computation over a Keyed Session Window on a {@link MessageStream} + //Notes: - Aggregations are done with session windows having gap SESSION_GAP_MS seconds. + // - Early results are emitted once every 50 messages, every 4000ms in processing time. + // - Late results are emitted only when 20 late messages have accumulated. + // - Previously fired window panes are accumulated + // - Messages have a maximum allowed lateness of 50000ms. + Function keyExtractor = null; + + final MessageStream windowedPercentiles = integerStream.window(Windows.keyedSessionWindow(keyExtractor, SESSION_GAP_MS) + .setTriggers(new TriggersBuilder() + .withEarlyFiringsAfterCountAtleast(50) + .withEarlyFiringsEvery(4000) + .withLateFiringsAfterCountAtleast(20) + .accumulateFiredPanes() + .build())) + .map(dataset -> compute99Percentile(dataset.getMessage())); + + //KEYED TUMBLING WINDOW WITH PERCENTILE COMPUTATION: Demonstrate percentile computation over a Keyed Tumbling Window on a {@link MessageStream} + final MessageStream tumblingPercentiles = integerStream.window(Windows.keyedTumblingWindow(keyExtractor, INTERVAL_MS) + .setTriggers(new TriggersBuilder() + .withEarlyFiringsAfterCountAtleast(50) + .accumulateFiredPanes() + .build())) + .map(dataset -> compute99Percentile(dataset.getMessage())); + + //KEYED TUMBLING WINDOW + MessageStream, Collection>> tumblingWindow = integerStream.window(Windows.keyedTumblingWindow(keyExtractor, INTERVAL_MS) + .setTriggers(new TriggersBuilder() + .withEarlyFiringsAfterCountAtleast(50) + .discardFiredPanes() + .build())); + + + /* + //A CUSTOM GLOBAL WINDOW: Demonstrates a window with custom triggering every 500 messages + final MessageStream>> customWindow = integerStream.window(Windows.customGlobalWindow().setTriggers(new TriggerSpecBuilder() + .withEarlyFiringsAfterCountAtleast(500) + .discardFiredPanes() + .build())); + */ + } +} diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/experimental/TriggersBuilder.java b/samza-api/src/main/java/org/apache/samza/operators/windows/experimental/TriggersBuilder.java new file mode 100644 index 0000000000..32ffe3423b --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/operators/windows/experimental/TriggersBuilder.java @@ -0,0 +1,220 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.samza.operators.windows.experimental; + +import org.apache.samza.operators.data.MessageEnvelope; + +import java.util.ArrayList; +import java.util.List; + + +/** + * Programmer API for specifying {@link Triggers}s for a {@link Window}. + * + * A {@link Triggers} instance specifies all the early and late triggers for a {@link Window}. It also specifies how results + * from a window relate over time - whether previously fired window results are accumulating (where every result builds + * over its previous ones) or discarding (where every result is independent). + * + * @param type of input {@link MessageEnvelope} in the window + * @param type of key in {@link MessageEnvelope} + * @param type of value in the {@link MessageEnvelope} + * + * The below example windows an input into tumbling windows of 10s, and emits early results periodically every 4s in + * processing time, or for every 50 messages. It also specifies that window results are accumulating. + * + *

+ *   {@code
+ *    MessageStream<> windowedStream = stream.window(Windows.tumblingWindow(10000)
+ *      setTriggers(new Triggers<>()
+ *       .withEarlyFiringsAfterCountAtleast(50)
+ *       .withEarlyFiringsEvery(4000)
+ *       .accumulateFiredPanes().build());
+ *    }
+ *  
+ */ +public final class TriggersBuilder { + + /* + * Is this an early, late or a default trigger. + */ + enum TriggerType { + EARLY, LATE, DEFAULT + } + + /* + * Should the trigger fire in event time or processing time. + */ + enum TimeCharacteristic { + EVENT_TIME, PROCESSING_TIME + } + + static class Trigger { + private final TriggerType type; + + Trigger(TriggerType type) { + this.type = type; + } + } + + /* + * Defines a {@link Trigger} that triggers periodically based on the specified time duration. + * + */ + static class PeriodicTimeTrigger extends Trigger { + private final long durationMs; + private final TimeCharacteristic timeCharacteristic; + + PeriodicTimeTrigger(TriggerType type, long duration, TimeCharacteristic timeCharacteristic) { + super(type); + this.durationMs = duration; + this.timeCharacteristic = timeCharacteristic; + } + } + + /* + * Defines a {@link Trigger} that triggers when there is no message in a {@link Window} for the specified gap duration. + * A late trigger specifies handling of + * late arrivals (caused typically due to skews, and upstream delays) in the data. + */ + static class TimeSinceLastMessageTrigger extends Trigger { + private final long gapMs; + private final TimeCharacteristic timeCharacteristic; + + TimeSinceLastMessageTrigger(TriggerType type, long timeout, TimeCharacteristic timeCharacteristic) { + super(type); + this.gapMs = timeout; + this.timeCharacteristic = timeCharacteristic; + } + } + + /* + * Defines a {@link Trigger} that triggers based on the count of messages in the window. + */ + + static class CountTrigger extends Trigger { + private final long count; + CountTrigger(TriggerType type , long count) { + super(type); + this.count = count; + } + } + + + private final List earlyTriggers = new ArrayList<>(); + private final List lateTriggers = new ArrayList<>(); + + private AccumulationMode accumulationMode = AccumulationMode.ACCUMULATING; + + private enum AccumulationMode { + ACCUMULATING, DISCARDING + } + + static class Triggers { + + /** + * Early and late firings, accumulation modes for a {@link Window} + */ + private final List earlyTriggers; + private final List lateTriggers; + private final AccumulationMode accumulationMode; + + Triggers(List earlyTriggers, List lateTriggers, AccumulationMode mode) { + this.earlyTriggers = earlyTriggers; + this.lateTriggers = lateTriggers; + this.accumulationMode = mode; + } + + List getEarlyTriggers() { + return earlyTriggers; + } + + List getLateTriggers() { + return lateTriggers; + } + + AccumulationMode getAccumulationMode() { + return accumulationMode; + } + } + + /** + * Add an early trigger that periodically emits results after a processing time delay. An early trigger specifies + * emission of an early result before all data for the {@link Window} have arrived. + * @param period the delay period between firings + */ + public TriggersBuilder withEarlyFiringsEvery(long period) { + earlyTriggers.add(new PeriodicTimeTrigger(TriggerType.EARLY, period, TimeCharacteristic.PROCESSING_TIME)); + return this; + } + + /** + * Add a late trigger that periodically emits results after a processing time delay. + * @param period the delay period between firings + */ + public TriggersBuilder withLateFiringsEvery(long period) { + lateTriggers.add(new PeriodicTimeTrigger(TriggerType.LATE, period, TimeCharacteristic.PROCESSING_TIME)); + return this; + } + + /** + * Add an early trigger that periodically emits results after a certain number of messages in the window. + * @param numMessages the delay period between firings An early trigger specifies + * emission of an early result before all data for the {@link Window} have arrived. + */ + public TriggersBuilder withEarlyFiringsAfterCountAtleast(long numMessages) { + earlyTriggers.add(new CountTrigger(TriggerType.EARLY, numMessages)); + return this; + } + + /** + * Add a late trigger that periodically emits results after a certain number of messages in the window. + * @param numMessages the delay period between firings + */ + + public TriggersBuilder withLateFiringsAfterCountAtleast(long numMessages) { + lateTriggers.add(new CountTrigger(TriggerType.LATE, numMessages)); + return this; + } + + /** + * Specifies that previously fired results should be accumulated. This is applicable when each window output builds + * on the previous ones. + */ + public TriggersBuilder accumulateFiredPanes() { + this.accumulationMode = AccumulationMode.ACCUMULATING; + return this; + } + + /** + * Specifies that previously fired results should be discarded. This is applicable when each window output is + * independent. + */ + public TriggersBuilder discardFiredPanes() { + this.accumulationMode = AccumulationMode.DISCARDING; + return this; + } + + /** + * Build an immutable {@link Triggers} from this {@link TriggersBuilder} + */ + public Triggers build() { + return new Triggers(earlyTriggers, lateTriggers, accumulationMode); + } +} diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/experimental/Window.java b/samza-api/src/main/java/org/apache/samza/operators/windows/experimental/Window.java new file mode 100644 index 0000000000..e0753322f6 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/operators/windows/experimental/Window.java @@ -0,0 +1,43 @@ +package org.apache.samza.operators.windows.experimental; + +import org.apache.samza.operators.data.MessageEnvelope; +import org.apache.samza.operators.windows.WindowOutput; + +/** + * A {@link Window} subdivides a {@link org.apache.samza.operators.MessageStream} into smaller finite chunks. Programmers + * should use the API methods of {@link Windows} to specify their windowing functions. + * + * There are the following aspects to windowing in Samza: + * + * Default Trigger: Every {@link Window} has + * a default {@link TriggersBuilder.Trigger} that specifies when to emit + * its results. + * + * Early and Late Triggers: Users can choose to emit early, partial results speculatively by configuring an early trigger. Users + * can choose to handle arrival of late data by configuring a late trigger. Refer to the {@link TriggersBuilder} APIs for + * configuring early and late triggers. + * + * Aggregation: Every {@link Window} has an aggregation function to be applied on each element that returns the type of the value + * stored in the window. + * + * Key Function: A {@link Window} can be keyed by a certain key in the input {@link MessageEnvelope}. For example, A common use-case is to perform + * key based aggregations over a time window. When a key function is specified, the triggering behavior is per key per window. + * + * + * @param type of input {@link MessageEnvelope}. + * @param type of key to use for aggregation. + * @param type of key in the window output. + * @param type of value stored in the {@link Window}. + * @param type of the {@link Window} result. + */ + +public interface Window> { + + /** + * Set the triggers for this {@link Window} + * + * @param wndTrigger trigger conditions set by the programmers + * @return the {@link Window} function w/ the trigger {@code wndTrigger} + */ + Window setTriggers(TriggersBuilder.Triggers wndTrigger); +} diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/experimental/WindowDefinition.java b/samza-api/src/main/java/org/apache/samza/operators/windows/experimental/WindowDefinition.java new file mode 100644 index 0000000000..8f9f675ded --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/operators/windows/experimental/WindowDefinition.java @@ -0,0 +1,83 @@ +package org.apache.samza.operators.windows.experimental; + +import org.apache.samza.operators.data.MessageEnvelope; +import org.apache.samza.operators.windows.WindowOutput; + +import java.util.List; +import java.util.function.BiFunction; +import java.util.function.Function; + +/** + * The base class of all types of {@link Window}s. Sub-classes can specify the default triggering semantics + * for the {@link Window}, semantics for emission of early or late results and whether to accumulate or discard + * previous results. + */ + +public class WindowDefinition implements Window, WV, WindowOutput, WV>> { + + /** + * Defines the default triggering semantics for the {@link Window}. + */ + private final List defaultFirings; + + + /** + * Defines the triggering semantics for emission of early or late results. + */ + private List earlyFirings; + private List lateFirings; + + /** + * Defines the fold function that is applied each time a {@link MessageEnvelope} is added to the window. + */ + private final BiFunction aggregator; + + /* + * Defines the function that extracts the event time from a {@link MessageEnvelope} + */ + private final Function eventTimeExtractor; + + /* + * Defines the function that extracts the key from a {@link MessageEnvelope} + */ + private final Function keyExtractor; + + + public WindowDefinition(Function keyExtractor, BiFunction aggregator, Function eventTimeExtractor, List defaultFirings) { + this.aggregator = aggregator; + this.eventTimeExtractor = eventTimeExtractor; + this.keyExtractor = keyExtractor; + this.defaultFirings = defaultFirings; + } + + @Override + public Window, WV, WindowOutput, WV>> setTriggers(TriggersBuilder.Triggers wndTrigger) { + this.earlyFirings = wndTrigger.getEarlyTriggers(); + this.lateFirings = wndTrigger.getLateTriggers(); + return this; + } + + public List getDefaultFirings() { + return defaultFirings; + } + + public List getEarlyFirings() { + return earlyFirings; + } + + public List getLateFirings() { + return lateFirings; + } + + public BiFunction getAggregator() { + return aggregator; + } + + public Function getEventTimeExtractor() { + return eventTimeExtractor; + } + + public Function getKeyExtractor() { + return keyExtractor; + } +} diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/experimental/WindowKey.java b/samza-api/src/main/java/org/apache/samza/operators/windows/experimental/WindowKey.java new file mode 100644 index 0000000000..9408ca24a4 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/operators/windows/experimental/WindowKey.java @@ -0,0 +1,27 @@ + +package org.apache.samza.operators.windows.experimental; + +public class WindowKey { + private final long windowStart; + private final long windowEnd; + private final K key; + + WindowKey(K key, long windowStart, long windowEnd) { + this.windowStart = windowStart; + this.windowEnd = windowEnd; + this.key = key; + } + + public long getWindowStart() { + return windowStart; + } + + public long getWindowEnd() { + return windowEnd; + } + + public K getKey() { + return key; + } +} + diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/experimental/Windows.java b/samza-api/src/main/java/org/apache/samza/operators/windows/experimental/Windows.java new file mode 100644 index 0000000000..098cc018a0 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/operators/windows/experimental/Windows.java @@ -0,0 +1,115 @@ +package org.apache.samza.operators.windows.experimental; + +import org.apache.samza.operators.data.MessageEnvelope; +import org.apache.samza.operators.windows.WindowOutput; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.function.BiFunction; +import java.util.function.Function; + +public class Windows { + + public static Window, WV, WindowOutput, WV>> keyedTumblingWindow(Function keyFn, long interval, BiFunction aggregateFunction) { + + List defaultTriggers = new ArrayList<>(); + + TriggersBuilder.Trigger timeTrigger = new TriggersBuilder.PeriodicTimeTrigger(TriggersBuilder.TriggerType.DEFAULT, interval, + TriggersBuilder.TimeCharacteristic.PROCESSING_TIME); + defaultTriggers.add(timeTrigger); + + return new WindowDefinition(keyFn, aggregateFunction, null, defaultTriggers); + } + + public static Window, Collection, WindowOutput, Collection>> keyedTumblingWindow(Function keyFn, long interval) { + BiFunction, Collection> aggregator = (m, c) -> { + c.add(m); + return c; + }; + return keyedTumblingWindow(keyFn, interval, aggregator); + } + + public static Window, WV, WindowOutput, WV>> tumblingWindow(long interval, BiFunction aggregateFunction) { + + List defaultTriggers = new ArrayList<>(); + + TriggersBuilder.Trigger timeTrigger = new TriggersBuilder.PeriodicTimeTrigger(TriggersBuilder.TriggerType.DEFAULT, interval, + TriggersBuilder.TimeCharacteristic.PROCESSING_TIME); + defaultTriggers.add(timeTrigger); + + return new WindowDefinition(null, aggregateFunction, null, defaultTriggers); + } + + public static Window, Collection, WindowOutput, Collection>> tumblingWindow(long interval) { + BiFunction, Collection> aggregator = (m, c) -> { + c.add(m); + return c; + }; + return tumblingWindow(interval, aggregator); + } + + public static Window, WV, WindowOutput, WV>> sessionWindow(long sessionGap, BiFunction aggregateFunction) { + + List defaultTriggers = new ArrayList<>(); + + TriggersBuilder.Trigger timeTrigger = new TriggersBuilder.TimeSinceLastMessageTrigger(TriggersBuilder.TriggerType.DEFAULT, sessionGap, + TriggersBuilder.TimeCharacteristic.PROCESSING_TIME); + defaultTriggers.add(timeTrigger); + + return new WindowDefinition(null, aggregateFunction, null, defaultTriggers); + } + + public static Window, Collection, WindowOutput, Collection>> sessionWindow(long gap) { + BiFunction, Collection> aggregator = (m, c) -> { + c.add(m); + return c; + }; + return sessionWindow(gap, aggregator); + } + + public static Window, WV, WindowOutput, WV>> keyedSessionWindow(Function keyFn, long sessionGap, BiFunction aggregateFunction) { + List defaultTriggers = new ArrayList<>(); + + TriggersBuilder.Trigger timeTrigger = new TriggersBuilder.TimeSinceLastMessageTrigger(TriggersBuilder.TriggerType.DEFAULT, sessionGap, + TriggersBuilder.TimeCharacteristic.PROCESSING_TIME); + defaultTriggers.add(timeTrigger); + + return new WindowDefinition(keyFn, aggregateFunction, null, defaultTriggers); + } + + public static Window, Collection, WindowOutput, Collection>> keyedSessionWindow(Function keyFn, long sessionGap) { + + BiFunction, Collection> aggregator = (m, c) -> { + c.add(m); + return c; + }; + return keyedSessionWindow(keyFn, sessionGap, aggregator); + } + + + + public static Window, WV, WindowOutput, WV>> globalWindow(BiFunction aggregateFunction) { + return new WindowDefinition(null, aggregateFunction, null, null); + } + + public static Window, Collection, WindowOutput, Collection>> globalWindow() { + BiFunction, Collection> aggregator = (m, c) -> { + c.add(m); + return c; + }; + return globalWindow(aggregator); + } + + public static Window, WV, WindowOutput, WV>> keyedGlobalWindow(Function keyFn, BiFunction aggregateFunction) { + return new WindowDefinition(keyFn, aggregateFunction, null, null); + } + + public static Window, Collection, WindowOutput, Collection>> keyedGlobalWindow(Function keyFn) { + BiFunction, Collection> aggregator = (m, c) -> { + c.add(m); + return c; + }; + return keyedGlobalWindow(keyFn, aggregator); + } +} diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/ExperimentalWindowOperatorSpec.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/ExperimentalWindowOperatorSpec.java new file mode 100644 index 0000000000..1a331fa013 --- /dev/null +++ b/samza-operator/src/main/java/org/apache/samza/operators/spec/ExperimentalWindowOperatorSpec.java @@ -0,0 +1,42 @@ +package org.apache.samza.operators.spec; + +import org.apache.samza.operators.MessageStream; +import org.apache.samza.operators.MessageStreamImpl; +import org.apache.samza.operators.data.MessageEnvelope; +import org.apache.samza.operators.windows.WindowOutput; +import org.apache.samza.operators.windows.experimental.WindowDefinition; + +/** + * + */ +public class ExperimentalWindowOperatorSpec> implements OperatorSpec{ + + private final WindowDefinition window; + + /** + * The output {@link MessageStream}. + */ + private final MessageStreamImpl outputStream; + + private final String operatorId; + + + public ExperimentalWindowOperatorSpec(WindowDefinition window, String operatorId) { + this.window = window; + this.outputStream = new MessageStreamImpl<>(); + this.operatorId = operatorId; + } + + @Override + public MessageStream getOutputStream() { + return this.outputStream; + } + + public WindowDefinition getWindow() { + return window; + } + + public String getOperatorId() { + return operatorId; + } +} diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java new file mode 100644 index 0000000000..2f5b1e76f2 --- /dev/null +++ b/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.samza.operators.spec; + +import org.apache.samza.operators.MessageStream; +import org.apache.samza.operators.data.MessageEnvelope; +import org.apache.samza.operators.MessageStreamImpl; +import org.apache.samza.operators.windows.StoreFunctions; +import org.apache.samza.operators.windows.Trigger; +import org.apache.samza.operators.windows.WindowFn; +import org.apache.samza.operators.windows.WindowOutput; +import org.apache.samza.operators.windows.WindowState; +import org.apache.samza.storage.kv.Entry; + +import java.util.function.BiFunction; + + +/** + * Defines a window operator that takes one {@link MessageStreamImpl} as an input, accumulates the window state, + * and generates an output {@link MessageStreamImpl} with output type {@code WM} which extends {@link WindowOutput} + * + * @param the type of input {@link MessageEnvelope} + * @param the type of key in the output {@link MessageEnvelope} from the {@link WindowOperatorSpec} function + * @param the type of window state in the {@link WindowOperatorSpec} function + * @param the type of window output {@link MessageEnvelope} + */ +public class WindowOperatorSpec> implements + OperatorSpec { + + /** + * The output {@link MessageStream}. + */ + private final MessageStreamImpl outputStream; + + /** + * The window transformation function that takes {@link MessageEnvelope}s from one input stream, aggregates with the window + * state(s) from the window state store, and generate output {@link MessageEnvelope}s for the output stream. + */ + private final BiFunction, WM> transformFn; + + /** + * The state store functions for the {@link WindowOperatorSpec}. + */ + private final StoreFunctions storeFns; + + /** + * The window trigger. + */ + private final Trigger trigger; + + /** + * The unique ID of this operator. + */ + private final String operatorId; + + /** + * Constructor for {@link WindowOperatorSpec}. + * + * @param windowFn the window function + * @param operatorId auto-generated unique ID of this operator + */ + WindowOperatorSpec(WindowFn windowFn, String operatorId) { + this.outputStream = new MessageStreamImpl<>(); + this.transformFn = windowFn.getTransformFn(); + this.storeFns = windowFn.getStoreFns(); + this.trigger = windowFn.getTrigger(); + this.operatorId = operatorId; + } + + @Override + public String toString() { + return this.operatorId; + } + + @Override + public MessageStreamImpl getOutputStream() { + return this.outputStream; + } + + public StoreFunctions getStoreFns() { + return this.storeFns; + } + + public BiFunction, WM> getTransformFn() { + return this.transformFn; + } + + public Trigger getTrigger() { + return this.trigger; + } + + /** + * Method to generate the window operator's state store name + * TODO HIGH pmaheshw: should this be here? + * + * @param inputStream the input {@link MessageStreamImpl} to this state store + * @return the persistent store name of the window operator + */ + public String getStoreName(MessageStream inputStream) { + //TODO: need to get the persistent name of ds and the operator in a serialized form + return String.format("input-%s-wndop-%s", inputStream.toString(), this.toString()); + } +} From b283777f3cd94cd73382db6c5103aba595ab5aea Mon Sep 17 00:00:00 2001 From: vjagadish1989 Date: Tue, 6 Dec 2016 14:35:11 -0800 Subject: [PATCH 3/6] APIs for trigger and windows --- .../apache/samza/operators/MessageStream.java | 16 +- .../operators/windows/SessionWindow.java | 102 ------ .../operators/windows/StoreFunctions.java | 67 ---- .../samza/operators/windows/Trigger.java | 94 ----- .../operators/windows/TriggerBuilder.java | 320 ------------------ .../samza/operators/windows/Window.java | 49 --- .../samza/operators/windows/WindowFn.java | 59 ---- .../samza/operators/windows/Windows.java | 100 ------ .../samza/operators/MessageStreamImpl.java | 2 +- .../samza/operators/impl/OperatorImpls.java | 2 +- .../impl/SessionWindowOperatorImpl.java | 2 +- .../samza/operators/spec/OperatorSpecs.java | 1 - .../spec/PartialJoinOperatorSpec.java | 1 - .../operators/spec/WindowOperatorSpec.java | 2 - .../samza/operators/spec}/WindowState.java | 2 +- .../samza/operators/TestStateStoreImpl.java | 4 +- .../operators/impl/TestSessionWindowImpl.java | 4 +- .../operators/spec/TestOperatorSpecs.java | 2 - 18 files changed, 14 insertions(+), 815 deletions(-) delete mode 100644 samza-api/src/main/java/org/apache/samza/operators/windows/SessionWindow.java delete mode 100644 samza-api/src/main/java/org/apache/samza/operators/windows/StoreFunctions.java delete mode 100644 samza-api/src/main/java/org/apache/samza/operators/windows/Trigger.java delete mode 100644 samza-api/src/main/java/org/apache/samza/operators/windows/TriggerBuilder.java delete mode 100644 samza-api/src/main/java/org/apache/samza/operators/windows/Window.java delete mode 100644 samza-api/src/main/java/org/apache/samza/operators/windows/WindowFn.java delete mode 100644 samza-api/src/main/java/org/apache/samza/operators/windows/Windows.java rename {samza-api/src/main/java/org/apache/samza/operators/windows => samza-operator/src/main/java/org/apache/samza/operators/spec}/WindowState.java (98%) diff --git a/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java b/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java index d1c0fd80a0..18855e12a3 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java +++ b/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java @@ -25,9 +25,7 @@ import org.apache.samza.operators.functions.JoinFunction; import org.apache.samza.operators.functions.MapFunction; import org.apache.samza.operators.functions.SinkFunction; -import org.apache.samza.operators.windows.Window; import org.apache.samza.operators.windows.WindowOutput; -import org.apache.samza.operators.windows.WindowState; import java.util.Collection; @@ -83,21 +81,19 @@ public interface MessageStream { void sink(SinkFunction sinkFn); /** - * Groups the {@link MessageEnvelope}s in this {@link MessageStream} according to the provided {@link Window} semantics + * Groups the {@link MessageEnvelope}s in this {@link MessageStream} according to the provided {@link org.apache.samza.operators.windows.experimental.Window} semantics * (e.g. tumbling, sliding or session windows) and returns the transformed {@link MessageStream} of * {@link WindowOutput}s. *

- * Use the {@link org.apache.samza.operators.windows.Windows} helper methods to create the appropriate windows. + * Use the {@link org.apache.samza.operators.windows.experimental.Windows} helper methods to create the appropriate windows. * - * @param window the {@link Window} to group and process {@link MessageEnvelope}s from this {@link MessageStream} - * @param the type of key in the {@link WindowOutput} from the {@link Window} - * @param the type of value in the {@link WindowOutput} from the {@link Window} - * @param the type of window state kept in the {@link Window} + * @param window the {@link org.apache.samza.operators.windows.experimental.Window} to group and process {@link MessageEnvelope}s from this {@link MessageStream} + * @param the type of key in the {@link WindowOutput} from the {@link org.apache.samza.operators.windows.experimental.Window} + * @param the type of value in the {@link WindowOutput} from the {@link org.apache.samza.operators.windows.experimental.Window} + * @param the type of window state kept in the {@link org.apache.samza.operators.windows.experimental.Window} * @param the type of {@link WindowOutput} in the transformed {@link MessageStream} * @return the transformed {@link MessageStream} */ - , WM extends WindowOutput> MessageStream window( - Window window); > MessageStream window( org.apache.samza.operators.windows.experimental.Window window); diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/SessionWindow.java b/samza-api/src/main/java/org/apache/samza/operators/windows/SessionWindow.java deleted file mode 100644 index 287025c854..0000000000 --- a/samza-api/src/main/java/org/apache/samza/operators/windows/SessionWindow.java +++ /dev/null @@ -1,102 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.samza.operators.windows; - -import org.apache.samza.operators.data.MessageEnvelope; -import org.apache.samza.storage.kv.Entry; - -import java.util.function.BiFunction; -import java.util.function.Function; - -/** - * This class defines a session window function class - * - * @param the type of input {@link MessageEnvelope} - * @param the type of session key in the session window - * @param the type of output value in each session window - */ -public class SessionWindow implements Window> { - - /** - * Constructor. Made private s.t. it can only be instantiated via the static API methods in {@link Windows} - * - * @param sessionKeyFunction function to get the session key from the input {@link MessageEnvelope} - * @param aggregator function to calculate the output value based on the input {@link MessageEnvelope} and current output value - */ - SessionWindow(Function sessionKeyFunction, BiFunction aggregator) { - this.wndKeyFunction = sessionKeyFunction; - this.aggregator = aggregator; - } - - /** - * function to calculate the window key from input {@link MessageEnvelope} - */ - private final Function wndKeyFunction; - - /** - * function to calculate the output value from the input {@link MessageEnvelope} and the current output value - */ - private final BiFunction aggregator; - - /** - * trigger condition that determines when to send the {@link WindowOutput} - */ - private Trigger> trigger = null; - - //TODO: need to create a set of {@link StoreFunctions} that is default to input {@link MessageEnvelope} type for {@link Window} - private StoreFunctions> storeFunctions = null; - - /** - * Public API methods start here - */ - - /** - * Public API method to define the watermark trigger for the window operator - * - * @param wndTrigger {@link Trigger} function defines the watermark trigger for this {@link SessionWindow} - * @return The window operator w/ the defined watermark trigger - */ - @Override - public Window> setTriggers(TriggerBuilder wndTrigger) { - this.trigger = wndTrigger.build(); - return this; - } - - private BiFunction>, WindowOutput> getTransformFunc() { - // TODO: actual implementation of the main session window logic, based on the wndKeyFunction, aggregator, and triggers; - return null; - } - - public WindowFn, WindowOutput> getInternalWindowFn() { - return new WindowFn, WindowOutput>() { - - @Override public BiFunction>, WindowOutput> getTransformFn() { - return SessionWindow.this.getTransformFunc(); - } - - @Override public StoreFunctions> getStoreFns() { - return SessionWindow.this.storeFunctions; - } - - @Override public Trigger> getTrigger() { - return SessionWindow.this.trigger; - } - }; - } -} diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/StoreFunctions.java b/samza-api/src/main/java/org/apache/samza/operators/windows/StoreFunctions.java deleted file mode 100644 index 0d407617c5..0000000000 --- a/samza-api/src/main/java/org/apache/samza/operators/windows/StoreFunctions.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.samza.operators.windows; - -import org.apache.samza.operators.data.MessageEnvelope; - -import java.util.function.BiFunction; -import java.util.function.Function; - -/** - * The store functions that are used by window and partial join operators to store and retrieve buffered {@link MessageEnvelope}s - * and partial aggregation results. - * - * @param the type of key used to store the operator state - * @param the type of operator state. E.g. could be the partial aggregation result for a window, or a buffered - * input {@link MessageEnvelope} from the join stream for a join - */ -public class StoreFunctions { - /** - * Function that returns the key to query in the operator state store for a particular {@link MessageEnvelope}. - * This 1:1 function only returns a single key for the incoming {@link MessageEnvelope}. This is sufficient to support - * non-overlapping windows and unique-key based joins. - * - * TODO: for windows that overlaps (i.e. sliding windows and hopping windows) and non-unique-key-based join, - * the query to the state store is usually a range scan. We need to add a rangeKeyFinder function - * (or make this function return a collection) to map from a single input {@link MessageEnvelope} to a range of keys in the store. - */ - private final Function storeKeyFn; - - /** - * Function to update the store entry based on the current operator state and the incoming {@link MessageEnvelope}. - * - * TODO: this is assuming a 1:1 mapping from the input {@link MessageEnvelope} to the store entry. When implementing sliding/hopping - * windows and non-unique-key-based join, we may need to include the corresponding state key in addition to the - * state value. Alternatively this can be called once for each store key for the {@link MessageEnvelope}. - */ - private final BiFunction stateUpdaterFn; - - public StoreFunctions(Function storeKeyFn, BiFunction stateUpdaterFn) { - this.storeKeyFn = storeKeyFn; - this.stateUpdaterFn = stateUpdaterFn; - } - - public Function getStoreKeyFn() { - return this.storeKeyFn; - } - - public BiFunction getStateUpdaterFn() { - return this.stateUpdaterFn; - } -} diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/Trigger.java b/samza-api/src/main/java/org/apache/samza/operators/windows/Trigger.java deleted file mode 100644 index c8b0edb2af..0000000000 --- a/samza-api/src/main/java/org/apache/samza/operators/windows/Trigger.java +++ /dev/null @@ -1,94 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.samza.operators.windows; - -import org.apache.samza.operators.data.MessageEnvelope; - -import java.util.function.BiFunction; -import java.util.function.Function; - -/** - * Defines the trigger functions for the window operator. This class is immutable. - * - * @param the type of {@link MessageEnvelope} in the input stream - * @param the type of state variable in the window's state store - */ -public class Trigger { - - /** - * System timer based trigger condition. This is the only guarantee that the window operator will proceed forward - */ - private final Function timerTrigger; - - /** - * early trigger condition that determines when to send the first output from the window operator - */ - private final BiFunction earlyTrigger; - - /** - * late trigger condition that determines when to send the updated output after the first one from a window operator - */ - private final BiFunction lateTrigger; - - /** - * the function to updated the window state when the first output is triggered - */ - private final Function earlyTriggerUpdater; - - /** - * the function to updated the window state when the late output is triggered - */ - private final Function lateTriggerUpdater; - - /** - * Private constructor to prevent instantiation - * - * @param timerTrigger system timer trigger condition - * @param earlyTrigger early trigger condition - * @param lateTrigger late trigger condition - * @param earlyTriggerUpdater early trigger state updater - * @param lateTriggerUpdater late trigger state updater - */ - private Trigger(Function timerTrigger, BiFunction earlyTrigger, BiFunction lateTrigger, - Function earlyTriggerUpdater, Function lateTriggerUpdater) { - this.timerTrigger = timerTrigger; - this.earlyTrigger = earlyTrigger; - this.lateTrigger = lateTrigger; - this.earlyTriggerUpdater = earlyTriggerUpdater; - this.lateTriggerUpdater = lateTriggerUpdater; - } - - /** - * Static method to create a {@link Trigger} object - * - * @param timerTrigger system timer trigger condition - * @param earlyTrigger early trigger condition - * @param lateTrigger late trigger condition - * @param earlyTriggerUpdater early trigger state updater - * @param lateTriggerUpdater late trigger state updater - * @param the type of input {@link MessageEnvelope} - * @param the type of window state extends {@link WindowState} - * @return the {@link Trigger} function - */ - public static Trigger createTrigger(Function timerTrigger, - BiFunction earlyTrigger, BiFunction lateTrigger, Function earlyTriggerUpdater, - Function lateTriggerUpdater) { - return new Trigger(timerTrigger, earlyTrigger, lateTrigger, earlyTriggerUpdater, lateTriggerUpdater); - } -} diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/TriggerBuilder.java b/samza-api/src/main/java/org/apache/samza/operators/windows/TriggerBuilder.java deleted file mode 100644 index 6336a50602..0000000000 --- a/samza-api/src/main/java/org/apache/samza/operators/windows/TriggerBuilder.java +++ /dev/null @@ -1,320 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.samza.operators.windows; - - -import org.apache.samza.annotation.InterfaceStability; -import org.apache.samza.operators.data.MessageEnvelope; - -import java.util.concurrent.TimeUnit; -import java.util.function.BiFunction; -import java.util.function.Function; - - -/** - * This class defines a builder of {@link Trigger} object for a {@link Window}. The triggers are categorized into - * three types: - * - *

- * early trigger: defines the condition when the first output from the window function is sent. - * late trigger: defines the condition when the updated output after the first output is sent. - * timer trigger: defines a system timeout condition to trigger output if no more inputs are received to enable early/late triggers - *

- * - * If multiple conditions are defined for a specific type of trigger, the aggregated trigger is the disjunction - * of each individual trigger (i.e. OR). - * - * @param the type of input {@link MessageEnvelope} to the {@link Window} - * @param the type of output value from the {@link Window} - */ -@InterfaceStability.Unstable -public final class TriggerBuilder { - - /** - * Predicate helper to OR multiple trigger conditions - */ - static class PredicateHelper { - static BiFunction or(BiFunction lhs, BiFunction rhs) { - return (m, s) -> lhs.apply(m, s) || rhs.apply(m, s); - } - - static Function or(Function lhs, Function rhs) { - return s -> lhs.apply(s) || rhs.apply(s); - } - } - - /** - * The early trigger condition that determines the first output from the {@link Window} - */ - private BiFunction, Boolean> earlyTrigger = null; - - /** - * The late trigger condition that determines the late output(s) from the {@link Window} - */ - private BiFunction, Boolean> lateTrigger = null; - - /** - * The system timer based trigger conditions that guarantees the {@link Window} proceeds forward - */ - private Function, Boolean> timerTrigger = null; - - /** - * The state updater function to be applied after the first output is triggered - */ - private Function, WindowState> earlyTriggerUpdater = Function.identity(); - - /** - * The state updater function to be applied after the late output is triggered - */ - private Function, WindowState> lateTriggerUpdater = Function.identity(); - - /** - * Helper method to add a trigger condition - * - * @param currentTrigger current trigger condition - * @param newTrigger new trigger condition - * @return combined trigger condition that is {@code currentTrigger} OR {@code newTrigger} - */ - private BiFunction, Boolean> addTrigger(BiFunction, Boolean> currentTrigger, - BiFunction, Boolean> newTrigger) { - if (currentTrigger == null) { - return newTrigger; - } - - return PredicateHelper.or(currentTrigger, newTrigger); - } - - /** - * Helper method to add a system timer trigger - * - * @param currentTimer current timer condition - * @param newTimer new timer condition - * @return combined timer condition that is {@code currentTimer} OR {@code newTimer} - */ - private Function, Boolean> addTimerTrigger(Function, Boolean> currentTimer, - Function, Boolean> newTimer) { - if (currentTimer == null) { - return newTimer; - } - - return PredicateHelper.or(currentTimer, newTimer); - } - - /** - * default constructor to prevent instantiation - */ - private TriggerBuilder() {} - - /** - * Constructor that set the size limit as the early trigger for a window - * - * @param sizeLimit the number of {@link MessageEnvelope}s in a window that would trigger the first output - */ - private TriggerBuilder(long sizeLimit) { - this.earlyTrigger = (m, s) -> s.getNumberMessages() > sizeLimit; - } - - /** - * Constructor that set the event time length as the early trigger - * - * @param eventTimeFunction the function that calculate the event time in nano-second from the input {@link MessageEnvelope} - * @param wndLenMs the window length in event time in milli-second - */ - private TriggerBuilder(Function eventTimeFunction, long wndLenMs) { - this.earlyTrigger = (m, s) -> - TimeUnit.NANOSECONDS.toMillis(Math.max(s.getLatestEventTimeNs() - s.getEarliestEventTimeNs(), - eventTimeFunction.apply(m) - s.getEarliestEventTimeNs())) > wndLenMs; - } - - /** - * Constructor that set the special token {@link MessageEnvelope} as the early trigger - * - * @param tokenFunc the function that checks whether an input {@link MessageEnvelope} is a token {@link MessageEnvelope} that triggers window output - */ - private TriggerBuilder(Function tokenFunc) { - this.earlyTrigger = (m, s) -> tokenFunc.apply(m); - } - - /** - * Build method that creates an {@link Trigger} object based on the trigger conditions set in {@link TriggerBuilder} - * This is kept package private and only used by {@link Windows} to convert the mutable {@link TriggerBuilder} object to an immutable {@link Trigger} object - * - * @return the final {@link Trigger} object - */ - Trigger> build() { - return Trigger.createTrigger(this.timerTrigger, this.earlyTrigger, this.lateTrigger, this.earlyTriggerUpdater, this.lateTriggerUpdater); - } - - /** - * Public API methods start here - */ - - - /** - * API method to allow users to set an update method to update the output value after the first window output is triggered - * by the early trigger condition - * - * @param onTriggerFunc the method to update the output value after the early trigger - * @return the {@link TriggerBuilder} object - */ - public TriggerBuilder onEarlyTrigger(Function onTriggerFunc) { - this.earlyTriggerUpdater = s -> { - s.setOutputValue(onTriggerFunc.apply(s.getOutputValue())); - return s; - }; - return this; - } - - /** - * API method to allow users to set an update method to update the output value after a late window output is triggered - * by the late trigger condition - * - * @param onTriggerFunc the method to update the output value after the late trigger - * @return the {@link TriggerBuilder} object - */ - public TriggerBuilder onLateTrigger(Function onTriggerFunc) { - this.lateTriggerUpdater = s -> { - s.setOutputValue(onTriggerFunc.apply(s.getOutputValue())); - return s; - }; - return this; - } - - /** - * API method to allow users to add a system timer trigger based on timeout after the last {@link MessageEnvelope} received in the window - * - * @param timeoutMs the timeout in ms after the last {@link MessageEnvelope} received in the window - * @return the {@link TriggerBuilder} object - */ - public TriggerBuilder addTimeoutSinceLastMessage(long timeoutMs) { - this.timerTrigger = this.addTimerTrigger(this.timerTrigger, - s -> TimeUnit.NANOSECONDS.toMillis(s.getLastMessageTimeNs()) + timeoutMs < System.currentTimeMillis()); - return this; - } - - /** - * API method to allow users to add a system timer trigger based on the timeout after the first {@link MessageEnvelope} received in the window - * - * @param timeoutMs the timeout in ms after the first {@link MessageEnvelope} received in the window - * @return the {@link TriggerBuilder} object - */ - public TriggerBuilder addTimeoutSinceFirstMessage(long timeoutMs) { - this.timerTrigger = this.addTimerTrigger(this.timerTrigger, s -> - TimeUnit.NANOSECONDS.toMillis(s.getFirstMessageTimeNs()) + timeoutMs < System.currentTimeMillis()); - return this; - } - - /** - * API method allow users to add a late trigger based on the window size limit - * - * @param sizeLimit limit on the number of {@link MessageEnvelope}s in window - * @return the {@link TriggerBuilder} object - */ - public TriggerBuilder addLateTriggerOnSizeLimit(long sizeLimit) { - this.lateTrigger = this.addTrigger(this.lateTrigger, (m, s) -> s.getNumberMessages() > sizeLimit); - return this; - } - - /** - * API method to allow users to define a customized late trigger function based on input {@link MessageEnvelope} and the window state - * - * @param lateTrigger the late trigger condition based on input {@link MessageEnvelope} and the current {@link WindowState} - * @return the {@link TriggerBuilder} object - */ - public TriggerBuilder addLateTrigger(BiFunction, Boolean> lateTrigger) { - this.lateTrigger = this.addTrigger(this.lateTrigger, lateTrigger); - return this; - } - - /** - * Static API method to create a {@link TriggerBuilder} w/ early trigger condition based on window size limit - * - * @param sizeLimit window size limit - * @param the type of input {@link MessageEnvelope} - * @param the type of {@link Window} output value - * @return the {@link TriggerBuilder} object - */ - public static TriggerBuilder earlyTriggerWhenExceedWndLen(long sizeLimit) { - return new TriggerBuilder(sizeLimit); - } - - /** - * Static API method to create a {@link TriggerBuilder} w/ early trigger condition based on event time window - * - * - * @param eventTimeFunc the function to get the event time from the input {@link MessageEnvelope} - * @param eventTimeWndSizeMs the event time window size in Ms - * @param the type of input {@link MessageEnvelope} - * @param the type of {@link Window} output value - * @return the {@link TriggerBuilder} object - */ - public static TriggerBuilder earlyTriggerOnEventTime(Function eventTimeFunc, long eventTimeWndSizeMs) { - return new TriggerBuilder(eventTimeFunc, eventTimeWndSizeMs); - } - - /** - * Static API method to create a {@link TriggerBuilder} w/ early trigger condition based on token {@link MessageEnvelope}s - * - * @param tokenFunc the function to determine whether an input {@link MessageEnvelope} is a window token or not - * @param the type of input {@link MessageEnvelope} - * @param the type of {@link Window} output value - * @return the {@link TriggerBuilder} object - */ - public static TriggerBuilder earlyTriggerOnTokenMsg(Function tokenFunc) { - return new TriggerBuilder(tokenFunc); - } - - /** - * Static API method to allow customized early trigger condition based on input {@link MessageEnvelope} and the corresponding {@link WindowState} - * - * @param earlyTrigger the user defined early trigger condition - * @param the input {@link MessageEnvelope} type - * @param the output value from the window - * @return the {@link TriggerBuilder} object - */ - public static TriggerBuilder earlyTrigger(BiFunction, Boolean> earlyTrigger) { - TriggerBuilder newTriggers = new TriggerBuilder(); - newTriggers.earlyTrigger = newTriggers.addTrigger(newTriggers.earlyTrigger, earlyTrigger); - return newTriggers; - } - - /** - * Static API method to create a {@link TriggerBuilder} w/ system timeout after the last {@link MessageEnvelope} received in the window - * - * @param timeoutMs timeout in ms after the last {@link MessageEnvelope} received - * @param the type of input {@link MessageEnvelope} - * @param the type of {@link Window} output value - * @return the {@link TriggerBuilder} object - */ - public static TriggerBuilder timeoutSinceLastMessage(long timeoutMs) { - return new TriggerBuilder().addTimeoutSinceLastMessage(timeoutMs); - } - - /** - * Static API method to create a {@link TriggerBuilder} w/ system timeout after the first {@link MessageEnvelope} received in the window - * - * @param timeoutMs timeout in ms after the first {@link MessageEnvelope} received - * @param the type of input {@link MessageEnvelope} - * @param the type of {@link Window} output value - * @return the {@link TriggerBuilder} object - */ - public static TriggerBuilder timeoutSinceFirstMessage(long timeoutMs) { - return new TriggerBuilder().addTimeoutSinceFirstMessage(timeoutMs); - } -} diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/Window.java b/samza-api/src/main/java/org/apache/samza/operators/windows/Window.java deleted file mode 100644 index 56a307de39..0000000000 --- a/samza-api/src/main/java/org/apache/samza/operators/windows/Window.java +++ /dev/null @@ -1,49 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.samza.operators.windows; - -import org.apache.samza.operators.data.MessageEnvelope; - -/** - * The public programming interface class for window function - * - * @param the type of input {@link MessageEnvelope} - * @param the type of key to the {@link Window} - * @param the type of output value in the {@link WindowOutput} - * @param the type of {@link MessageEnvelope} in the window output stream - */ -public interface Window> { - - /** - * Set the triggers for this {@link Window} - * - * @param wndTrigger trigger conditions set by the programmers - * @return the {@link Window} function w/ the trigger {@code wndTrigger} - */ - Window setTriggers(TriggerBuilder wndTrigger); - - /** - * Internal implementation helper to get the functions associated with this Window. - * - * NOTE: This is purely an internal API and should not be used directly by users. - * - * @return the functions associated with this Window. - */ - WindowFn, WindowOutput> getInternalWindowFn(); -} diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/WindowFn.java b/samza-api/src/main/java/org/apache/samza/operators/windows/WindowFn.java deleted file mode 100644 index 8878bf93d3..0000000000 --- a/samza-api/src/main/java/org/apache/samza/operators/windows/WindowFn.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.samza.operators.windows; - -import org.apache.samza.operators.data.MessageEnvelope; -import org.apache.samza.storage.kv.Entry; - -import java.util.function.BiFunction; - - -/** - * Defines an internal representation of a window function. - * - * @param type of the input {@link MessageEnvelope} for the window - * @param type of the window key in the output {@link MessageEnvelope} - * @param type of the {@link WindowState} in the state store - * @param type of the {@link MessageEnvelope} in the output stream - */ -public interface WindowFn> { - - /** - * Get the transformation function of the {@link WindowFn}. - * - * @return the transformation function which takes a {@link MessageEnvelope} of type {@code M} and its window state entry, - * and transforms it to an {@link WindowOutput} - */ - BiFunction, WM> getTransformFn(); - - /** - * Get the state store functions for this {@link WindowFn}. - * - * @return the state store functions - */ - StoreFunctions getStoreFns(); - - /** - * Get the trigger conditions for this {@link WindowFn}. - * - * @return the trigger condition for this {@link WindowFn} - */ - Trigger getTrigger(); - -} diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/Windows.java b/samza-api/src/main/java/org/apache/samza/operators/windows/Windows.java deleted file mode 100644 index 1a4ed8f49f..0000000000 --- a/samza-api/src/main/java/org/apache/samza/operators/windows/Windows.java +++ /dev/null @@ -1,100 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.samza.operators.windows; - -import org.apache.samza.operators.data.MessageEnvelope; - -import java.util.Collection; -import java.util.function.Function; - - -/** - * This class defines a collection of {@link Window} functions. The public classes and methods here are intended to be - * used by the user (i.e. programmers) to create {@link Window} function directly. - * - */ -public final class Windows { - - /** - * private constructor to prevent instantiation - */ - private Windows() {} - - static , WM extends WindowOutput> WindowFn getInternalWindowFn( - Window window) { - if (window instanceof SessionWindow) { - SessionWindow sessionWindow = (SessionWindow) window; - return (WindowFn) sessionWindow.getInternalWindowFn(); - } - throw new IllegalArgumentException("Input window type not supported."); - } - - /** - * Public static API methods start here - * - */ - - /** - * Static API method to create a {@link SessionWindow} in which the output value is simply the collection of input {@link MessageEnvelope}s - * - * @param sessionKeyFunction function to calculate session window key - * @param type of input {@link MessageEnvelope} - * @param type of the session window key - * @return the {@link Window} function for the session - */ - public static Window, WindowOutput>> intoSessions(Function sessionKeyFunction) { - return new SessionWindow<>(sessionKeyFunction, (m, c) -> { - c.add(m); - return c; - } - ); - } - - /** - * Static API method to create a {@link SessionWindow} in which the output value is a collection of {@code SI} from the input {@link MessageEnvelope}s - * - * @param sessionKeyFunction function to calculate session window key - * @param sessionInfoExtractor function to retrieve session info of type {@code SI} from the input {@link MessageEnvelope} of type {@code M} - * @param type of the input {@link MessageEnvelope} - * @param type of the session window key - * @param type of the session information retrieved from each input {@link MessageEnvelope} of type {@code M} - * @return the {@link Window} function for the session - */ - public static Window, WindowOutput>> intoSessions(Function sessionKeyFunction, - Function sessionInfoExtractor) { - return new SessionWindow<>(sessionKeyFunction, (m, c) -> { - c.add(sessionInfoExtractor.apply(m)); - return c; - } - ); - } - - /** - * Static API method to create a {@link SessionWindow} as a counter of input {@link MessageEnvelope}s - * - * @param sessionKeyFunction function to calculate session window key - * @param type of the input {@link MessageEnvelope} - * @param type of the session window key - * @return the {@link Window} function for the session - */ - public static Window> intoSessionCounter(Function sessionKeyFunction) { - return new SessionWindow<>(sessionKeyFunction, (m, c) -> c + 1); - } - -} diff --git a/samza-operator/src/main/java/org/apache/samza/operators/MessageStreamImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/MessageStreamImpl.java index 231d3f5fd9..1e8e606f64 100644 --- a/samza-operator/src/main/java/org/apache/samza/operators/MessageStreamImpl.java +++ b/samza-operator/src/main/java/org/apache/samza/operators/MessageStreamImpl.java @@ -30,7 +30,7 @@ import org.apache.samza.operators.windows.Window; import org.apache.samza.operators.windows.WindowFn; import org.apache.samza.operators.windows.WindowOutput; -import org.apache.samza.operators.windows.WindowState; +import org.apache.samza.operators.spec.WindowState; import java.util.ArrayList; import java.util.Collection; diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpls.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpls.java index 79446be310..8d774d76cf 100644 --- a/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpls.java +++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpls.java @@ -28,7 +28,7 @@ import org.apache.samza.operators.spec.StreamOperatorSpec; import org.apache.samza.operators.spec.WindowOperatorSpec; import org.apache.samza.operators.windows.WindowOutput; -import org.apache.samza.operators.windows.WindowState; +import org.apache.samza.operators.spec.WindowState; import org.apache.samza.task.TaskContext; import java.util.Collection; diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/SessionWindowOperatorImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/SessionWindowOperatorImpl.java index e8a635c60f..8e4691f882 100644 --- a/samza-operator/src/main/java/org/apache/samza/operators/impl/SessionWindowOperatorImpl.java +++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/SessionWindowOperatorImpl.java @@ -22,7 +22,7 @@ import org.apache.samza.operators.StateStoreImpl; import org.apache.samza.operators.data.MessageEnvelope; import org.apache.samza.operators.spec.WindowOperatorSpec; -import org.apache.samza.operators.windows.WindowState; +import org.apache.samza.operators.spec.WindowState; import org.apache.samza.operators.windows.WindowOutput; import org.apache.samza.storage.kv.Entry; import org.apache.samza.task.MessageCollector; diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java index 14fc2db45a..04282a44a1 100644 --- a/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java +++ b/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java @@ -23,7 +23,6 @@ import org.apache.samza.operators.functions.FlatMapFunction; import org.apache.samza.operators.functions.SinkFunction; import org.apache.samza.operators.MessageStreamImpl; -import org.apache.samza.operators.windows.WindowState; import org.apache.samza.operators.windows.WindowFn; import org.apache.samza.operators.windows.WindowOutput; import org.apache.samza.operators.windows.experimental.WindowDefinition; diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/PartialJoinOperatorSpec.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/PartialJoinOperatorSpec.java index f74f35d584..7e8dbc158c 100644 --- a/samza-operator/src/main/java/org/apache/samza/operators/spec/PartialJoinOperatorSpec.java +++ b/samza-operator/src/main/java/org/apache/samza/operators/spec/PartialJoinOperatorSpec.java @@ -20,7 +20,6 @@ import org.apache.samza.operators.data.MessageEnvelope; import org.apache.samza.operators.MessageStreamImpl; -import org.apache.samza.operators.windows.StoreFunctions; import java.util.function.BiFunction; diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java index 2f5b1e76f2..30d2f52cda 100644 --- a/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java +++ b/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java @@ -21,11 +21,9 @@ import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.data.MessageEnvelope; import org.apache.samza.operators.MessageStreamImpl; -import org.apache.samza.operators.windows.StoreFunctions; import org.apache.samza.operators.windows.Trigger; import org.apache.samza.operators.windows.WindowFn; import org.apache.samza.operators.windows.WindowOutput; -import org.apache.samza.operators.windows.WindowState; import org.apache.samza.storage.kv.Entry; import java.util.function.BiFunction; diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/WindowState.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowState.java similarity index 98% rename from samza-api/src/main/java/org/apache/samza/operators/windows/WindowState.java rename to samza-operator/src/main/java/org/apache/samza/operators/spec/WindowState.java index 835d749907..e9af043f36 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/windows/WindowState.java +++ b/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowState.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.samza.operators.windows; +package org.apache.samza.operators.spec; import org.apache.samza.annotation.InterfaceStability; diff --git a/samza-operator/src/test/java/org/apache/samza/operators/TestStateStoreImpl.java b/samza-operator/src/test/java/org/apache/samza/operators/TestStateStoreImpl.java index 8fa7ccc4f0..3137a13458 100644 --- a/samza-operator/src/test/java/org/apache/samza/operators/TestStateStoreImpl.java +++ b/samza-operator/src/test/java/org/apache/samza/operators/TestStateStoreImpl.java @@ -18,8 +18,8 @@ */ package org.apache.samza.operators; -import org.apache.samza.operators.windows.StoreFunctions; -import org.apache.samza.operators.windows.WindowState; +import org.apache.samza.operators.spec.StoreFunctions; +import org.apache.samza.operators.spec.WindowState; import org.apache.samza.storage.kv.Entry; import org.apache.samza.storage.kv.KeyValueStore; import org.apache.samza.task.TaskContext; diff --git a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSessionWindowImpl.java b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSessionWindowImpl.java index c302df7f4c..9e45c1f960 100644 --- a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSessionWindowImpl.java +++ b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSessionWindowImpl.java @@ -21,9 +21,9 @@ import org.apache.samza.operators.MessageStreamImpl; import org.apache.samza.operators.TestMessageEnvelope; import org.apache.samza.operators.spec.WindowOperatorSpec; -import org.apache.samza.operators.windows.StoreFunctions; +import org.apache.samza.operators.spec.StoreFunctions; import org.apache.samza.operators.windows.WindowOutput; -import org.apache.samza.operators.windows.WindowState; +import org.apache.samza.operators.spec.WindowState; import org.apache.samza.storage.kv.Entry; import org.apache.samza.storage.kv.KeyValueStore; import org.apache.samza.task.MessageCollector; diff --git a/samza-operator/src/test/java/org/apache/samza/operators/spec/TestOperatorSpecs.java b/samza-operator/src/test/java/org/apache/samza/operators/spec/TestOperatorSpecs.java index 028bd6715e..f7449efd55 100644 --- a/samza-operator/src/test/java/org/apache/samza/operators/spec/TestOperatorSpecs.java +++ b/samza-operator/src/test/java/org/apache/samza/operators/spec/TestOperatorSpecs.java @@ -23,11 +23,9 @@ import org.apache.samza.operators.functions.FlatMapFunction; import org.apache.samza.operators.functions.SinkFunction; import org.apache.samza.operators.MessageStreamImpl; -import org.apache.samza.operators.windows.StoreFunctions; import org.apache.samza.operators.windows.Trigger; import org.apache.samza.operators.windows.WindowFn; import org.apache.samza.operators.windows.WindowOutput; -import org.apache.samza.operators.windows.WindowState; import org.apache.samza.storage.kv.Entry; import org.junit.Test; From 61e027152f13055ae9d206fce89bbe984cd85706 Mon Sep 17 00:00:00 2001 From: vjagadish1989 Date: Tue, 6 Dec 2016 14:35:34 -0800 Subject: [PATCH 4/6] APIs for trigger and windows --- .../samza/operators/spec/StoreFunctions.java | 67 +++++++++++++++++++ 1 file changed, 67 insertions(+) create mode 100644 samza-operator/src/main/java/org/apache/samza/operators/spec/StoreFunctions.java diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/StoreFunctions.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/StoreFunctions.java new file mode 100644 index 0000000000..b8dc516a91 --- /dev/null +++ b/samza-operator/src/main/java/org/apache/samza/operators/spec/StoreFunctions.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.samza.operators.spec; + +import org.apache.samza.operators.data.MessageEnvelope; + +import java.util.function.BiFunction; +import java.util.function.Function; + +/** + * The store functions that are used by window and partial join operators to store and retrieve buffered {@link MessageEnvelope}s + * and partial aggregation results. + * + * @param the type of key used to store the operator state + * @param the type of operator state. E.g. could be the partial aggregation result for a window, or a buffered + * input {@link MessageEnvelope} from the join stream for a join + */ +public class StoreFunctions { + /** + * Function that returns the key to query in the operator state store for a particular {@link MessageEnvelope}. + * This 1:1 function only returns a single key for the incoming {@link MessageEnvelope}. This is sufficient to support + * non-overlapping windows and unique-key based joins. + * + * TODO: for windows that overlaps (i.e. sliding windows and hopping windows) and non-unique-key-based join, + * the query to the state store is usually a range scan. We need to add a rangeKeyFinder function + * (or make this function return a collection) to map from a single input {@link MessageEnvelope} to a range of keys in the store. + */ + private final Function storeKeyFn; + + /** + * Function to update the store entry based on the current operator state and the incoming {@link MessageEnvelope}. + * + * TODO: this is assuming a 1:1 mapping from the input {@link MessageEnvelope} to the store entry. When implementing sliding/hopping + * windows and non-unique-key-based join, we may need to include the corresponding state key in addition to the + * state value. Alternatively this can be called once for each store key for the {@link MessageEnvelope}. + */ + private final BiFunction stateUpdaterFn; + + public StoreFunctions(Function storeKeyFn, BiFunction stateUpdaterFn) { + this.storeKeyFn = storeKeyFn; + this.stateUpdaterFn = stateUpdaterFn; + } + + public Function getStoreKeyFn() { + return this.storeKeyFn; + } + + public BiFunction getStateUpdaterFn() { + return this.stateUpdaterFn; + } +} From 0416f2a89abda4f2447f8effcf5f5d21ef2558ea Mon Sep 17 00:00:00 2001 From: vjagadish1989 Date: Wed, 7 Dec 2016 17:10:46 -0800 Subject: [PATCH 5/6] new APIs for time specification, javadocs cleanup --- .../apache/samza/operators/MessageStream.java | 17 +- .../samza/operators/windows/BaseWindow.java | 100 ++++ .../apache/samza/operators/windows/Time.java | 121 +++++ .../{experimental => }/TriggersBuilder.java | 127 ++++-- .../samza/operators/windows/Window.java | 60 +++ .../samza/operators/windows/WindowKey.java | 63 +++ .../samza/operators/windows/Windows.java | 426 ++++++++++++++++++ .../windows/examples/WindowingExamples.java | 38 +- .../windows/experimental/Window.java | 43 -- .../experimental/WindowDefinition.java | 83 ---- .../windows/experimental/WindowKey.java | 27 -- .../windows/experimental/Windows.java | 115 ----- .../samza/operators/windows/TestTrigger.java | 68 --- .../operators/windows/TestTriggerBuilder.java | 226 ---------- .../samza/operators/windows/TestWindows.java | 109 ----- .../samza/operators/MessageStreamImpl.java | 12 +- .../samza/operators/impl/OperatorImpls.java | 11 +- .../impl/SessionWindowOperatorImpl.java | 67 --- .../operators/impl/WindowOperatorImpl.java | 44 ++ .../spec/ExperimentalWindowOperatorSpec.java | 42 -- .../samza/operators/spec/OperatorSpecs.java | 26 +- .../spec/PartialJoinOperatorSpec.java | 23 - .../samza/operators/spec/StoreFunctions.java | 67 --- .../operators/spec/WindowOperatorSpec.java | 84 +--- .../apache/samza/operators/BroadcastTask.java | 42 +- .../operators/TestMessageStreamImpl.java | 8 +- .../samza/operators/TestStateStoreImpl.java | 72 --- .../apache/samza/operators/WindowTask.java | 14 +- .../operators/impl/TestOperatorImpls.java | 11 +- .../operators/impl/TestSessionWindowImpl.java | 111 ----- .../operators/spec/TestOperatorSpecs.java | 13 +- 31 files changed, 995 insertions(+), 1275 deletions(-) create mode 100644 samza-api/src/main/java/org/apache/samza/operators/windows/BaseWindow.java create mode 100644 samza-api/src/main/java/org/apache/samza/operators/windows/Time.java rename samza-api/src/main/java/org/apache/samza/operators/windows/{experimental => }/TriggersBuilder.java (57%) create mode 100644 samza-api/src/main/java/org/apache/samza/operators/windows/Window.java create mode 100644 samza-api/src/main/java/org/apache/samza/operators/windows/WindowKey.java create mode 100644 samza-api/src/main/java/org/apache/samza/operators/windows/Windows.java delete mode 100644 samza-api/src/main/java/org/apache/samza/operators/windows/experimental/Window.java delete mode 100644 samza-api/src/main/java/org/apache/samza/operators/windows/experimental/WindowDefinition.java delete mode 100644 samza-api/src/main/java/org/apache/samza/operators/windows/experimental/WindowKey.java delete mode 100644 samza-api/src/main/java/org/apache/samza/operators/windows/experimental/Windows.java delete mode 100644 samza-api/src/test/java/org/apache/samza/operators/windows/TestTrigger.java delete mode 100644 samza-api/src/test/java/org/apache/samza/operators/windows/TestTriggerBuilder.java delete mode 100644 samza-api/src/test/java/org/apache/samza/operators/windows/TestWindows.java delete mode 100644 samza-operator/src/main/java/org/apache/samza/operators/impl/SessionWindowOperatorImpl.java create mode 100644 samza-operator/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java delete mode 100644 samza-operator/src/main/java/org/apache/samza/operators/spec/ExperimentalWindowOperatorSpec.java delete mode 100644 samza-operator/src/main/java/org/apache/samza/operators/spec/StoreFunctions.java delete mode 100644 samza-operator/src/test/java/org/apache/samza/operators/TestStateStoreImpl.java delete mode 100644 samza-operator/src/test/java/org/apache/samza/operators/impl/TestSessionWindowImpl.java diff --git a/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java b/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java index 18855e12a3..94ec6fc7b3 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java +++ b/samza-api/src/main/java/org/apache/samza/operators/MessageStream.java @@ -25,6 +25,7 @@ import org.apache.samza.operators.functions.JoinFunction; import org.apache.samza.operators.functions.MapFunction; import org.apache.samza.operators.functions.SinkFunction; +import org.apache.samza.operators.windows.Window; import org.apache.samza.operators.windows.WindowOutput; import java.util.Collection; @@ -35,7 +36,7 @@ *

* A {@link MessageStream} can be transformed into another {@link MessageStream} by applying the transforms in this API. * - * @param type of {@link MessageEnvelope}s in this stream + * @param type of {@link MessageEnvelope}s in this stream */ @InterfaceStability.Unstable public interface MessageStream { @@ -81,22 +82,22 @@ public interface MessageStream { void sink(SinkFunction sinkFn); /** - * Groups the {@link MessageEnvelope}s in this {@link MessageStream} according to the provided {@link org.apache.samza.operators.windows.experimental.Window} semantics + * Groups the {@link MessageEnvelope}s in this {@link MessageStream} according to the provided {@link Window} semantics * (e.g. tumbling, sliding or session windows) and returns the transformed {@link MessageStream} of * {@link WindowOutput}s. *

- * Use the {@link org.apache.samza.operators.windows.experimental.Windows} helper methods to create the appropriate windows. + * Use the {@link org.apache.samza.operators.windows.Windows} helper methods to create the appropriate windows. * - * @param window the {@link org.apache.samza.operators.windows.experimental.Window} to group and process {@link MessageEnvelope}s from this {@link MessageStream} - * @param the type of key in the {@link WindowOutput} from the {@link org.apache.samza.operators.windows.experimental.Window} - * @param the type of value in the {@link WindowOutput} from the {@link org.apache.samza.operators.windows.experimental.Window} - * @param the type of window state kept in the {@link org.apache.samza.operators.windows.experimental.Window} + * @param window the {@link Window} to group and process {@link MessageEnvelope}s from this {@link MessageStream} + * @param the type of key the {@link Window} is computed on. + * @param the type of key in the {@link WindowOutput} from the {@link Window} + * @param the type of value in the {@link WindowOutput} from the {@link Window} * @param the type of {@link WindowOutput} in the transformed {@link MessageStream} * @return the transformed {@link MessageStream} */ > MessageStream window( - org.apache.samza.operators.windows.experimental.Window window); + Window window); /** diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/BaseWindow.java b/samza-api/src/main/java/org/apache/samza/operators/windows/BaseWindow.java new file mode 100644 index 0000000000..3e85d23514 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/operators/windows/BaseWindow.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.samza.operators.windows; + +import org.apache.samza.operators.data.MessageEnvelope; + +import java.util.List; +import java.util.function.BiFunction; +import java.util.function.Function; + +/** + * The base class of all types of {@link Window}s. Sub-classes can specify the default triggering semantics + * for the {@link Window}, semantics for emission of early or late results and whether to accumulate or discard + * previous results. + */ + +public class BaseWindow implements Window, WV, WindowOutput, WV>> { + + /** + * Defines the default triggering semantics for the {@link Window}. + */ + private final List defaultTriggers; + + + /** + * Defines the triggering semantics for emission of early or late results. + */ + private List earlyTriggers; + private List lateTriggers; + + /** + * Defines the fold function that is applied each time a {@link MessageEnvelope} is added to this window. + */ + private final BiFunction aggregator; + + /* + * Defines the function that extracts the event time from a {@link MessageEnvelope} + */ + private final Function eventTimeExtractor; + + /* + * Defines the function that extracts the key from a {@link MessageEnvelope} + */ + private final Function keyExtractor; + + + public BaseWindow(Function keyExtractor, BiFunction aggregator, Function eventTimeExtractor, List defaultTriggers) { + this.aggregator = aggregator; + this.eventTimeExtractor = eventTimeExtractor; + this.keyExtractor = keyExtractor; + this.defaultTriggers = defaultTriggers; + } + + @Override + public Window, WV, WindowOutput, WV>> setTriggers(TriggersBuilder.Triggers wndTrigger) { + this.earlyTriggers = wndTrigger.getEarlyTriggers(); + this.lateTriggers = wndTrigger.getLateTriggers(); + return this; + } + + public List getDefaultTriggers() { + return defaultTriggers; + } + + public List getEarlyTriggers() { + return earlyTriggers; + } + + public List getLateTriggers() { + return lateTriggers; + } + + public BiFunction getAggregator() { + return aggregator; + } + + public Function getEventTimeExtractor() { + return eventTimeExtractor; + } + + public Function getKeyExtractor() { + return keyExtractor; + } +} diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/Time.java b/samza-api/src/main/java/org/apache/samza/operators/windows/Time.java new file mode 100644 index 0000000000..465e441f6c --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/operators/windows/Time.java @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.samza.operators.windows; + +import java.util.concurrent.TimeUnit; + +/** + * Specifies a time interval for applying a {@link Window} transform over a {@code MessageStream}. Users should + * rely on the factory methods to create instances of this class. + * + *

 {@code
+ *  Time threeDays = Time.of(3, TimeUnit.DAYS);
+ *  Time threeMins = Time.minutes(3);
+ *  Time threeSeconds = Time.of(3, TimeUnit.SECONDS);
+ *  MessageStream> windowed = integerStream.window(Windows.tumblingWindow(10000, maxAggregator);
+ * }
+ * 
+ */ +public class Time { + + private final TimeUnit timeUnit; + + private final long value; + + private final Time.TimeCharacteristic characteristic = Time.TimeCharacteristic.PROCESSING_TIME; + + /* + * Should this {@link Time} be specified in terms of event time or processing time. + */ + enum TimeCharacteristic { + EVENT_TIME, PROCESSING_TIME + } + + /** + * Private constructor to allow instantiation only via factory methods + */ + private Time(long value, TimeUnit timeunit) { + this.timeUnit = timeunit; + this.value = value; + } + + public TimeUnit getTimeUnit() { + return timeUnit; + } + + public long getValue() { + return value; + } + + public long toMilliseconds() { + return timeUnit.toMillis(value); + } + + + public static Time of(long size, TimeUnit unit) { + return new Time(size, unit); + } + + /** + * Creates a new {@link Time} that specifies the provided days. + * @param days the provided number of days. + * @return the created instance. + */ + public static Time days(long days) { + return of(days, TimeUnit.DAYS); + } + + /** + * Creates a new {@link Time} that specifies the provided hours. + * @param hours the provided number of hours. + * @return the created instance. + */ + public static Time hours(long hours) { + return of(hours, TimeUnit.HOURS); + } + + /** + * Creates a new {@link Time} that specifies the provided minutes. + * @param min the provided number of minutes. + * @return the created instance. + */ + public static Time minutes(long min) { + return of(min, TimeUnit.MINUTES); + } + + /** + * Creates a new {@link Time} that specifies the provided seconds. + * @param sec the provided number of minutes. + * @return the created instance. + */ + public static Time seconds(long sec) { + return of(sec, TimeUnit.SECONDS); + } + + /** + * Creates a new {@link Time} that specifies the provided milliseconds. + * @param millis the provided number of minutes. + * @return the created instance. + */ + public static Time milliseconds(long millis) { + return of(millis, TimeUnit.MILLISECONDS); + } + +} diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/experimental/TriggersBuilder.java b/samza-api/src/main/java/org/apache/samza/operators/windows/TriggersBuilder.java similarity index 57% rename from samza-api/src/main/java/org/apache/samza/operators/windows/experimental/TriggersBuilder.java rename to samza-api/src/main/java/org/apache/samza/operators/windows/TriggersBuilder.java index 32ffe3423b..d9031e0996 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/windows/experimental/TriggersBuilder.java +++ b/samza-api/src/main/java/org/apache/samza/operators/windows/TriggersBuilder.java @@ -17,35 +17,37 @@ * under the License. */ -package org.apache.samza.operators.windows.experimental; +package org.apache.samza.operators.windows; import org.apache.samza.operators.data.MessageEnvelope; import java.util.ArrayList; +import java.util.Collections; import java.util.List; /** * Programmer API for specifying {@link Triggers}s for a {@link Window}. * - * A {@link Triggers} instance specifies all the early and late triggers for a {@link Window}. It also specifies how results - * from a window relate over time - whether previously fired window results are accumulating (where every result builds - * over its previous ones) or discarding (where every result is independent). + *

A {@link Triggers} instance specifies all the early and late triggers for a {@link Window}. It also specifies how + * window results relate over time. Window results can be accumulating (where every result builds over its previous ones) or + * discarding (where every result is independent). * * @param type of input {@link MessageEnvelope} in the window * @param type of key in {@link MessageEnvelope} * @param type of value in the {@link MessageEnvelope} * - * The below example windows an input into tumbling windows of 10s, and emits early results periodically every 4s in + *

The below example windows an input into tumbling windows of 10s, and emits early results periodically every 4s in * processing time, or for every 50 messages. It also specifies that window results are accumulating. * - *

- *   {@code
- *    MessageStream<> windowedStream = stream.window(Windows.tumblingWindow(10000)
- *      setTriggers(new Triggers<>()
+ * 
 {@code
+ * MessageStream<> windowedStream = stream.window(
+ *   Windows.tumblingWindow(Time.of(10, TimeUnit.SECONDS))
+ *     .setTriggers(new TriggersBuilder<>()
  *       .withEarlyFiringsAfterCountAtleast(50)
- *       .withEarlyFiringsEvery(4000)
- *       .accumulateFiredPanes().build());
+ *       .withEarlyFiringsEvery(Time.of(4, TimeUnit.SECONDS))
+ *       .accumulateFiredPanes()
+ *       .build());
  *    }
  *  
*/ @@ -58,12 +60,6 @@ enum TriggerType { EARLY, LATE, DEFAULT } - /* - * Should the trigger fire in event time or processing time. - */ - enum TimeCharacteristic { - EVENT_TIME, PROCESSING_TIME - } static class Trigger { private final TriggerType type; @@ -74,46 +70,65 @@ static class Trigger { } /* - * Defines a {@link Trigger} that triggers periodically based on the specified time duration. + * Defines a {@link Trigger} that triggers periodically based on the specified time delay. * */ static class PeriodicTimeTrigger extends Trigger { - private final long durationMs; - private final TimeCharacteristic timeCharacteristic; + private final long delayMillis; + private final Time.TimeCharacteristic timeCharacteristic; - PeriodicTimeTrigger(TriggerType type, long duration, TimeCharacteristic timeCharacteristic) { + PeriodicTimeTrigger(TriggerType type, long delayMillis, Time.TimeCharacteristic timeCharacteristic) { super(type); - this.durationMs = duration; + this.delayMillis = delayMillis; this.timeCharacteristic = timeCharacteristic; } + + public long getDelayMillis() { + return delayMillis; + } + + public Time.TimeCharacteristic getTimeCharacteristic() { + return timeCharacteristic; + } } /* * Defines a {@link Trigger} that triggers when there is no message in a {@link Window} for the specified gap duration. - * A late trigger specifies handling of - * late arrivals (caused typically due to skews, and upstream delays) in the data. */ static class TimeSinceLastMessageTrigger extends Trigger { - private final long gapMs; - private final TimeCharacteristic timeCharacteristic; + private final long gapMillis; + private final Time.TimeCharacteristic timeCharacteristic; - TimeSinceLastMessageTrigger(TriggerType type, long timeout, TimeCharacteristic timeCharacteristic) { + TimeSinceLastMessageTrigger(TriggerType type, long gapMillis, Time.TimeCharacteristic timeCharacteristic) { super(type); - this.gapMs = timeout; + this.gapMillis = gapMillis; this.timeCharacteristic = timeCharacteristic; } + + public long getGapMillis() { + return gapMillis; + } + + public Time.TimeCharacteristic getTimeCharacteristic() { + return timeCharacteristic; + } } /* - * Defines a {@link Trigger} that triggers based on the count of messages in the window. + * Defines a {@link Trigger} that triggers based on the count of messages in the {@link Window}. */ static class CountTrigger extends Trigger { private final long count; + CountTrigger(TriggerType type , long count) { super(type); this.count = count; } + + public long getCount() { + return count; + } } @@ -135,7 +150,7 @@ static class Triggers { private final List lateTriggers; private final AccumulationMode accumulationMode; - Triggers(List earlyTriggers, List lateTriggers, AccumulationMode mode) { + private Triggers(List earlyTriggers, List lateTriggers, AccumulationMode mode) { this.earlyTriggers = earlyTriggers; this.lateTriggers = lateTriggers; this.accumulationMode = mode; @@ -155,37 +170,46 @@ AccumulationMode getAccumulationMode() { } /** - * Add an early trigger that periodically emits results after a processing time delay. An early trigger specifies - * emission of an early result before all data for the {@link Window} have arrived. - * @param period the delay period between firings + * Add an early trigger that periodically emits results after a processing time delay. Used to + * emit an early result before all data for the {@link Window} have arrived. + * + * @param delay the delay period between firings + * @return a reference to this object */ - public TriggersBuilder withEarlyFiringsEvery(long period) { - earlyTriggers.add(new PeriodicTimeTrigger(TriggerType.EARLY, period, TimeCharacteristic.PROCESSING_TIME)); + public TriggersBuilder withEarlyFiringsEvery(Time delay) { + earlyTriggers.add(new PeriodicTimeTrigger(TriggerType.EARLY, delay.toMilliseconds(), Time.TimeCharacteristic.PROCESSING_TIME)); return this; } /** - * Add a late trigger that periodically emits results after a processing time delay. - * @param period the delay period between firings + * Add a late trigger that periodically emits results after a processing time delay. Used to emit + * a results for late arrivals. + * + * @param delay the delay period between firings + * @return a reference to this object */ - public TriggersBuilder withLateFiringsEvery(long period) { - lateTriggers.add(new PeriodicTimeTrigger(TriggerType.LATE, period, TimeCharacteristic.PROCESSING_TIME)); + public TriggersBuilder withLateFiringsEvery(Time delay) { + lateTriggers.add(new PeriodicTimeTrigger(TriggerType.LATE, delay.toMilliseconds(), Time.TimeCharacteristic.PROCESSING_TIME)); return this; } /** - * Add an early trigger that periodically emits results after a certain number of messages in the window. - * @param numMessages the delay period between firings An early trigger specifies - * emission of an early result before all data for the {@link Window} have arrived. + * Add an early trigger that emits results after every {@code count} messages. Used to + * emit an early result before all data for the {@link Window} have arrived. + * + * @param count the number of messages + * @return a reference to this object */ - public TriggersBuilder withEarlyFiringsAfterCountAtleast(long numMessages) { - earlyTriggers.add(new CountTrigger(TriggerType.EARLY, numMessages)); + public TriggersBuilder withEarlyFiringsAfterCountAtleast(long count) { + earlyTriggers.add(new CountTrigger(TriggerType.EARLY, count)); return this; } /** * Add a late trigger that periodically emits results after a certain number of messages in the window. + * * @param numMessages the delay period between firings + * @return a reference to this object */ public TriggersBuilder withLateFiringsAfterCountAtleast(long numMessages) { @@ -194,8 +218,10 @@ public TriggersBuilder withLateFiringsAfterCountAtleast(long numMessages) { } /** - * Specifies that previously fired results should be accumulated. This is applicable when each window output builds + * Specifies that previously fired results should be accumulated. This is applicable when each window result builds * on the previous ones. + * + * @return a reference to this object */ public TriggersBuilder accumulateFiredPanes() { this.accumulationMode = AccumulationMode.ACCUMULATING; @@ -203,8 +229,10 @@ public TriggersBuilder accumulateFiredPanes() { } /** - * Specifies that previously fired results should be discarded. This is applicable when each window output is + * Specifies that previously fired results should be discarded. This is applicable when each window result is * independent. + * + * @return a reference to this object */ public TriggersBuilder discardFiredPanes() { this.accumulationMode = AccumulationMode.DISCARDING; @@ -212,9 +240,12 @@ public TriggersBuilder discardFiredPanes() { } /** - * Build an immutable {@link Triggers} from this {@link TriggersBuilder} + * Build an immutable {@link Triggers} instance from this {@link TriggersBuilder} + * + * @return a reference to this object */ public Triggers build() { - return new Triggers(earlyTriggers, lateTriggers, accumulationMode); + return new Triggers(Collections.unmodifiableList(earlyTriggers), Collections.unmodifiableList(lateTriggers), + accumulationMode); } } diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/Window.java b/samza-api/src/main/java/org/apache/samza/operators/windows/Window.java new file mode 100644 index 0000000000..151db48028 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/operators/windows/Window.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.samza.operators.windows; + +import org.apache.samza.annotation.InterfaceStability; +import org.apache.samza.operators.data.MessageEnvelope; + +/** + * A {@link Window} transform slices a {@link org.apache.samza.operators.MessageStream} into smaller finite chunks for + * further processing. Programmers should use the API methods of {@link Windows} to specify their windowing functions. + * + *

There are the following aspects to windowing in Samza: + * + *

    + *
  • Default Trigger: Every {@link Window} has a default trigger that specifies when to emit + * results for the window. + * + *
  • Early and Late Triggers: Users can choose to emit early, partial results speculatively by configuring an early trigger. + * Users can choose to handle arrival of late data by configuring a late trigger. Refer to the {@link TriggersBuilder} APIs for + * configuring early and late triggers. + * + *
  • Key: A {@link Window} transform can be evaluated on a "per-key" basis. For instance, A common use-case is to group a + * stream based on a specified key over a tumbling time window. In this case, the triggering behavior is per-key and per-window. + * + *
+ * + * @param type of input {@link MessageEnvelope}. + * @param type of key to group by when evaluating a {@link Window}. + * @param type of key in the {@link Window} output. + * @param type of value stored in the {@link Window}. + * @param type of the {@link Window} result. + */ + +@InterfaceStability.Unstable +public interface Window> { + + /** + * Set the triggers for this {@link Window} + * + * @param wndTrigger trigger conditions set by the programmers + * @return the {@link Window} function w/ the trigger {@code wndTrigger} + */ + Window setTriggers(TriggersBuilder.Triggers wndTrigger); +} diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/WindowKey.java b/samza-api/src/main/java/org/apache/samza/operators/windows/WindowKey.java new file mode 100644 index 0000000000..4394c5e589 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/operators/windows/WindowKey.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.samza.operators.windows; + +/** + * Provides information about every result emitted from a {@link Window}. Every {@link Window} fires results according to + * its triggering semantics. + * + * @param , the key that the {@link Window} is keyed by. Windows that are not keyed have a Void + * key type. + */ +public class WindowKey { + + /** + * The start time of this firing. + */ + private final long windowStart; + + /** + * The end time of this firing. + */ + private final long windowEnd; + + /** + * The key that the {@link Window} is keyed by. + */ + private final K key; + + WindowKey(K key, long windowStart, long windowEnd) { + this.windowStart = windowStart; + this.windowEnd = windowEnd; + this.key = key; + } + + public long getWindowStart() { + return windowStart; + } + + public long getWindowEnd() { + return windowEnd; + } + + public K getKey() { + return key; + } +} + diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/Windows.java b/samza-api/src/main/java/org/apache/samza/operators/windows/Windows.java new file mode 100644 index 0000000000..8ab37e1e46 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/operators/windows/Windows.java @@ -0,0 +1,426 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.samza.operators.windows; + +import org.apache.samza.operators.data.MessageEnvelope; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.function.BiFunction; +import java.util.function.Function; + +/** + * Factory methods for creating a {@link Window}. Provides APIs for specifying different types of {@link Window} transforms. + * + */ +public class Windows { + + /** + * Returns a keyed {@link Window} that windows groups its values per-key into fixed-size processing time-based windows + * and aggregates values and applying the provided aggregator function. The windows are non-overlapping. + * + *

The below example groups the data by the provided key and into fixed-size 10 second windows for each key. It + * emits the maximum element per-window per-key. + * + *

 {@code
+   * MessageStream stream = ...;
+   * Function< UserClick, String> keyExtractor = ...;
+   *  BiFunction maxAggregator = (m, c)-> Math.max(parseInt(m), c);
+   *  MessageStream, Integer>> windowed = integerStream.window(
+   *  Windows.tumblingWindow(keyExtractor, Time.seconds(10), maxAggregator));
+   * }
+   * 
+ * + * @param interval the duration in processing time on which the window is computed. + * @param aggregateFunction the function to aggregate window results + * @param keyFn the function to extract the key from a {@link MessageEnvelope} + * @param type of the input {@link MessageEnvelope} + * @param type of the {@link WindowOutput} output value + * @param type of the key in the {@link Window} + * @return the created {@link Window} function. + */ + + + public static Window, WV, WindowOutput, WV>> + keyedTumblingWindow(Function keyFn, Time interval, BiFunction aggregateFunction) { + + List defaultTriggers = new ArrayList<>(); + long intervalMs = interval.toMilliseconds(); + TriggersBuilder.Trigger timeTrigger = new TriggersBuilder.PeriodicTimeTrigger(TriggersBuilder.TriggerType.DEFAULT, intervalMs, + Time.TimeCharacteristic.PROCESSING_TIME); + defaultTriggers.add(timeTrigger); + + return new BaseWindow(keyFn, aggregateFunction, null, defaultTriggers); + } + + + /** + * Returns a keyed {@link Window} that windows groups its values per-key into fixed-size processing time-based windows. + * The windows are non-overlapping. + * + *

The below example groups the data by the provided key and into fixed-size 10 second tumbling windows for each key. + * + *

 {@code
+   * MessageStream stream = ...;
+   * Function< UserClick, String> keyExtractor = ...;
+   *  MessageStream, Collection>> windowed = integerStream.window(
+   *  Windows.tumblingWindow(keyExtractor, Time.seconds(10)));
+   * }
+   * 
+ * + * @param interval the duration in processing time on which the window is computed. + * @param keyFn function to extract key from the {@link MessageEnvelope}. + * @param type of the input {@link MessageEnvelope} + * @param type of the key in the {@link Window} + * @return the created {@link Window} function. + */ + + public static Window, Collection, WindowOutput, Collection>> + keyedTumblingWindow(Function keyFn, Time interval) { + BiFunction, Collection> aggregator = (m, c) -> { + c.add(m); + return c; + }; + return keyedTumblingWindow(keyFn, interval, aggregator); + } + + /** + * Returns a {@link Window} that windows values into fixed-size processing time-based windows and aggregates values + * applying the provided aggregator function. + * + *

For example, in order to partition the data by 10 second windows and emit the maximum element every 10 seconds. + * + *

 {@code
+   * MessageStream stream = ...;
+   *  BiFunction maxAggregator = (m, c)-> Math.max(parseInt(m), c);
+   *  MessageStream> windowed = integerStream.window(
+   *  Windows.tumblingWindow(Time.seconds(10), maxAggregator));
+   * }
+   * 
+ * + * @param duration the duration in processing time on which the window is computed. + * @param aggregateFunction the function to aggregate window results + * @param type of the input {@link MessageEnvelope} + * @param type of the {@link WindowOutput} output value + * @return the created {@link Window} function. + */ + + public static Window, WV, WindowOutput, WV>> + tumblingWindow(Time duration, BiFunction aggregateFunction) { + + List defaultTriggers = new ArrayList<>(); + + TriggersBuilder.Trigger timeTrigger = new TriggersBuilder.PeriodicTimeTrigger(TriggersBuilder.TriggerType.DEFAULT, duration.toMilliseconds(), + Time.TimeCharacteristic.PROCESSING_TIME); + defaultTriggers.add(timeTrigger); + + return new BaseWindow(null, aggregateFunction, null, defaultTriggers); + } + + /** + * Returns a {@link Window} that windows its values into fixed-size processing time-based windows. + * + *

For example, in order to partition the data by 10 minute windows and apply a map transform later on the list of + * messages in the window. + * + *

 {@code
+   * MessageStream stream = ...;
+   * Function> percentile99 = ..
+   *
+   *  MessageStream>> windowed = integerStream.window(Windows.tumblingWindow(Time.minutes(10));
+   *  MessageStream windowedPercentiles = windowed.map(windowedOutput -> percentile99(windowedOutput.getMessage())));
+   * }
+   * 
+ * + * @param duration the duration in processing time on which the window is computed. + * @param type of the input {@link MessageEnvelope} + * @return the created {@link Window} function. + */ + + public static Window, Collection, WindowOutput, Collection>> tumblingWindow(Time duration) { + BiFunction, Collection> aggregator = (m, c) -> { + c.add(m); + return c; + }; + return tumblingWindow(duration, aggregator); + } + + /** + * Returns a {@link Window} that windows values into sessions based on the {@code sessionGap} and aggregates values + * applying the provided aggregator function. A Session captures some period of activity over a {@code MessageStream}. + * The boundary for the session is defined by a timeout gap. All data that arrives within a span of the timeout gap are + * grouped to constitute a single session. + * + *

For example, in order to group the data by 10 second session windows and emit the maximum element in each session. + * + *

 {@code
+   * MessageStream stream = ...;
+   *  BiFunction maxAggregator = (m, c)-> Math.max(parseInt(m), c);
+   *  MessageStream> windowed = integerStream.window(
+   *  Windows.sessionWindow(Time.seconds(10), maxAggregator));
+   * }
+   * 
+ * + * @param sessionGap the timeout gap for defining the session. + * @param aggregateFunction the function to aggregate window results + * @param type of the input {@link MessageEnvelope} + * @param type of the {@link WindowOutput} output value + * @return the created {@link Window} function. + */ + + public static Window, WV, WindowOutput, WV>> sessionWindow(Time sessionGap, BiFunction aggregateFunction) { + + List defaultTriggers = new ArrayList<>(); + + TriggersBuilder.Trigger timeTrigger = new TriggersBuilder.TimeSinceLastMessageTrigger(TriggersBuilder.TriggerType.DEFAULT, sessionGap.toMilliseconds(), + Time.TimeCharacteristic.PROCESSING_TIME); + defaultTriggers.add(timeTrigger); + + return new BaseWindow(null, aggregateFunction, null, defaultTriggers); + } + + /** + * Returns a {@link Window} that windows values into sessions based on the provided {@code sessionGap}. A Session + * captures some period of activity over a {@code MessageStream}. + * + * The boundary for the session is defined by a timeout gap. All data that arrives within a span of the timeout gap + * are grouped to constitute a single session. + * + *

For example, in order to partition the data by 10 minute session windows and apply a map transform later on the list of + * messages in the window. + * + *

 {@code
+   * MessageStream stream = ...;
+   * Function> percentile99 = ..
+   *
+   *  MessageStream>> windowed = integerStream.window(Windows.sessionWindows(Time.minutes(10));
+   *  MessageStream windowedPercentiles = windowed.map(windowedOutput -> percentile99(windowedOutput.getMessage())));
+   * }
+   * 
+ * + * @param sessionGap the timeout gap for defining the session. + * @param type of the input {@link MessageEnvelope} + * @return the created {@link Window} function. + */ + + public static Window, Collection, WindowOutput, Collection>> sessionWindow(Time sessionGap) { + BiFunction, Collection> aggregator = (m, c) -> { + c.add(m); + return c; + }; + return sessionWindow(sessionGap, aggregator); + } + + /** + * Returns a keyed {@link Window} that windows values into sessions based on the {@code sessionGap} + * and aggregates values applying the provided aggregator function. Windowing operations are done "per-key". + * + *

A Session captures some period of activity over a {@code MessageStream}. The boundary for the session is defined by + * the timeout gap per-key. All data that arrives within a span of the timeout gap are grouped into a + * single session for that key. + * + *

For example, to track sessions(defined by a timeout gap of 1 minute) grouped by userId and compute some + * aggregation over a stream of user clicks. + * + *

 {@code
+   * MessageStream stream = ...;
+   *  BiFunction maxAggregator = (m, c)-> Math.max(parseInt(m), c);
+   *  Function userIdExtractor = m -> m.getUserId()..;
+   *  MessageStream> windowed = integerStream.window(
+   *  Windows.keyedSessionWindow(userIdExtractor, Time.minute(1), maxAggregator));
+   * }
+   * 
+ * + * @param keyFn the function to extract the key from a {@link MessageEnvelope} + * @param sessionGap the timeout gap for defining the session. + * @param aggregateFunction the function to aggregate window results + * @param type of the input {@link MessageEnvelope} + * @param type of the key in the {@link Window} + * @param type of the {@link WindowOutput} output value + * @return the created {@link Window} function. + */ + + + public static Window, WV, WindowOutput, WV>> keyedSessionWindow(Function keyFn, Time sessionGap, BiFunction aggregateFunction) { + List defaultTriggers = new ArrayList<>(); + + TriggersBuilder.Trigger timeTrigger = new TriggersBuilder.TimeSinceLastMessageTrigger(TriggersBuilder.TriggerType.DEFAULT, sessionGap.toMilliseconds(), + Time.TimeCharacteristic.PROCESSING_TIME); + defaultTriggers.add(timeTrigger); + + return new BaseWindow(keyFn, aggregateFunction, null, defaultTriggers); + } + + + /** + * Returns a keyed {@link Window} that windows values into sessions based on the {@code sessionGap} + * Windowing operations are done "per-key". + * + *

A Session captures some period of activity over a {@code MessageStream}. The boundary for the session is defined by + * the timeout gap per-key. All data that arrives within a span of the timeout gap are grouped into a + * single session for that key. + * + *

For example, to track sessions(defined by a timeout gap of 1 minute) grouped by userId. + * + *

 {@code
+   * MessageStream stream = ...;
+   *  BiFunction maxAggregator = (m, c)-> Math.max(parseIntField(m), c);
+   *  Function userIdExtractor = m -> m.getUserId()..;
+   *  MessageStream, Collection>> windowed = stream.window(
+   *  Windows.keyedSessionWindow(userIdExtractor, Time.minute(1)));
+   * }
+   * 
+ * + * @param keyFn the function to extract the key from a {@link MessageEnvelope} + * @param sessionGap the timeout gap for defining the session. + * @param type of the input {@link MessageEnvelope} + * @param type of the key in the {@link Window} + * @return the created {@link Window} function. + */ + + public static Window, Collection, WindowOutput, Collection>> keyedSessionWindow(Function keyFn, Time sessionGap) { + + BiFunction, Collection> aggregator = (m, c) -> { + c.add(m); + return c; + }; + return keyedSessionWindow(keyFn, sessionGap, aggregator); + } + + + /** + * Returns a global {@link Window} that windows all values into a single large window. This window does not have a + * default triggering behavior. The triggering behavior must be specified by setting early triggers using the APIs + * of {@link TriggersBuilder}. + * + *

For example, to window a stream into count based windows that trigger every 50 messages or every 10 minutes and + * compute the maximum value over each window. + + *

 {@code
+   * MessageStream stream = ...;
+   * BiFunction maxAggregator = (m, c)-> Math.max(m, c);
+   * MessageStream> windowed = stream.window(Windows.globalWindow(maxAggregator)
+   * .setTriggers(new TriggersBuilder()
+   *   .withEarlyFiringsEvery(Time.minutes(10))
+   *   .withEarlyFiringsAfterCountAtleast(50).build()));
+   * }
+   * 
+ * + * @param aggregateFunction the function to aggregate window results + * @param the type of {@link MessageEnvelope} + * @param type of the {@link WindowOutput} output value + * @return the created {@link Window} function. + */ + public static Window, WV, WindowOutput, WV>> globalWindow(BiFunction aggregateFunction) { + return new BaseWindow(null, aggregateFunction, null, null); + } + + /** + * Returns a global {@link Window} that windows all values into a single large window. This window does not have a + * default triggering behavior. The triggering behavior must be specified by setting early triggers using the APIs + * of {@link TriggersBuilder}. + * + * For example, to window a stream into count based windows that trigger every 50 messages or every 10 minutes. + *
 {@code
+   * MessageStream stream = ...;
+   * MessageStream>> windowed = stream.window(Windows.globalWindow()
+   * .setTriggers(new TriggersBuilder()
+   *   .withEarlyFiringsEvery(Time.minutes(10))
+   *   .withEarlyFiringsAfterCountAtleast(50).build()));
+   * }
+   * 
+ * + * @param the type of {@link MessageEnvelope} + * @return the created {@link Window} function. + */ + + public static Window, Collection, WindowOutput, Collection>> globalWindow() { + BiFunction, Collection> aggregator = (m, c) -> { + c.add(m); + return c; + }; + return globalWindow(aggregator); + } + + /** + * Returns a global {@link Window} that groups the values per-key and assigns them to their own global window. + * This window does not have a default triggering behavior. The triggering behavior must be specified by setting early + * triggers using the APIs of {@link TriggersBuilder}. Triggers are fired per-key. + * + *

The below example groups the stream by the key and windows the stream into count based windows. The window triggers + * every 50 messages or every 10 minutes. It emits the maximum value per-window per-key. + * + *

 {@code
+   * MessageStream stream = ...;
+   * BiFunction maxAggregator = (m, c)-> Math.max(parseLongField(m), c);
+   * Function keyFn = ...;
+   * MessageStream, Long>> windowed = stream.window(Windows.keyedGlobalWindow(keyFn, maxAggregator)
+   * .setTriggers(new TriggersBuilder()
+   *   .withEarlyFiringsEvery(Time.minutes(10))
+   *   .withEarlyFiringsAfterCountAtleast(50).build()));
+   * }
+   * 
+ * + * @param keyFn the function to extract the key from a {@link MessageEnvelope} + * @param aggregateFunction the function to aggregate window results + * @param the type of {@link MessageEnvelope} + * @param type of the key in the {@link Window} + * @param type of the {@link WindowOutput} output value + * @return the created {@link Window} function. + */ + + public static Window, WV, WindowOutput, WV>> keyedGlobalWindow(Function keyFn, BiFunction aggregateFunction) { + return new BaseWindow(keyFn, aggregateFunction, null, null); + } + + /** + * Returns a global {@link Window} that groups the values per-key and assigns them to their own global window. + * This window does not have a default triggering behavior. The triggering behavior must be specified by setting early + * triggers using the APIs of {@link TriggersBuilder}. Triggers are fired per-key. + * + *

The below example groups the stream by the key and windows the stream into count based windows. The window triggers + * every 50 messages or every 10 minutes. + + *

 {@code
+   * MessageStream stream = ...;
+   * Function keyFn = ...;
+   * MessageStream, Collection>> windowed = stream.window(Windows.keyedGlobalWindow(keyFn)
+   * .setTriggers(new TriggersBuilder()
+   *   .withEarlyFiringsEvery(Time.minutes(10))
+   *   .withEarlyFiringsAfterCountAtleast(50).build()));
+   * }
+   * 
+ * + * @param keyFn the function to extract the key from a {@link MessageEnvelope} + * @param the type of {@link MessageEnvelope} + * @param type of the key in the {@link Window} + * @return the created {@link Window} function. + */ + + + public static Window, Collection, WindowOutput, Collection>> keyedGlobalWindow(Function keyFn) { + BiFunction, Collection> aggregator = (m, c) -> { + c.add(m); + return c; + }; + return keyedGlobalWindow(keyFn, aggregator); + } +} diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/examples/WindowingExamples.java b/samza-api/src/main/java/org/apache/samza/operators/windows/examples/WindowingExamples.java index e0650b042f..2372af414f 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/windows/examples/WindowingExamples.java +++ b/samza-api/src/main/java/org/apache/samza/operators/windows/examples/WindowingExamples.java @@ -1,24 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.samza.operators.windows.examples; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.data.IncomingSystemMessageEnvelope; import org.apache.samza.operators.data.MessageEnvelope; -import org.apache.samza.operators.windows.WindowOutput; -import org.apache.samza.operators.windows.experimental.TriggersBuilder; -import org.apache.samza.operators.windows.experimental.WindowKey; -import org.apache.samza.operators.windows.experimental.Windows; +import org.apache.samza.operators.windows.*; import java.util.Collection; import java.util.function.BiFunction; import java.util.function.Function; /** - * Examples for programming using the {@link org.apache.samza.operators.windows.experimental.Windows} APIs. + * Examples for programming using the {@link Windows} APIs. */ public class WindowingExamples { - private static final long INTERVAL_MS = 1000; - private static final long SESSION_GAP_MS = 2000; + private static final Time INTERVAL_MS = Time.milliseconds(1000); + private static final Time SESSION_GAP_MS = Time.milliseconds(2000); private static Integer parseInt(IncomingSystemMessageEnvelope msg) { //parse an integer from the message @@ -56,10 +72,10 @@ public static void main(String[] args) { // - Messages have a maximum allowed lateness of 50000ms. BiFunction maxAggregator = (m, c)-> Math.max(parseInt(m), c); - MessageStream, Integer>> window1 = integerStream.window(Windows.tumblingWindow(10000, maxAggregator) + MessageStream, Integer>> window1 = integerStream.window(Windows.tumblingWindow(INTERVAL_MS, maxAggregator) .setTriggers(new TriggersBuilder() .withEarlyFiringsAfterCountAtleast(50) - .withEarlyFiringsEvery(4000) + .withEarlyFiringsEvery(Time.seconds(4)) .withLateFiringsAfterCountAtleast(20) .discardFiredPanes() .build())); @@ -76,7 +92,7 @@ public static void main(String[] args) { final MessageStream windowedPercentiles = integerStream.window(Windows.keyedSessionWindow(keyExtractor, SESSION_GAP_MS) .setTriggers(new TriggersBuilder() .withEarlyFiringsAfterCountAtleast(50) - .withEarlyFiringsEvery(4000) + .withEarlyFiringsEvery(Time.seconds(4)) .withLateFiringsAfterCountAtleast(20) .accumulateFiredPanes() .build())) @@ -100,7 +116,7 @@ public static void main(String[] args) { /* //A CUSTOM GLOBAL WINDOW: Demonstrates a window with custom triggering every 500 messages - final MessageStream>> customWindow = integerStream.window(Windows.customGlobalWindow().setTriggers(new TriggerSpecBuilder() + final MessageStream>> customWindow = integerStream.window(Windows.customGlobalWindow().setTriggers(new TriggerSpecBuilder() .withEarlyFiringsAfterCountAtleast(500) .discardFiredPanes() .build())); diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/experimental/Window.java b/samza-api/src/main/java/org/apache/samza/operators/windows/experimental/Window.java deleted file mode 100644 index e0753322f6..0000000000 --- a/samza-api/src/main/java/org/apache/samza/operators/windows/experimental/Window.java +++ /dev/null @@ -1,43 +0,0 @@ -package org.apache.samza.operators.windows.experimental; - -import org.apache.samza.operators.data.MessageEnvelope; -import org.apache.samza.operators.windows.WindowOutput; - -/** - * A {@link Window} subdivides a {@link org.apache.samza.operators.MessageStream} into smaller finite chunks. Programmers - * should use the API methods of {@link Windows} to specify their windowing functions. - * - * There are the following aspects to windowing in Samza: - * - * Default Trigger: Every {@link Window} has - * a default {@link TriggersBuilder.Trigger} that specifies when to emit - * its results. - * - * Early and Late Triggers: Users can choose to emit early, partial results speculatively by configuring an early trigger. Users - * can choose to handle arrival of late data by configuring a late trigger. Refer to the {@link TriggersBuilder} APIs for - * configuring early and late triggers. - * - * Aggregation: Every {@link Window} has an aggregation function to be applied on each element that returns the type of the value - * stored in the window. - * - * Key Function: A {@link Window} can be keyed by a certain key in the input {@link MessageEnvelope}. For example, A common use-case is to perform - * key based aggregations over a time window. When a key function is specified, the triggering behavior is per key per window. - * - * - * @param type of input {@link MessageEnvelope}. - * @param type of key to use for aggregation. - * @param type of key in the window output. - * @param type of value stored in the {@link Window}. - * @param type of the {@link Window} result. - */ - -public interface Window> { - - /** - * Set the triggers for this {@link Window} - * - * @param wndTrigger trigger conditions set by the programmers - * @return the {@link Window} function w/ the trigger {@code wndTrigger} - */ - Window setTriggers(TriggersBuilder.Triggers wndTrigger); -} diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/experimental/WindowDefinition.java b/samza-api/src/main/java/org/apache/samza/operators/windows/experimental/WindowDefinition.java deleted file mode 100644 index 8f9f675ded..0000000000 --- a/samza-api/src/main/java/org/apache/samza/operators/windows/experimental/WindowDefinition.java +++ /dev/null @@ -1,83 +0,0 @@ -package org.apache.samza.operators.windows.experimental; - -import org.apache.samza.operators.data.MessageEnvelope; -import org.apache.samza.operators.windows.WindowOutput; - -import java.util.List; -import java.util.function.BiFunction; -import java.util.function.Function; - -/** - * The base class of all types of {@link Window}s. Sub-classes can specify the default triggering semantics - * for the {@link Window}, semantics for emission of early or late results and whether to accumulate or discard - * previous results. - */ - -public class WindowDefinition implements Window, WV, WindowOutput, WV>> { - - /** - * Defines the default triggering semantics for the {@link Window}. - */ - private final List defaultFirings; - - - /** - * Defines the triggering semantics for emission of early or late results. - */ - private List earlyFirings; - private List lateFirings; - - /** - * Defines the fold function that is applied each time a {@link MessageEnvelope} is added to the window. - */ - private final BiFunction aggregator; - - /* - * Defines the function that extracts the event time from a {@link MessageEnvelope} - */ - private final Function eventTimeExtractor; - - /* - * Defines the function that extracts the key from a {@link MessageEnvelope} - */ - private final Function keyExtractor; - - - public WindowDefinition(Function keyExtractor, BiFunction aggregator, Function eventTimeExtractor, List defaultFirings) { - this.aggregator = aggregator; - this.eventTimeExtractor = eventTimeExtractor; - this.keyExtractor = keyExtractor; - this.defaultFirings = defaultFirings; - } - - @Override - public Window, WV, WindowOutput, WV>> setTriggers(TriggersBuilder.Triggers wndTrigger) { - this.earlyFirings = wndTrigger.getEarlyTriggers(); - this.lateFirings = wndTrigger.getLateTriggers(); - return this; - } - - public List getDefaultFirings() { - return defaultFirings; - } - - public List getEarlyFirings() { - return earlyFirings; - } - - public List getLateFirings() { - return lateFirings; - } - - public BiFunction getAggregator() { - return aggregator; - } - - public Function getEventTimeExtractor() { - return eventTimeExtractor; - } - - public Function getKeyExtractor() { - return keyExtractor; - } -} diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/experimental/WindowKey.java b/samza-api/src/main/java/org/apache/samza/operators/windows/experimental/WindowKey.java deleted file mode 100644 index 9408ca24a4..0000000000 --- a/samza-api/src/main/java/org/apache/samza/operators/windows/experimental/WindowKey.java +++ /dev/null @@ -1,27 +0,0 @@ - -package org.apache.samza.operators.windows.experimental; - -public class WindowKey { - private final long windowStart; - private final long windowEnd; - private final K key; - - WindowKey(K key, long windowStart, long windowEnd) { - this.windowStart = windowStart; - this.windowEnd = windowEnd; - this.key = key; - } - - public long getWindowStart() { - return windowStart; - } - - public long getWindowEnd() { - return windowEnd; - } - - public K getKey() { - return key; - } -} - diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/experimental/Windows.java b/samza-api/src/main/java/org/apache/samza/operators/windows/experimental/Windows.java deleted file mode 100644 index 098cc018a0..0000000000 --- a/samza-api/src/main/java/org/apache/samza/operators/windows/experimental/Windows.java +++ /dev/null @@ -1,115 +0,0 @@ -package org.apache.samza.operators.windows.experimental; - -import org.apache.samza.operators.data.MessageEnvelope; -import org.apache.samza.operators.windows.WindowOutput; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import java.util.function.BiFunction; -import java.util.function.Function; - -public class Windows { - - public static Window, WV, WindowOutput, WV>> keyedTumblingWindow(Function keyFn, long interval, BiFunction aggregateFunction) { - - List defaultTriggers = new ArrayList<>(); - - TriggersBuilder.Trigger timeTrigger = new TriggersBuilder.PeriodicTimeTrigger(TriggersBuilder.TriggerType.DEFAULT, interval, - TriggersBuilder.TimeCharacteristic.PROCESSING_TIME); - defaultTriggers.add(timeTrigger); - - return new WindowDefinition(keyFn, aggregateFunction, null, defaultTriggers); - } - - public static Window, Collection, WindowOutput, Collection>> keyedTumblingWindow(Function keyFn, long interval) { - BiFunction, Collection> aggregator = (m, c) -> { - c.add(m); - return c; - }; - return keyedTumblingWindow(keyFn, interval, aggregator); - } - - public static Window, WV, WindowOutput, WV>> tumblingWindow(long interval, BiFunction aggregateFunction) { - - List defaultTriggers = new ArrayList<>(); - - TriggersBuilder.Trigger timeTrigger = new TriggersBuilder.PeriodicTimeTrigger(TriggersBuilder.TriggerType.DEFAULT, interval, - TriggersBuilder.TimeCharacteristic.PROCESSING_TIME); - defaultTriggers.add(timeTrigger); - - return new WindowDefinition(null, aggregateFunction, null, defaultTriggers); - } - - public static Window, Collection, WindowOutput, Collection>> tumblingWindow(long interval) { - BiFunction, Collection> aggregator = (m, c) -> { - c.add(m); - return c; - }; - return tumblingWindow(interval, aggregator); - } - - public static Window, WV, WindowOutput, WV>> sessionWindow(long sessionGap, BiFunction aggregateFunction) { - - List defaultTriggers = new ArrayList<>(); - - TriggersBuilder.Trigger timeTrigger = new TriggersBuilder.TimeSinceLastMessageTrigger(TriggersBuilder.TriggerType.DEFAULT, sessionGap, - TriggersBuilder.TimeCharacteristic.PROCESSING_TIME); - defaultTriggers.add(timeTrigger); - - return new WindowDefinition(null, aggregateFunction, null, defaultTriggers); - } - - public static Window, Collection, WindowOutput, Collection>> sessionWindow(long gap) { - BiFunction, Collection> aggregator = (m, c) -> { - c.add(m); - return c; - }; - return sessionWindow(gap, aggregator); - } - - public static Window, WV, WindowOutput, WV>> keyedSessionWindow(Function keyFn, long sessionGap, BiFunction aggregateFunction) { - List defaultTriggers = new ArrayList<>(); - - TriggersBuilder.Trigger timeTrigger = new TriggersBuilder.TimeSinceLastMessageTrigger(TriggersBuilder.TriggerType.DEFAULT, sessionGap, - TriggersBuilder.TimeCharacteristic.PROCESSING_TIME); - defaultTriggers.add(timeTrigger); - - return new WindowDefinition(keyFn, aggregateFunction, null, defaultTriggers); - } - - public static Window, Collection, WindowOutput, Collection>> keyedSessionWindow(Function keyFn, long sessionGap) { - - BiFunction, Collection> aggregator = (m, c) -> { - c.add(m); - return c; - }; - return keyedSessionWindow(keyFn, sessionGap, aggregator); - } - - - - public static Window, WV, WindowOutput, WV>> globalWindow(BiFunction aggregateFunction) { - return new WindowDefinition(null, aggregateFunction, null, null); - } - - public static Window, Collection, WindowOutput, Collection>> globalWindow() { - BiFunction, Collection> aggregator = (m, c) -> { - c.add(m); - return c; - }; - return globalWindow(aggregator); - } - - public static Window, WV, WindowOutput, WV>> keyedGlobalWindow(Function keyFn, BiFunction aggregateFunction) { - return new WindowDefinition(keyFn, aggregateFunction, null, null); - } - - public static Window, Collection, WindowOutput, Collection>> keyedGlobalWindow(Function keyFn) { - BiFunction, Collection> aggregator = (m, c) -> { - c.add(m); - return c; - }; - return keyedGlobalWindow(keyFn, aggregator); - } -} diff --git a/samza-api/src/test/java/org/apache/samza/operators/windows/TestTrigger.java b/samza-api/src/test/java/org/apache/samza/operators/windows/TestTrigger.java deleted file mode 100644 index a91af243b4..0000000000 --- a/samza-api/src/test/java/org/apache/samza/operators/windows/TestTrigger.java +++ /dev/null @@ -1,68 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.samza.operators.windows; - -import org.apache.samza.operators.data.MessageEnvelope; -import org.junit.Test; - -import java.lang.reflect.Field; -import java.util.concurrent.TimeUnit; -import java.util.function.BiFunction; -import java.util.function.Function; - -import static org.junit.Assert.assertEquals; - - -public class TestTrigger { - - @Test - public void testConstructor() throws Exception { - BiFunction, WindowState, Boolean> earlyTrigger = (m, s) -> s.getOutputValue() > 1000; - BiFunction, WindowState, Boolean> lateTrigger = (m, s) -> s.getOutputValue() > 1000; - Function, Boolean> timerTrigger = s -> TimeUnit.NANOSECONDS.toMillis(s.getLastMessageTimeNs()) + 50000 < System.currentTimeMillis(); - Function, WindowState> earlyTriggerUpdater = s -> { - s.setOutputValue(0); - return s; - }; - Function, WindowState> lateTriggerUpdater = s -> { - s.setOutputValue(1); - return s; - }; - - Trigger, WindowState> trigger = Trigger.createTrigger(timerTrigger, earlyTrigger, lateTrigger, - earlyTriggerUpdater, lateTriggerUpdater); - - Field earlyTriggerField = Trigger.class.getDeclaredField("earlyTrigger"); - Field timerTriggerField = Trigger.class.getDeclaredField("timerTrigger"); - Field lateTriggerField = Trigger.class.getDeclaredField("lateTrigger"); - Field earlyTriggerUpdaterField = Trigger.class.getDeclaredField("earlyTriggerUpdater"); - Field lateTriggerUpdaterField = Trigger.class.getDeclaredField("lateTriggerUpdater"); - earlyTriggerField.setAccessible(true); - lateTriggerField.setAccessible(true); - timerTriggerField.setAccessible(true); - earlyTriggerUpdaterField.setAccessible(true); - lateTriggerUpdaterField.setAccessible(true); - - assertEquals(earlyTrigger, earlyTriggerField.get(trigger)); - assertEquals(timerTrigger, timerTriggerField.get(trigger)); - assertEquals(lateTrigger, lateTriggerField.get(trigger)); - assertEquals(earlyTriggerUpdater, earlyTriggerUpdaterField.get(trigger)); - assertEquals(lateTriggerUpdater, lateTriggerUpdaterField.get(trigger)); - } -} diff --git a/samza-api/src/test/java/org/apache/samza/operators/windows/TestTriggerBuilder.java b/samza-api/src/test/java/org/apache/samza/operators/windows/TestTriggerBuilder.java deleted file mode 100644 index 6a9b55d3e5..0000000000 --- a/samza-api/src/test/java/org/apache/samza/operators/windows/TestTriggerBuilder.java +++ /dev/null @@ -1,226 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.samza.operators.windows; - -import org.apache.samza.operators.TestMessageEnvelope; -import org.junit.Before; -import org.junit.Test; - -import java.lang.reflect.Field; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import java.util.concurrent.TimeUnit; -import java.util.function.BiFunction; -import java.util.function.Function; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - - -public class TestTriggerBuilder { - private Field earlyTriggerField; - private Field lateTriggerField; - private Field timerTriggerField; - private Field earlyTriggerUpdater; - private Field lateTriggerUpdater; - - @Before - public void testPrep() throws Exception { - this.earlyTriggerField = TriggerBuilder.class.getDeclaredField("earlyTrigger"); - this.lateTriggerField = TriggerBuilder.class.getDeclaredField("lateTrigger"); - this.timerTriggerField = TriggerBuilder.class.getDeclaredField("timerTrigger"); - this.earlyTriggerUpdater = TriggerBuilder.class.getDeclaredField("earlyTriggerUpdater"); - this.lateTriggerUpdater = TriggerBuilder.class.getDeclaredField("lateTriggerUpdater"); - - this.earlyTriggerField.setAccessible(true); - this.lateTriggerField.setAccessible(true); - this.timerTriggerField.setAccessible(true); - this.earlyTriggerUpdater.setAccessible(true); - this.lateTriggerUpdater.setAccessible(true); - } - - @Test - public void testStaticCreators() throws NoSuchFieldException, IllegalAccessException { - TriggerBuilder> builder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000); - BiFunction>, Boolean> triggerField = - (BiFunction>, Boolean>) this.earlyTriggerField.get(builder); - WindowState> mockState = mock(WindowState.class); - when(mockState.getNumberMessages()).thenReturn(200L); - assertFalse(triggerField.apply(null, mockState)); - when(mockState.getNumberMessages()).thenReturn(2000L); - assertTrue(triggerField.apply(null, mockState)); - - Function tokenFunc = m -> true; - builder = TriggerBuilder.earlyTriggerOnTokenMsg(tokenFunc); - triggerField = (BiFunction>, Boolean>) this.earlyTriggerField.get(builder); - TestMessageEnvelope m = mock(TestMessageEnvelope.class); - assertTrue(triggerField.apply(m, mockState)); - - builder = TriggerBuilder.earlyTriggerOnEventTime(tm -> tm.getMessage().getEventTime(), 30000L); - triggerField = (BiFunction>, Boolean>) this.earlyTriggerField.get(builder); - when(mockState.getEarliestEventTimeNs()).thenReturn(1000000000L); - when(mockState.getLatestEventTimeNs()).thenReturn(20000000000L); - TestMessageEnvelope.MessageType mockInnerMessage; - mockInnerMessage = mock(TestMessageEnvelope.MessageType.class); - when(mockInnerMessage.getEventTime()).thenReturn(19999000000L); - when(m.getMessage()).thenReturn(mockInnerMessage); - assertFalse(triggerField.apply(m, mockState)); - mockInnerMessage = mock(TestMessageEnvelope.MessageType.class); - when(mockInnerMessage.getEventTime()).thenReturn(32000000000L); - when(m.getMessage()).thenReturn(mockInnerMessage); - assertTrue(triggerField.apply(m, mockState)); - mockInnerMessage = mock(TestMessageEnvelope.MessageType.class); - when(m.getMessage()).thenReturn(mockInnerMessage); - when(mockInnerMessage.getEventTime()).thenReturn(1001000000L); - when(mockState.getLatestEventTimeNs()).thenReturn(32000000000L); - assertTrue(triggerField.apply(m, mockState)); - - BiFunction>, Boolean> mockFunc = mock(BiFunction.class); - builder = TriggerBuilder.earlyTrigger(mockFunc); - triggerField = (BiFunction>, Boolean>) this.earlyTriggerField.get(builder); - assertEquals(triggerField, mockFunc); - - builder = TriggerBuilder.timeoutSinceFirstMessage(10000L); - Function>, Boolean> timerTrigger = - (Function>, Boolean>) this.timerTriggerField.get(builder); - when(mockState.getFirstMessageTimeNs()).thenReturn(0L); - assertTrue(timerTrigger.apply(mockState)); - // set the firstMessageTimeNs to 9 second earlier, giving the test 1 second to fire up the timerTrigger before assertion - when(mockState.getFirstMessageTimeNs()).thenReturn(TimeUnit.MILLISECONDS.toNanos(System.currentTimeMillis() - 9000L)); - assertFalse(timerTrigger.apply(mockState)); - - builder = TriggerBuilder.timeoutSinceLastMessage(10000L); - timerTrigger = (Function>, Boolean>) this.timerTriggerField.get(builder); - when(mockState.getLastMessageTimeNs()).thenReturn(0L); - assertTrue(timerTrigger.apply(mockState)); - // set the lastMessageTimeNs to 9 second earlier, giving the test 1 second to fire up the timerTrigger before assertion - when(mockState.getLastMessageTimeNs()).thenReturn(TimeUnit.MILLISECONDS.toNanos(System.currentTimeMillis() - 9000)); - assertFalse(timerTrigger.apply(mockState)); - } - - @Test - public void testAddTimerTriggers() throws IllegalAccessException { - TriggerBuilder> builder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000); - builder.addTimeoutSinceFirstMessage(10000L); - // exam that both earlyTrigger and timer triggers are set up - BiFunction>, Boolean> triggerField = - (BiFunction>, Boolean>) this.earlyTriggerField.get(builder); - WindowState> mockState = mock(WindowState.class); - when(mockState.getNumberMessages()).thenReturn(200L); - assertFalse(triggerField.apply(null, mockState)); - // check the timer trigger - Function>, Boolean> timerTrigger = - (Function>, Boolean>) this.timerTriggerField.get(builder); - when(mockState.getFirstMessageTimeNs()).thenReturn(0L); - assertTrue(timerTrigger.apply(mockState)); - // set the firstMessageTimeNs to 9 second earlier, giving the test 1 second to fire up the timerTrigger before assertion - when(mockState.getFirstMessageTimeNs()).thenReturn(TimeUnit.MILLISECONDS.toNanos(System.currentTimeMillis() - 9000L)); - assertFalse(timerTrigger.apply(mockState)); - - // exam that both early trigger and timer triggers are set up - builder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000); - triggerField = (BiFunction>, Boolean>) this.earlyTriggerField.get(builder); - mockState = mock(WindowState.class); - when(mockState.getNumberMessages()).thenReturn(200L); - assertFalse(triggerField.apply(null, mockState)); - builder.addTimeoutSinceLastMessage(20000L); - // check the timer trigger - timerTrigger = (Function>, Boolean>) this.timerTriggerField.get(builder); - when(mockState.getLastMessageTimeNs()).thenReturn(0L); - assertTrue(timerTrigger.apply(mockState)); - // set the firstMessageTimeNs to 9 second earlier, giving the test 1 second to fire up the timerTrigger before assertion - when(mockState.getLastMessageTimeNs()).thenReturn(TimeUnit.MILLISECONDS.toNanos(System.currentTimeMillis() - 9000L)); - assertFalse(timerTrigger.apply(mockState)); - } - - @Test - public void testAddLateTriggers() throws IllegalAccessException { - TriggerBuilder> builder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000); - builder.addLateTriggerOnSizeLimit(10000L); - // exam that both earlyTrigger and lateTriggers are set up - BiFunction>, Boolean> earlyTrigger = - (BiFunction>, Boolean>) this.earlyTriggerField.get(builder); - WindowState> mockState = mock(WindowState.class); - when(mockState.getNumberMessages()).thenReturn(200L); - assertFalse(earlyTrigger.apply(null, mockState)); - // check the late trigger - BiFunction>, Boolean> lateTrigger = - (BiFunction>, Boolean>) this.lateTriggerField.get(builder); - assertFalse(lateTrigger.apply(null, mockState)); - // set the number of messages to 10001 to trigger the late trigger - when(mockState.getNumberMessages()).thenReturn(10001L); - assertTrue(lateTrigger.apply(null, mockState)); - - builder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000); - builder.addLateTrigger((m, s) -> s.getOutputValue().size() > 0); - // exam that both earlyTrigger and lateTriggers are set up - earlyTrigger = (BiFunction>, Boolean>) this.earlyTriggerField.get(builder); - mockState = mock(WindowState.class); - when(mockState.getNumberMessages()).thenReturn(200L); - assertFalse(earlyTrigger.apply(null, mockState)); - // exam the lateTrigger - when(mockState.getOutputValue()).thenReturn(new ArrayList<>()); - lateTrigger = (BiFunction>, Boolean>) this.lateTriggerField.get(builder); - assertFalse(lateTrigger.apply(null, mockState)); - List mockList = mock(ArrayList.class); - when(mockList.size()).thenReturn(200); - when(mockState.getOutputValue()).thenReturn(mockList); - assertTrue(lateTrigger.apply(null, mockState)); - } - - @Test - public void testAddTriggerUpdater() throws IllegalAccessException { - TriggerBuilder> builder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000); - builder.onEarlyTrigger(c -> { - c.clear(); - return c; - }); - List collection = new ArrayList() { { - for (int i = 0; i < 10; i++) { - this.add(new TestMessageEnvelope(String.format("key-%d", i), "string-value", System.nanoTime())); - } - } }; - // exam that earlyTriggerUpdater is set up - Function>, WindowState>> earlyTriggerUpdater = - (Function>, WindowState>>) this.earlyTriggerUpdater.get(builder); - WindowState> mockState = mock(WindowState.class); - when(mockState.getOutputValue()).thenReturn(collection); - earlyTriggerUpdater.apply(mockState); - assertTrue(collection.isEmpty()); - - collection.add(new TestMessageEnvelope("key-to-stay", "string-to-stay", System.nanoTime())); - collection.add(new TestMessageEnvelope("key-to-remove", "string-to-remove", System.nanoTime())); - builder.onLateTrigger(c -> { - c.removeIf(t -> t.getKey().equals("key-to-remove")); - return c; - }); - // check the late trigger updater - Function>, WindowState>> lateTriggerUpdater = - (Function>, WindowState>>) this.lateTriggerUpdater.get(builder); - when(mockState.getOutputValue()).thenReturn(collection); - lateTriggerUpdater.apply(mockState); - assertTrue(collection.size() == 1); - assertFalse(collection.get(0).isDelete()); - assertEquals(collection.get(0).getKey(), "key-to-stay"); - } -} diff --git a/samza-api/src/test/java/org/apache/samza/operators/windows/TestWindows.java b/samza-api/src/test/java/org/apache/samza/operators/windows/TestWindows.java deleted file mode 100644 index 26af26e232..0000000000 --- a/samza-api/src/test/java/org/apache/samza/operators/windows/TestWindows.java +++ /dev/null @@ -1,109 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.samza.operators.windows; - -import org.apache.samza.operators.TestMessageEnvelope; -import org.junit.Test; - -import java.lang.reflect.Field; -import java.util.ArrayList; -import java.util.Collection; -import java.util.function.BiFunction; -import java.util.function.Function; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - - -public class TestWindows { - - @Test - public void testSessionWindows() throws NoSuchFieldException, IllegalAccessException { - // test constructing the default session window - Window, WindowOutput>> testWnd = Windows - .intoSessions( - TestMessageEnvelope::getKey); - assertTrue(testWnd instanceof SessionWindow); - Field wndKeyFuncField = SessionWindow.class.getDeclaredField("wndKeyFunction"); - Field aggregatorField = SessionWindow.class.getDeclaredField("aggregator"); - wndKeyFuncField.setAccessible(true); - aggregatorField.setAccessible(true); - Function wndKeyFunc = (Function) wndKeyFuncField.get(testWnd); - assertEquals(wndKeyFunc.apply(new TestMessageEnvelope("test-key", "test-value", 0)), "test-key"); - BiFunction, Collection> aggrFunc = - (BiFunction, Collection>) aggregatorField.get(testWnd); - TestMessageEnvelope mockMsg = mock(TestMessageEnvelope.class); - Collection collection = aggrFunc.apply(mockMsg, new ArrayList<>()); - assertTrue(collection.size() == 1); - assertTrue(collection.contains(mockMsg)); - - // test constructing the session window w/ customized session info - Window, WindowOutput>> testWnd2 = Windows.intoSessions( - m -> String.format("key-%d", m.getMessage().getEventTime()), m -> m.getMessage().getValue().charAt(0)); - assertTrue(testWnd2 instanceof SessionWindow); - wndKeyFunc = (Function) wndKeyFuncField.get(testWnd2); - aggrFunc = (BiFunction, Collection>) aggregatorField.get(testWnd2); - assertEquals(wndKeyFunc.apply(new TestMessageEnvelope("test-key", "test-value", 0)), "key-0"); - TestMessageEnvelope.MessageType mockInnerMessage = mock(TestMessageEnvelope.MessageType.class); - when(mockMsg.getMessage()).thenReturn(mockInnerMessage); - when(mockInnerMessage.getValue()).thenReturn("x-001"); - collection = aggrFunc.apply(mockMsg, new ArrayList<>()); - assertTrue(collection.size() == 1); - assertTrue(collection.contains('x')); - - // test constructing session window w/ a default counter - Window> testCounter = Windows.intoSessionCounter( - m -> String.format("key-%d", m.getMessage().getEventTime())); - assertTrue(testCounter instanceof SessionWindow); - wndKeyFunc = (Function) wndKeyFuncField.get(testCounter); - BiFunction counterFn = (BiFunction) aggregatorField.get(testCounter); - when(mockMsg.getMessage().getEventTime()).thenReturn(12345L); - assertEquals(wndKeyFunc.apply(mockMsg), "key-12345"); - assertEquals(counterFn.apply(mockMsg, 1), Integer.valueOf(2)); - } - - @Test - public void testSetTriggers() throws NoSuchFieldException, IllegalAccessException { - Window> testCounter = Windows.intoSessionCounter( - m -> String.format("key-%d", m.getMessage().getEventTime())); - // test session window w/ a trigger - TriggerBuilder triggerBuilder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000L); - testCounter.setTriggers(triggerBuilder); - Trigger> expectedTrigger = triggerBuilder.build(); - Trigger> actualTrigger = Windows.getInternalWindowFn(testCounter).getTrigger(); - // examine all trigger fields are expected - Field earlyTriggerField = Trigger.class.getDeclaredField("earlyTrigger"); - Field lateTriggerField = Trigger.class.getDeclaredField("lateTrigger"); - Field timerTriggerField = Trigger.class.getDeclaredField("timerTrigger"); - Field earlyTriggerUpdater = Trigger.class.getDeclaredField("earlyTriggerUpdater"); - Field lateTriggerUpdater = Trigger.class.getDeclaredField("lateTriggerUpdater"); - earlyTriggerField.setAccessible(true); - lateTriggerField.setAccessible(true); - timerTriggerField.setAccessible(true); - earlyTriggerUpdater.setAccessible(true); - lateTriggerUpdater.setAccessible(true); - assertEquals(earlyTriggerField.get(expectedTrigger), earlyTriggerField.get(actualTrigger)); - assertEquals(lateTriggerField.get(expectedTrigger), lateTriggerField.get(actualTrigger)); - assertEquals(timerTriggerField.get(expectedTrigger), timerTriggerField.get(actualTrigger)); - assertEquals(earlyTriggerUpdater.get(expectedTrigger), earlyTriggerUpdater.get(actualTrigger)); - assertEquals(lateTriggerUpdater.get(expectedTrigger), lateTriggerUpdater.get(actualTrigger)); - } -} diff --git a/samza-operator/src/main/java/org/apache/samza/operators/MessageStreamImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/MessageStreamImpl.java index 1e8e606f64..f2fd365d23 100644 --- a/samza-operator/src/main/java/org/apache/samza/operators/MessageStreamImpl.java +++ b/samza-operator/src/main/java/org/apache/samza/operators/MessageStreamImpl.java @@ -28,9 +28,8 @@ import org.apache.samza.operators.spec.OperatorSpec; import org.apache.samza.operators.spec.OperatorSpecs; import org.apache.samza.operators.windows.Window; -import org.apache.samza.operators.windows.WindowFn; import org.apache.samza.operators.windows.WindowOutput; -import org.apache.samza.operators.spec.WindowState; +import org.apache.samza.operators.windows.BaseWindow; import java.util.ArrayList; import java.util.Collection; @@ -89,9 +88,9 @@ public void sink(SinkFunction sinkFn) { } @Override - public , WM extends WindowOutput> MessageStream window( - Window window) { - OperatorSpec wndOp = OperatorSpecs.createWindowOperator((WindowFn) window.getInternalWindowFn()); + public > MessageStream window( + Window window) { + OperatorSpec wndOp = OperatorSpecs.createWindowOperator((BaseWindow) window); this.registeredOperatorSpecs.add(wndOp); return wndOp.getOutputStream(); } @@ -106,8 +105,7 @@ public , WM extends WindowOutput> Mes // TODO: need to add default store functions for the two partial join functions - ((MessageStreamImpl) otherStream).registeredOperatorSpecs.add( - OperatorSpecs.createPartialJoinOperator(parJoin2, outputStream)); + ((MessageStreamImpl) otherStream).registeredOperatorSpecs.add(OperatorSpecs.createPartialJoinOperator(parJoin2, outputStream)); this.registeredOperatorSpecs.add(OperatorSpecs.createPartialJoinOperator(parJoin1, outputStream)); return outputStream; } diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpls.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpls.java index 8d774d76cf..2c7eb55c0c 100644 --- a/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpls.java +++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpls.java @@ -22,13 +22,8 @@ import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.MessageStreamImpl; import org.apache.samza.operators.data.MessageEnvelope; -import org.apache.samza.operators.spec.OperatorSpec; -import org.apache.samza.operators.spec.PartialJoinOperatorSpec; -import org.apache.samza.operators.spec.SinkOperatorSpec; -import org.apache.samza.operators.spec.StreamOperatorSpec; -import org.apache.samza.operators.spec.WindowOperatorSpec; +import org.apache.samza.operators.spec.*; import org.apache.samza.operators.windows.WindowOutput; -import org.apache.samza.operators.spec.WindowState; import org.apache.samza.task.TaskContext; import java.util.Collection; @@ -62,7 +57,7 @@ public static RootOperatorImpl createOperatorImpls(M RootOperatorImpl rootOperator = new RootOperatorImpl<>(); // create the pipeline/topology starting from the source source.getRegisteredOperatorSpecs().forEach(registeredOperator -> { - // pass in the source and context s.t. stateful stream operators can initialize their stores + // pass in the source and context s.t. stateful stream operators can initialize their stores OperatorImpl operatorImpl = createAndRegisterOperatorImpl(registeredOperator, source, context); rootOperator.registerNextOperator(operatorImpl); @@ -115,7 +110,7 @@ public static RootOperatorImpl createOperatorImpls(M } else if (operatorSpec instanceof SinkOperatorSpec) { return new SinkOperatorImpl<>((SinkOperatorSpec) operatorSpec); } else if (operatorSpec instanceof WindowOperatorSpec) { - return new SessionWindowOperatorImpl<>((WindowOperatorSpec) operatorSpec); + return new WindowOperatorImpl<>((WindowOperatorSpec) operatorSpec); } else if (operatorSpec instanceof PartialJoinOperatorSpec) { return new PartialJoinOperatorImpl<>((PartialJoinOperatorSpec) operatorSpec); } diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/SessionWindowOperatorImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/SessionWindowOperatorImpl.java deleted file mode 100644 index 8e4691f882..0000000000 --- a/samza-operator/src/main/java/org/apache/samza/operators/impl/SessionWindowOperatorImpl.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.samza.operators.impl; - -import org.apache.samza.operators.MessageStream; -import org.apache.samza.operators.StateStoreImpl; -import org.apache.samza.operators.data.MessageEnvelope; -import org.apache.samza.operators.spec.WindowOperatorSpec; -import org.apache.samza.operators.spec.WindowState; -import org.apache.samza.operators.windows.WindowOutput; -import org.apache.samza.storage.kv.Entry; -import org.apache.samza.task.MessageCollector; -import org.apache.samza.task.TaskContext; -import org.apache.samza.task.TaskCoordinator; - - -/** - * Default implementation class of a {@link WindowOperatorSpec} for a session window. - * - * @param the type of input {@link MessageEnvelope} - * @param the type of window key - * @param the type of window state - * @param the type of aggregated value of the window - */ -class SessionWindowOperatorImpl> - extends OperatorImpl { - - private final WindowOperatorSpec windowSpec; - private StateStoreImpl stateStore = null; - - SessionWindowOperatorImpl(WindowOperatorSpec windowSpec) { - this.windowSpec = windowSpec; - } - - @Override - public void init(MessageStream source, TaskContext context) { - this.stateStore = new StateStoreImpl<>(this.windowSpec.getStoreFns(), windowSpec.getStoreName(source)); - this.stateStore.init(context); - } - - @Override - public void onNext(M message, MessageCollector collector, TaskCoordinator coordinator) { - Entry state = this.stateStore.getState(message); - this.propagateResult(this.windowSpec.getTransformFn().apply(message, state), collector, coordinator); - this.stateStore.updateState(message, state); - } - - public void onTimer(MessageCollector collector, TaskCoordinator coordinator) { - // This is to periodically check the timeout triggers to get the list of window states to be updated - } -} diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java new file mode 100644 index 0000000000..898b13206a --- /dev/null +++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.samza.operators.impl; + +import org.apache.samza.operators.data.MessageEnvelope; +import org.apache.samza.operators.spec.WindowOperatorSpec; +import org.apache.samza.operators.windows.WindowOutput; +import org.apache.samza.operators.windows.BaseWindow; +import org.apache.samza.task.MessageCollector; +import org.apache.samza.task.TaskCoordinator; + +/** + * + */ +public class WindowOperatorImpl> extends OperatorImpl { + + private final BaseWindow window; + + + public WindowOperatorImpl(WindowOperatorSpec spec) { + window = spec.getWindow(); + } + + @Override + public void onNext(M message, MessageCollector collector, TaskCoordinator coordinator) { + + } +} diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/ExperimentalWindowOperatorSpec.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/ExperimentalWindowOperatorSpec.java deleted file mode 100644 index 1a331fa013..0000000000 --- a/samza-operator/src/main/java/org/apache/samza/operators/spec/ExperimentalWindowOperatorSpec.java +++ /dev/null @@ -1,42 +0,0 @@ -package org.apache.samza.operators.spec; - -import org.apache.samza.operators.MessageStream; -import org.apache.samza.operators.MessageStreamImpl; -import org.apache.samza.operators.data.MessageEnvelope; -import org.apache.samza.operators.windows.WindowOutput; -import org.apache.samza.operators.windows.experimental.WindowDefinition; - -/** - * - */ -public class ExperimentalWindowOperatorSpec> implements OperatorSpec{ - - private final WindowDefinition window; - - /** - * The output {@link MessageStream}. - */ - private final MessageStreamImpl outputStream; - - private final String operatorId; - - - public ExperimentalWindowOperatorSpec(WindowDefinition window, String operatorId) { - this.window = window; - this.outputStream = new MessageStreamImpl<>(); - this.operatorId = operatorId; - } - - @Override - public MessageStream getOutputStream() { - return this.outputStream; - } - - public WindowDefinition getWindow() { - return window; - } - - public String getOperatorId() { - return operatorId; - } -} diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java index 04282a44a1..ca596f4316 100644 --- a/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java +++ b/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java @@ -23,9 +23,8 @@ import org.apache.samza.operators.functions.FlatMapFunction; import org.apache.samza.operators.functions.SinkFunction; import org.apache.samza.operators.MessageStreamImpl; -import org.apache.samza.operators.windows.WindowFn; import org.apache.samza.operators.windows.WindowOutput; -import org.apache.samza.operators.windows.experimental.WindowDefinition; +import org.apache.samza.operators.windows.BaseWindow; import java.util.ArrayList; import java.util.UUID; @@ -68,34 +67,21 @@ public static SinkOperatorSpec createSinkOperator return new SinkOperatorSpec<>(sinkFn); } - /** - * Creates a {@link WindowOperatorSpec}. - * - * @param windowFn the {@link WindowFn} function - * @param type of input {@link MessageEnvelope} - * @param type of window key - * @param type of {@link WindowState} - * @param type of output {@link WindowOutput} - * @return the {@link WindowOperatorSpec} - */ - public static > WindowOperatorSpec createWindowOperator( - WindowFn windowFn) { - return new WindowOperatorSpec<>(windowFn, OperatorSpecs.getOperatorId()); - } /** * Creates a {@link WindowOperatorSpec}. * - * @param window the {@link WindowDefinition} for the window. + * @param window the {@link BaseWindow} for the window. * @param type of input {@link MessageEnvelope} - * @param type of window key + * @param type of the key on which the window is computed on + * @param type of key in the {@link WindowOutput} * @param type of value in the window. * @param type of output {@link WindowOutput} * @return the {@link WindowOperatorSpec} */ - public static > ExperimentalWindowOperatorSpec createWindowOperator(WindowDefinition window) { - return new ExperimentalWindowOperatorSpec<>(window, OperatorSpecs.getOperatorId()); + public static > WindowOperatorSpec createWindowOperator(BaseWindow window) { + return new WindowOperatorSpec<>(window, OperatorSpecs.getOperatorId()); } diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/PartialJoinOperatorSpec.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/PartialJoinOperatorSpec.java index 7e8dbc158c..e6d77f665a 100644 --- a/samza-operator/src/main/java/org/apache/samza/operators/spec/PartialJoinOperatorSpec.java +++ b/samza-operator/src/main/java/org/apache/samza/operators/spec/PartialJoinOperatorSpec.java @@ -45,17 +45,6 @@ public class PartialJoinOperatorSpec, K, JM exte */ private final BiFunction transformFn; - /** - * The {@link MessageEnvelope} store functions that read the buffered {@link MessageEnvelope}s from the other - * stream in the join. - */ - private final StoreFunctions joinStoreFns; - - /** - * The {@link MessageEnvelope} store functions that save the buffered {@link MessageEnvelope} of this - * {@link MessageStreamImpl} in the join. - */ - private final StoreFunctions selfStoreFns; /** * The unique ID for this operator. @@ -72,10 +61,6 @@ public class PartialJoinOperatorSpec, K, JM exte PartialJoinOperatorSpec(BiFunction partialJoinFn, MessageStreamImpl joinOutput, String operatorId) { this.joinOutput = joinOutput; this.transformFn = partialJoinFn; - // Read-only join store, no creator/updater functions required. - this.joinStoreFns = new StoreFunctions<>(m -> m.getKey(), null); - // Buffered message envelope store for this input stream. - this.selfStoreFns = new StoreFunctions<>(m -> m.getKey(), (m, s1) -> m); this.operatorId = operatorId; } @@ -89,14 +74,6 @@ public MessageStreamImpl getOutputStream() { return this.joinOutput; } - public StoreFunctions getJoinStoreFns() { - return this.joinStoreFns; - } - - public StoreFunctions getSelfStoreFns() { - return this.selfStoreFns; - } - public BiFunction getTransformFn() { return this.transformFn; } diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/StoreFunctions.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/StoreFunctions.java deleted file mode 100644 index b8dc516a91..0000000000 --- a/samza-operator/src/main/java/org/apache/samza/operators/spec/StoreFunctions.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.samza.operators.spec; - -import org.apache.samza.operators.data.MessageEnvelope; - -import java.util.function.BiFunction; -import java.util.function.Function; - -/** - * The store functions that are used by window and partial join operators to store and retrieve buffered {@link MessageEnvelope}s - * and partial aggregation results. - * - * @param the type of key used to store the operator state - * @param the type of operator state. E.g. could be the partial aggregation result for a window, or a buffered - * input {@link MessageEnvelope} from the join stream for a join - */ -public class StoreFunctions { - /** - * Function that returns the key to query in the operator state store for a particular {@link MessageEnvelope}. - * This 1:1 function only returns a single key for the incoming {@link MessageEnvelope}. This is sufficient to support - * non-overlapping windows and unique-key based joins. - * - * TODO: for windows that overlaps (i.e. sliding windows and hopping windows) and non-unique-key-based join, - * the query to the state store is usually a range scan. We need to add a rangeKeyFinder function - * (or make this function return a collection) to map from a single input {@link MessageEnvelope} to a range of keys in the store. - */ - private final Function storeKeyFn; - - /** - * Function to update the store entry based on the current operator state and the incoming {@link MessageEnvelope}. - * - * TODO: this is assuming a 1:1 mapping from the input {@link MessageEnvelope} to the store entry. When implementing sliding/hopping - * windows and non-unique-key-based join, we may need to include the corresponding state key in addition to the - * state value. Alternatively this can be called once for each store key for the {@link MessageEnvelope}. - */ - private final BiFunction stateUpdaterFn; - - public StoreFunctions(Function storeKeyFn, BiFunction stateUpdaterFn) { - this.storeKeyFn = storeKeyFn; - this.stateUpdaterFn = stateUpdaterFn; - } - - public Function getStoreKeyFn() { - return this.storeKeyFn; - } - - public BiFunction getStateUpdaterFn() { - return this.stateUpdaterFn; - } -} diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java index 30d2f52cda..c742a0d150 100644 --- a/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java +++ b/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java @@ -16,102 +16,46 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.samza.operators.spec; import org.apache.samza.operators.MessageStream; -import org.apache.samza.operators.data.MessageEnvelope; import org.apache.samza.operators.MessageStreamImpl; -import org.apache.samza.operators.windows.Trigger; -import org.apache.samza.operators.windows.WindowFn; +import org.apache.samza.operators.data.MessageEnvelope; import org.apache.samza.operators.windows.WindowOutput; -import org.apache.samza.storage.kv.Entry; - -import java.util.function.BiFunction; - +import org.apache.samza.operators.windows.BaseWindow; /** - * Defines a window operator that takes one {@link MessageStreamImpl} as an input, accumulates the window state, - * and generates an output {@link MessageStreamImpl} with output type {@code WM} which extends {@link WindowOutput} * - * @param the type of input {@link MessageEnvelope} - * @param the type of key in the output {@link MessageEnvelope} from the {@link WindowOperatorSpec} function - * @param the type of window state in the {@link WindowOperatorSpec} function - * @param the type of window output {@link MessageEnvelope} */ -public class WindowOperatorSpec> implements - OperatorSpec { +public class WindowOperatorSpec> implements OperatorSpec { + + private final BaseWindow window; /** * The output {@link MessageStream}. */ private final MessageStreamImpl outputStream; - /** - * The window transformation function that takes {@link MessageEnvelope}s from one input stream, aggregates with the window - * state(s) from the window state store, and generate output {@link MessageEnvelope}s for the output stream. - */ - private final BiFunction, WM> transformFn; - - /** - * The state store functions for the {@link WindowOperatorSpec}. - */ - private final StoreFunctions storeFns; - - /** - * The window trigger. - */ - private final Trigger trigger; - - /** - * The unique ID of this operator. - */ private final String operatorId; - /** - * Constructor for {@link WindowOperatorSpec}. - * - * @param windowFn the window function - * @param operatorId auto-generated unique ID of this operator - */ - WindowOperatorSpec(WindowFn windowFn, String operatorId) { + + public WindowOperatorSpec(BaseWindow window, String operatorId) { + this.window = window; this.outputStream = new MessageStreamImpl<>(); - this.transformFn = windowFn.getTransformFn(); - this.storeFns = windowFn.getStoreFns(); - this.trigger = windowFn.getTrigger(); this.operatorId = operatorId; } @Override - public String toString() { - return this.operatorId; - } - - @Override - public MessageStreamImpl getOutputStream() { + public MessageStream getOutputStream() { return this.outputStream; } - public StoreFunctions getStoreFns() { - return this.storeFns; - } - - public BiFunction, WM> getTransformFn() { - return this.transformFn; + public BaseWindow getWindow() { + return window; } - public Trigger getTrigger() { - return this.trigger; - } - - /** - * Method to generate the window operator's state store name - * TODO HIGH pmaheshw: should this be here? - * - * @param inputStream the input {@link MessageStreamImpl} to this state store - * @return the persistent store name of the window operator - */ - public String getStoreName(MessageStream inputStream) { - //TODO: need to get the persistent name of ds and the operator in a serialized form - return String.format("input-%s-wndop-%s", inputStream.toString(), this.toString()); + public String getOperatorId() { + return operatorId; } } diff --git a/samza-operator/src/test/java/org/apache/samza/operators/BroadcastTask.java b/samza-operator/src/test/java/org/apache/samza/operators/BroadcastTask.java index e45d068eae..753c950367 100644 --- a/samza-operator/src/test/java/org/apache/samza/operators/BroadcastTask.java +++ b/samza-operator/src/test/java/org/apache/samza/operators/BroadcastTask.java @@ -23,12 +23,13 @@ import org.apache.samza.operators.data.IncomingSystemMessageEnvelope; import org.apache.samza.operators.data.JsonIncomingSystemMessageEnvelope; import org.apache.samza.operators.data.Offset; -import org.apache.samza.operators.windows.TriggerBuilder; +import org.apache.samza.operators.windows.Time; +import org.apache.samza.operators.windows.TriggersBuilder; import org.apache.samza.operators.windows.Windows; import org.apache.samza.system.SystemStreamPartition; -import java.util.Collection; import java.util.Map; +import java.util.function.BiFunction; /** @@ -57,27 +58,28 @@ class JsonMessageEnvelope extends JsonIncomingSystemMessageEnvelope @Override public void transform(Map> messageStreams) { + BiFunction sumAggregator = (m, c) -> c + 1; messageStreams.values().forEach(entry -> { MessageStream inputStream = entry.map(this::getInputMessage); - inputStream.filter(this::myFilter1). - window(Windows.intoSessionCounter( - m -> String.format("%s-%s", m.getMessage().field1, m.getMessage().field2)). - setTriggers(TriggerBuilder.earlyTriggerWhenExceedWndLen(100). - addLateTriggerOnSizeLimit(10). - addTimeoutSinceLastMessage(30000))); - - inputStream.filter(this::myFilter2). - window(Windows.intoSessions( - m -> String.format("%s-%s", m.getMessage().field3, m.getMessage().field4)). - setTriggers(TriggerBuilder.>earlyTriggerWhenExceedWndLen(100). - addTimeoutSinceLastMessage(30000))); - - inputStream.filter(this::myFilter3). - window(Windows.intoSessions( - m -> String.format("%s-%s", m.getMessage().field3, m.getMessage().field4), m -> m.getMessage()). - setTriggers(TriggerBuilder.>earlyTriggerOnEventTime(m -> m.getMessage().getTimestamp(), 30000). - addTimeoutSinceFirstMessage(60000))); + inputStream.filter(this::myFilter1) + .window(Windows.sessionWindow(Time.milliseconds(100), sumAggregator) + .setTriggers(new TriggersBuilder<>().withEarlyFiringsAfterCountAtleast(30000) + .withLateFiringsEvery(Time.milliseconds(10)) + .build())); + + inputStream.filter(this::myFilter1) + .window(Windows.sessionWindow(Time.milliseconds(100), sumAggregator) + .setTriggers(new TriggersBuilder<>().withEarlyFiringsAfterCountAtleast(30000) + .withLateFiringsEvery(Time.milliseconds(10)) + .build())); + + inputStream.filter(this::myFilter1) + .window(Windows.sessionWindow(Time.milliseconds(100), sumAggregator) + .setTriggers(new TriggersBuilder<>().withEarlyFiringsAfterCountAtleast(30000) + .withLateFiringsEvery(Time.milliseconds(10)) + .build())); + }); } diff --git a/samza-operator/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java b/samza-operator/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java index 2ad6461716..8ca237eb55 100644 --- a/samza-operator/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java +++ b/samza-operator/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java @@ -27,10 +27,6 @@ import org.apache.samza.operators.spec.PartialJoinOperatorSpec; import org.apache.samza.operators.spec.SinkOperatorSpec; import org.apache.samza.operators.spec.StreamOperatorSpec; -import org.apache.samza.operators.spec.WindowOperatorSpec; -import org.apache.samza.operators.windows.SessionWindow; -import org.apache.samza.operators.windows.WindowFn; -import org.apache.samza.operators.windows.WindowOutput; import org.apache.samza.system.OutgoingMessageEnvelope; import org.apache.samza.system.SystemStream; import org.apache.samza.task.TaskCoordinator; @@ -44,7 +40,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; -import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -136,6 +131,7 @@ public void testSink() { assertNull(((SinkOperatorSpec) sinkOp).getOutputStream()); } + /* @Test public void testWindow() { MessageStreamImpl inputStream = new MessageStreamImpl<>(); @@ -147,7 +143,7 @@ public void testWindow() { OperatorSpec wndOp = subs.iterator().next(); assertTrue(wndOp instanceof WindowOperatorSpec); assertEquals(((WindowOperatorSpec) wndOp).getOutputStream(), outStream); - } + }*/ @Test public void testJoin() { diff --git a/samza-operator/src/test/java/org/apache/samza/operators/TestStateStoreImpl.java b/samza-operator/src/test/java/org/apache/samza/operators/TestStateStoreImpl.java deleted file mode 100644 index 3137a13458..0000000000 --- a/samza-operator/src/test/java/org/apache/samza/operators/TestStateStoreImpl.java +++ /dev/null @@ -1,72 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.samza.operators; - -import org.apache.samza.operators.spec.StoreFunctions; -import org.apache.samza.operators.spec.WindowState; -import org.apache.samza.storage.kv.Entry; -import org.apache.samza.storage.kv.KeyValueStore; -import org.apache.samza.task.TaskContext; -import org.junit.Test; - -import java.util.function.BiFunction; -import java.util.function.Function; - -import static org.junit.Assert.assertEquals; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - - -public class TestStateStoreImpl { - @Test - public void testStateStoreImpl() { - StoreFunctions mockStoreFunctions = mock(StoreFunctions.class); - // test constructor - StateStoreImpl storeImpl = new StateStoreImpl<>(mockStoreFunctions, "myStoreName"); - TaskContext mockContext = mock(TaskContext.class); - KeyValueStore mockKvStore = mock(KeyValueStore.class); - when(mockContext.getStore("myStoreName")).thenReturn(mockKvStore); - // test init() - storeImpl.init(mockContext); - verify(mockContext, times(1)).getStore("myStoreName"); - Function wndKeyFn = mock(Function.class); - when(mockStoreFunctions.getStoreKeyFn()).thenReturn(wndKeyFn); - TestMessageEnvelope mockMsg = mock(TestMessageEnvelope.class); - when(wndKeyFn.apply(mockMsg)).thenReturn("myKey"); - WindowState mockState = mock(WindowState.class); - when(mockKvStore.get("myKey")).thenReturn(mockState); - // test getState() - Entry storeEntry = storeImpl.getState(mockMsg); - assertEquals(storeEntry.getKey(), "myKey"); - assertEquals(storeEntry.getValue(), mockState); - verify(wndKeyFn, times(1)).apply(mockMsg); - verify(mockKvStore, times(1)).get("myKey"); - Entry oldEntry = new Entry<>("myKey", mockState); - WindowState mockNewState = mock(WindowState.class); - BiFunction mockUpdaterFn = mock(BiFunction.class); - when(mockStoreFunctions.getStateUpdaterFn()).thenReturn(mockUpdaterFn); - when(mockUpdaterFn.apply(mockMsg, mockState)).thenReturn(mockNewState); - // test updateState() - Entry newEntry = storeImpl.updateState(mockMsg, oldEntry); - assertEquals(newEntry.getKey(), "myKey"); - assertEquals(newEntry.getValue(), mockNewState); - } -} diff --git a/samza-operator/src/test/java/org/apache/samza/operators/WindowTask.java b/samza-operator/src/test/java/org/apache/samza/operators/WindowTask.java index f33510e52b..be4d13b8da 100644 --- a/samza-operator/src/test/java/org/apache/samza/operators/WindowTask.java +++ b/samza-operator/src/test/java/org/apache/samza/operators/WindowTask.java @@ -23,11 +23,12 @@ import org.apache.samza.operators.data.JsonIncomingSystemMessageEnvelope; import org.apache.samza.operators.data.MessageEnvelope; import org.apache.samza.operators.data.Offset; -import org.apache.samza.operators.windows.TriggerBuilder; +import org.apache.samza.operators.windows.Time; import org.apache.samza.operators.windows.Windows; import org.apache.samza.system.SystemStreamPartition; import java.util.Map; +import java.util.function.BiFunction; /** @@ -35,6 +36,7 @@ * */ public class WindowTask implements StreamOperatorTask { + class MessageType { String field1; String field2; @@ -48,6 +50,9 @@ class JsonMessageEnvelope extends JsonIncomingSystemMessageEnvelope } @Override public void transform(Map> messageStreams) { + + BiFunction maxAggregator = (m, c) -> c + 1; + messageStreams.values().forEach(source -> source.map(m1 -> new JsonMessageEnvelope( @@ -55,11 +60,8 @@ class JsonMessageEnvelope extends JsonIncomingSystemMessageEnvelope (MessageType) m1.getMessage(), m1.getOffset(), m1.getSystemStreamPartition())). - window( - Windows.intoSessionCounter( - m -> String.format("%s-%s", m.getMessage().field1, m.getMessage().field2)). - setTriggers(TriggerBuilder.earlyTriggerWhenExceedWndLen(100). - addTimeoutSinceLastMessage(30000))) + window(Windows.sessionWindow(Time.milliseconds(200), maxAggregator) + ) ); } diff --git a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpls.java b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpls.java index 1d0d547e93..c55acf96bd 100644 --- a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpls.java +++ b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpls.java @@ -24,10 +24,11 @@ import org.apache.samza.operators.data.MessageEnvelope; import org.apache.samza.operators.functions.FlatMapFunction; import org.apache.samza.operators.functions.SinkFunction; +import org.apache.samza.operators.spec.WindowOperatorSpec; import org.apache.samza.operators.spec.PartialJoinOperatorSpec; import org.apache.samza.operators.spec.SinkOperatorSpec; import org.apache.samza.operators.spec.StreamOperatorSpec; -import org.apache.samza.operators.spec.WindowOperatorSpec; +import org.apache.samza.operators.windows.Time; import org.apache.samza.operators.windows.Windows; import org.apache.samza.task.TaskContext; import org.junit.Before; @@ -60,11 +61,7 @@ public void testCreateOperator() throws NoSuchFieldException, IllegalAccessExcep // get window operator WindowOperatorSpec mockWnd = mock(WindowOperatorSpec.class); OperatorImpl opImpl = OperatorImpls.createOperatorImpl(mockWnd); - assertTrue(opImpl instanceof SessionWindowOperatorImpl); - Field sessWndField = SessionWindowOperatorImpl.class.getDeclaredField("windowSpec"); - sessWndField.setAccessible(true); - WindowOperatorSpec sessWnd = (WindowOperatorSpec) sessWndField.get(opImpl); - assertEquals(sessWnd, mockWnd); + assertTrue(opImpl instanceof WindowOperatorImpl); // get simple operator StreamOperatorSpec mockSimpleOp = mock(StreamOperatorSpec.class); @@ -109,7 +106,7 @@ public void testLinearChain() throws IllegalAccessException { // test creation of linear chain MessageStreamImpl testInput = new MessageStreamImpl<>(); TaskContext mockContext = mock(TaskContext.class); - testInput.map(m -> m).window(Windows.intoSessionCounter(TestMessageEnvelope::getKey)); + testInput.map(m -> m).window(Windows.sessionWindow(Time.milliseconds(1000))); RootOperatorImpl operatorChain = OperatorImpls.createOperatorImpls(testInput, mockContext); Set subsSet = (Set) nextOperatorsField.get(operatorChain); assertEquals(subsSet.size(), 1); diff --git a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSessionWindowImpl.java b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSessionWindowImpl.java deleted file mode 100644 index 9e45c1f960..0000000000 --- a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSessionWindowImpl.java +++ /dev/null @@ -1,111 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.samza.operators.impl; - -import org.apache.samza.operators.MessageStreamImpl; -import org.apache.samza.operators.TestMessageEnvelope; -import org.apache.samza.operators.spec.WindowOperatorSpec; -import org.apache.samza.operators.spec.StoreFunctions; -import org.apache.samza.operators.windows.WindowOutput; -import org.apache.samza.operators.spec.WindowState; -import org.apache.samza.storage.kv.Entry; -import org.apache.samza.storage.kv.KeyValueStore; -import org.apache.samza.task.MessageCollector; -import org.apache.samza.task.TaskContext; -import org.apache.samza.task.TaskCoordinator; -import org.junit.Before; -import org.junit.Test; -import org.mockito.ArgumentMatcher; - -import java.lang.reflect.Field; -import java.util.function.BiFunction; -import java.util.function.Function; - -import static org.junit.Assert.assertEquals; -import static org.mockito.Mockito.argThat; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - - -public class TestSessionWindowImpl { - Field wndStoreField = null; - Field sessWndField = null; - - @Before public void prep() throws NoSuchFieldException { - wndStoreField = SessionWindowOperatorImpl.class.getDeclaredField("stateStore"); - sessWndField = SessionWindowOperatorImpl.class.getDeclaredField("windowSpec"); - wndStoreField.setAccessible(true); - sessWndField.setAccessible(true); - } - - @Test - public void testConstructor() throws IllegalAccessException, NoSuchFieldException { - // test constructing a SessionWindowOperatorImpl w/ expected mock functions - WindowOperatorSpec, WindowOutput> wndOp = mock(WindowOperatorSpec.class); - SessionWindowOperatorImpl, WindowOutput> sessWnd = new SessionWindowOperatorImpl<>(wndOp); - assertEquals(wndOp, sessWndField.get(sessWnd)); - } - - @Test - public void testInitAndProcess() throws IllegalAccessException { - WindowOperatorSpec, WindowOutput> wndOp = mock(WindowOperatorSpec.class); - BiFunction>, WindowOutput> mockTxfmFn = mock(BiFunction.class); - SessionWindowOperatorImpl, WindowOutput> sessWnd = new SessionWindowOperatorImpl<>(wndOp); - - // construct and init the SessionWindowOperatorImpl object - MessageStreamImpl mockInputStrm = mock(MessageStreamImpl.class); - StoreFunctions> mockStoreFns = mock(StoreFunctions.class); - Function wndKeyFn = m -> "test-msg-key"; - when(mockStoreFns.getStoreKeyFn()).thenReturn(wndKeyFn); - when(wndOp.getStoreFns()).thenReturn(mockStoreFns); - when(wndOp.getStoreName(mockInputStrm)).thenReturn("test-wnd-store"); - when(wndOp.getTransformFn()).thenReturn(mockTxfmFn); - TaskContext mockContext = mock(TaskContext.class); - KeyValueStore> mockKvStore = mock(KeyValueStore.class); - when(mockContext.getStore("test-wnd-store")).thenReturn(mockKvStore); - sessWnd.init(mockInputStrm, mockContext); - - // test onNext() method. Make sure the transformation function and the state update functions are invoked. - TestMessageEnvelope mockMsg = mock(TestMessageEnvelope.class); - MessageCollector mockCollector = mock(MessageCollector.class); - TaskCoordinator mockCoordinator = mock(TaskCoordinator.class); - BiFunction, WindowState> stateUpdaterFn = mock(BiFunction.class); - when(mockStoreFns.getStateUpdaterFn()).thenReturn(stateUpdaterFn); - WindowState mockNewState = mock(WindowState.class); - WindowState oldState = mock(WindowState.class); - when(mockKvStore.get("test-msg-key")).thenReturn(oldState); - when(stateUpdaterFn.apply(mockMsg, oldState)).thenReturn(mockNewState); - sessWnd.onNext(mockMsg, mockCollector, mockCoordinator); - verify(mockTxfmFn, times(1)).apply(argThat(new ArgumentMatcher() { - @Override public boolean matches(Object argument) { - TestMessageEnvelope xIn = (TestMessageEnvelope) argument; - return xIn.equals(mockMsg); - } - }), argThat(new ArgumentMatcher>>() { - @Override public boolean matches(Object argument) { - Entry> xIn = (Entry>) argument; - return xIn.getKey().equals("test-msg-key") && xIn.getValue().equals(oldState); - } - })); - verify(stateUpdaterFn, times(1)).apply(mockMsg, oldState); - verify(mockKvStore, times(1)).put("test-msg-key", mockNewState); - } -} diff --git a/samza-operator/src/test/java/org/apache/samza/operators/spec/TestOperatorSpecs.java b/samza-operator/src/test/java/org/apache/samza/operators/spec/TestOperatorSpecs.java index f7449efd55..61b660c0c5 100644 --- a/samza-operator/src/test/java/org/apache/samza/operators/spec/TestOperatorSpecs.java +++ b/samza-operator/src/test/java/org/apache/samza/operators/spec/TestOperatorSpecs.java @@ -23,10 +23,6 @@ import org.apache.samza.operators.functions.FlatMapFunction; import org.apache.samza.operators.functions.SinkFunction; import org.apache.samza.operators.MessageStreamImpl; -import org.apache.samza.operators.windows.Trigger; -import org.apache.samza.operators.windows.WindowFn; -import org.apache.samza.operators.windows.WindowOutput; -import org.apache.samza.storage.kv.Entry; import org.junit.Test; import java.util.ArrayList; @@ -35,10 +31,8 @@ import java.util.function.Function; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; public class TestOperatorSpecs { @@ -60,6 +54,7 @@ public void testGetSinkOperator() { assertTrue(sinkOp.getOutputStream() == null); } + /* @Test public void testGetWindowOperator() { WindowFn, WindowOutput> windowFn = mock(WindowFn.class); @@ -78,7 +73,7 @@ public void testGetWindowOperator() { assertEquals(windowOp.getStoreFns(), storeFns); assertEquals(windowOp.getTrigger(), trigger); assertEquals(windowOp.getStoreName(mockInput), String.format("input-mockStream1-wndop-%s", windowOp.toString())); - } + }*/ @Test public void testGetPartialJoinOperator() { @@ -92,10 +87,6 @@ public void testGetPartialJoinOperator() { MessageEnvelope m = mock(MessageEnvelope.class); MessageEnvelope s = mock(MessageEnvelope.class); assertEquals(partialJoin.getTransformFn(), merger); - assertEquals(partialJoin.getSelfStoreFns().getStoreKeyFn().apply(m), m.getKey()); - assertEquals(partialJoin.getSelfStoreFns().getStateUpdaterFn().apply(m, s), m); - assertEquals(partialJoin.getJoinStoreFns().getStoreKeyFn().apply(m), m.getKey()); - assertNull(partialJoin.getJoinStoreFns().getStateUpdaterFn()); } @Test From 805a423a3b37213c3ebdee42eb8e9653b67e1ef0 Mon Sep 17 00:00:00 2001 From: vjagadish1989 Date: Wed, 7 Dec 2016 17:34:29 -0800 Subject: [PATCH 6/6] add javadocs. fix tests --- .../samza/operators/windows/Window.java | 6 +- .../windows/examples/WindowingExamples.java | 125 ------------------ .../operators/spec/TestOperatorSpecs.java | 30 ++--- 3 files changed, 15 insertions(+), 146 deletions(-) delete mode 100644 samza-api/src/main/java/org/apache/samza/operators/windows/examples/WindowingExamples.java diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/Window.java b/samza-api/src/main/java/org/apache/samza/operators/windows/Window.java index 151db48028..4286c73b90 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/windows/Window.java +++ b/samza-api/src/main/java/org/apache/samza/operators/windows/Window.java @@ -22,8 +22,8 @@ import org.apache.samza.operators.data.MessageEnvelope; /** - * A {@link Window} transform slices a {@link org.apache.samza.operators.MessageStream} into smaller finite chunks for - * further processing. Programmers should use the API methods of {@link Windows} to specify their windowing functions. + * A {@link Window} transform slices a stream into smaller finite chunks for further processing. Programmers should + * use the API methods of {@link Windows} to specify their windowing functions. * *

There are the following aspects to windowing in Samza: * @@ -41,7 +41,7 @@ * * * @param type of input {@link MessageEnvelope}. - * @param type of key to group by when evaluating a {@link Window}. + * @param type of key in the {@link MessageEnvelope} on which the window is computed on. * @param type of key in the {@link Window} output. * @param type of value stored in the {@link Window}. * @param type of the {@link Window} result. diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/examples/WindowingExamples.java b/samza-api/src/main/java/org/apache/samza/operators/windows/examples/WindowingExamples.java deleted file mode 100644 index 2372af414f..0000000000 --- a/samza-api/src/main/java/org/apache/samza/operators/windows/examples/WindowingExamples.java +++ /dev/null @@ -1,125 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.samza.operators.windows.examples; - -import org.apache.samza.operators.MessageStream; -import org.apache.samza.operators.data.IncomingSystemMessageEnvelope; -import org.apache.samza.operators.data.MessageEnvelope; -import org.apache.samza.operators.windows.*; - -import java.util.Collection; -import java.util.function.BiFunction; -import java.util.function.Function; - -/** - * Examples for programming using the {@link Windows} APIs. - */ -public class WindowingExamples { - - private static final Time INTERVAL_MS = Time.milliseconds(1000); - private static final Time SESSION_GAP_MS = Time.milliseconds(2000); - - private static Integer parseInt(IncomingSystemMessageEnvelope msg) { - //parse an integer from the message - return 0; - } - - private static PercentileMessage compute99Percentile(Collection msg) { - //iterate over values, sort them, and compute percentiles. - return null; - } - - private static class PercentileMessage implements MessageEnvelope { - - @Override - public String getKey() { - return null; - } - - @Override - public Integer getMessage() { - return null; - } - - } - - public static void main(String[] args) { - - MessageStream integerStream = null; - - //GLOBAL TUMBLING WINDOWS WITH CUSTOM AGGREGATES: Demonstrate a custom aggregation (MAX) that returns the max element in a {@link MessageStream} - //Notes: - Aggregations are done with a tumbling window of 1000ms. - // - Early results are emitted once every 50 messages, every 4000ms in processing time. - // - Late results are emitted only when 20 late messages have accumulated. - // - Previously fired window panes are discarded - // - Messages have a maximum allowed lateness of 50000ms. - - BiFunction maxAggregator = (m, c)-> Math.max(parseInt(m), c); - MessageStream, Integer>> window1 = integerStream.window(Windows.tumblingWindow(INTERVAL_MS, maxAggregator) - .setTriggers(new TriggersBuilder() - .withEarlyFiringsAfterCountAtleast(50) - .withEarlyFiringsEvery(Time.seconds(4)) - .withLateFiringsAfterCountAtleast(20) - .discardFiredPanes() - .build())); - - - //KEYED SESSION WINDOW WITH PERCENTILE COMPUTATION: Demonstrate percentile computation over a Keyed Session Window on a {@link MessageStream} - //Notes: - Aggregations are done with session windows having gap SESSION_GAP_MS seconds. - // - Early results are emitted once every 50 messages, every 4000ms in processing time. - // - Late results are emitted only when 20 late messages have accumulated. - // - Previously fired window panes are accumulated - // - Messages have a maximum allowed lateness of 50000ms. - Function keyExtractor = null; - - final MessageStream windowedPercentiles = integerStream.window(Windows.keyedSessionWindow(keyExtractor, SESSION_GAP_MS) - .setTriggers(new TriggersBuilder() - .withEarlyFiringsAfterCountAtleast(50) - .withEarlyFiringsEvery(Time.seconds(4)) - .withLateFiringsAfterCountAtleast(20) - .accumulateFiredPanes() - .build())) - .map(dataset -> compute99Percentile(dataset.getMessage())); - - //KEYED TUMBLING WINDOW WITH PERCENTILE COMPUTATION: Demonstrate percentile computation over a Keyed Tumbling Window on a {@link MessageStream} - final MessageStream tumblingPercentiles = integerStream.window(Windows.keyedTumblingWindow(keyExtractor, INTERVAL_MS) - .setTriggers(new TriggersBuilder() - .withEarlyFiringsAfterCountAtleast(50) - .accumulateFiredPanes() - .build())) - .map(dataset -> compute99Percentile(dataset.getMessage())); - - //KEYED TUMBLING WINDOW - MessageStream, Collection>> tumblingWindow = integerStream.window(Windows.keyedTumblingWindow(keyExtractor, INTERVAL_MS) - .setTriggers(new TriggersBuilder() - .withEarlyFiringsAfterCountAtleast(50) - .discardFiredPanes() - .build())); - - - /* - //A CUSTOM GLOBAL WINDOW: Demonstrates a window with custom triggering every 500 messages - final MessageStream>> customWindow = integerStream.window(Windows.customGlobalWindow().setTriggers(new TriggerSpecBuilder() - .withEarlyFiringsAfterCountAtleast(500) - .discardFiredPanes() - .build())); - */ - } -} diff --git a/samza-operator/src/test/java/org/apache/samza/operators/spec/TestOperatorSpecs.java b/samza-operator/src/test/java/org/apache/samza/operators/spec/TestOperatorSpecs.java index 61b660c0c5..88f6680839 100644 --- a/samza-operator/src/test/java/org/apache/samza/operators/spec/TestOperatorSpecs.java +++ b/samza-operator/src/test/java/org/apache/samza/operators/spec/TestOperatorSpecs.java @@ -23,6 +23,9 @@ import org.apache.samza.operators.functions.FlatMapFunction; import org.apache.samza.operators.functions.SinkFunction; import org.apache.samza.operators.MessageStreamImpl; +import org.apache.samza.operators.windows.BaseWindow; +import org.apache.samza.operators.windows.WindowKey; +import org.apache.samza.operators.windows.WindowOutput; import org.junit.Test; import java.util.ArrayList; @@ -54,26 +57,17 @@ public void testGetSinkOperator() { assertTrue(sinkOp.getOutputStream() == null); } - /* @Test public void testGetWindowOperator() { - WindowFn, WindowOutput> windowFn = mock(WindowFn.class); - BiFunction>, WindowOutput> xFunction = (m, e) -> null; - StoreFunctions> storeFns = mock(StoreFunctions.class); - Trigger> trigger = mock(Trigger.class); - MessageStreamImpl mockInput = mock(MessageStreamImpl.class); - when(windowFn.getTransformFn()).thenReturn(xFunction); - when(windowFn.getStoreFns()).thenReturn(storeFns); - when(windowFn.getTrigger()).thenReturn(trigger); - when(mockInput.toString()).thenReturn("mockStream1"); - - WindowOperatorSpec, WindowOutput> windowOp = OperatorSpecs - .createWindowOperator(windowFn); - assertEquals(windowOp.getTransformFn(), xFunction); - assertEquals(windowOp.getStoreFns(), storeFns); - assertEquals(windowOp.getTrigger(), trigger); - assertEquals(windowOp.getStoreName(mockInput), String.format("input-mockStream1-wndop-%s", windowOp.toString())); - }*/ + Function keyExtractor = m -> "globalkey"; + BiFunction aggregator = (m, c) -> c + 1; + BaseWindow window = new BaseWindow<>(keyExtractor, aggregator, null, null); + WindowOperatorSpec spec = OperatorSpecs., Integer, + WindowOutput, Integer>>createWindowOperator(window); + assertEquals(spec.getWindow(), window); + assertEquals(spec.getWindow().getKeyExtractor(), keyExtractor); + assertEquals(spec.getWindow().getAggregator(), aggregator); + } @Test public void testGetPartialJoinOperator() {