You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@samza.apache.org by ni...@apache.org on 2016/12/01 22:51:01 UTC
[1/6] samza git commit: SAMZA-1054: Refactor Operator APIs
Repository: samza
Updated Branches:
refs/heads/master a980c9622 -> 00543804b
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java b/samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java
deleted file mode 100644
index f6b3ff8..0000000
--- a/samza-operator/src/test/java/org/apache/samza/task/JoinOperatorTask.java
+++ /dev/null
@@ -1,80 +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.task;
-
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.MessageStreams.SystemMessageStream;
-import org.apache.samza.operators.data.IncomingSystemMessage;
-import org.apache.samza.operators.data.Offset;
-import org.apache.samza.operators.task.StreamOperatorTask;
-import org.apache.samza.system.SystemStreamPartition;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
-
-/**
- * Example implementation of unique key-based stream-stream join tasks
- *
- */
-public class JoinOperatorTask implements StreamOperatorTask {
- class MessageType {
- String joinKey;
- List<String> joinFields = new ArrayList<>();
- }
-
- class JsonMessage extends InputJsonSystemMessage<MessageType> {
-
- JsonMessage(String key, MessageType data, Offset offset, long timestamp, SystemStreamPartition partition) {
- super(key, data, offset, timestamp, partition);
- }
- }
-
- MessageStream<JsonMessage> joinOutput = null;
-
- @Override public void initOperators(Collection<SystemMessageStream> sources) {
- sources.forEach(source -> {
- MessageStream<JsonMessage> newSource = source.map(this::getInputMessage);
- if (joinOutput == null) {
- joinOutput = newSource;
- } else {
- joinOutput = joinOutput.join(newSource, (m1, m2) -> this.myJoinResult(m1, m2));
- }
- });
- }
-
- private JsonMessage getInputMessage(IncomingSystemMessage ism) {
- return new JsonMessage(
- ((MessageType) ism.getMessage()).joinKey,
- (MessageType) ism.getMessage(),
- ism.getOffset(),
- ism.getTimestamp(),
- ism.getSystemStreamPartition());
- }
-
- JsonMessage myJoinResult(JsonMessage m1, JsonMessage m2) {
- MessageType newJoinMsg = new MessageType();
- newJoinMsg.joinKey = m1.getKey();
- newJoinMsg.joinFields.addAll(m1.getMessage().joinFields);
- newJoinMsg.joinFields.addAll(m2.getMessage().joinFields);
- return new JsonMessage(m1.getMessage().joinKey, newJoinMsg, null, m1.getTimestamp(), null);
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorAdaptorTask.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorAdaptorTask.java b/samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorAdaptorTask.java
deleted file mode 100644
index 47d6b3a..0000000
--- a/samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorAdaptorTask.java
+++ /dev/null
@@ -1,80 +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.task;
-
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.impl.ChainedOperators;
-import org.apache.samza.operators.task.StreamOperatorTask;
-import org.apache.samza.system.SystemStreamPartition;
-import org.apache.samza.Partition;
-import org.junit.Before;
-import org.junit.Test;
-import org.mockito.Mockito;
-
-import java.lang.reflect.Field;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.*;
-
-
-public class TestStreamOperatorAdaptorTask {
- Field userTaskField = null;
- Field chainedOpsField = null;
-
- @Before public void prep() throws NoSuchFieldException {
- userTaskField = StreamOperatorAdaptorTask.class.getDeclaredField("userTask");
- chainedOpsField = StreamOperatorAdaptorTask.class.getDeclaredField("operatorChains");
- userTaskField.setAccessible(true);
- chainedOpsField.setAccessible(true);
- }
-
-
- @Test public void testConstructor() throws IllegalAccessException {
- StreamOperatorTask userTask = mock(StreamOperatorTask.class);
- StreamOperatorAdaptorTask adaptorTask = new StreamOperatorAdaptorTask(userTask);
- StreamOperatorTask taskMemberVar = (StreamOperatorTask) userTaskField.get(adaptorTask);
- Map<SystemStreamPartition, ChainedOperators> chainsMap = (Map<SystemStreamPartition, ChainedOperators>) chainedOpsField.get(adaptorTask);
- assertEquals(taskMemberVar, userTask);
- assertTrue(chainsMap.isEmpty());
- }
-
- @Test public void testInit() throws Exception {
- StreamOperatorTask userTask = mock(StreamOperatorTask.class);
- StreamOperatorAdaptorTask adaptorTask = new StreamOperatorAdaptorTask(userTask);
- Config mockConfig = mock(Config.class);
- TaskContext mockContext = mock(TaskContext.class);
- Set<SystemStreamPartition> testInputs = new HashSet() { {
- this.add(new SystemStreamPartition("test-sys", "test-strm", new Partition(0)));
- this.add(new SystemStreamPartition("test-sys", "test-strm", new Partition(1)));
- } };
- when(mockContext.getSystemStreamPartitions()).thenReturn(testInputs);
- adaptorTask.init(mockConfig, mockContext);
- verify(userTask, times(1)).initOperators(Mockito.anyCollection());
- Map<SystemStreamPartition, ChainedOperators> chainsMap = (Map<SystemStreamPartition, ChainedOperators>) chainedOpsField.get(adaptorTask);
- assertTrue(chainsMap.size() == 2);
- assertTrue(chainsMap.containsKey(testInputs.toArray()[0]));
- assertTrue(chainsMap.containsKey(testInputs.toArray()[1]));
- }
-
- // TODO: window and process methods to be added after implementation of ChainedOperators.create()
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorTasks.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorTasks.java b/samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorTasks.java
deleted file mode 100644
index 44efa6d..0000000
--- a/samza-operator/src/test/java/org/apache/samza/task/TestStreamOperatorTasks.java
+++ /dev/null
@@ -1,105 +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.task;
-
-import org.apache.samza.Partition;
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.impl.ChainedOperators;
-import org.apache.samza.system.SystemStreamPartition;
-import org.junit.Test;
-
-import java.lang.reflect.Field;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-
-/**
- * Unit test for {@link org.apache.samza.operators.task.StreamOperatorTask}
- */
-public class TestStreamOperatorTasks {
-
- private final WindowOperatorTask userTask = new WindowOperatorTask();
-
- private final BroadcastOperatorTask splitTask = new BroadcastOperatorTask();
-
- private final JoinOperatorTask joinTask = new JoinOperatorTask();
-
- private final Set<SystemStreamPartition> inputPartitions = new HashSet<SystemStreamPartition>() { {
- for (int i = 0; i < 4; i++) {
- this.add(new SystemStreamPartition("my-system", "my-topic1", new Partition(i)));
- }
- } };
-
- @Test public void testUserTask() throws Exception {
- Config mockConfig = mock(Config.class);
- TaskContext mockContext = mock(TaskContext.class);
- when(mockContext.getSystemStreamPartitions()).thenReturn(this.inputPartitions);
- StreamOperatorAdaptorTask adaptorTask = new StreamOperatorAdaptorTask(this.userTask);
- Field pipelineMapFld = StreamOperatorAdaptorTask.class.getDeclaredField("operatorChains");
- pipelineMapFld.setAccessible(true);
- Map<SystemStreamPartition, ChainedOperators> pipelineMap =
- (Map<SystemStreamPartition, ChainedOperators>) pipelineMapFld.get(adaptorTask);
-
- adaptorTask.init(mockConfig, mockContext);
- assertEquals(pipelineMap.size(), 4);
- this.inputPartitions.forEach(partition -> {
- assertNotNull(pipelineMap.get(partition));
- });
- }
-
- @Test public void testSplitTask() throws Exception {
- Config mockConfig = mock(Config.class);
- TaskContext mockContext = mock(TaskContext.class);
- when(mockContext.getSystemStreamPartitions()).thenReturn(this.inputPartitions);
- StreamOperatorAdaptorTask adaptorTask = new StreamOperatorAdaptorTask(this.splitTask);
- Field pipelineMapFld = StreamOperatorAdaptorTask.class.getDeclaredField("operatorChains");
- pipelineMapFld.setAccessible(true);
- Map<SystemStreamPartition, ChainedOperators> pipelineMap =
- (Map<SystemStreamPartition, ChainedOperators>) pipelineMapFld.get(adaptorTask);
-
- adaptorTask.init(mockConfig, mockContext);
- assertEquals(pipelineMap.size(), 4);
- this.inputPartitions.forEach(partition -> {
- assertNotNull(pipelineMap.get(partition));
- });
- }
-
- @Test public void testJoinTask() throws Exception {
- Config mockConfig = mock(Config.class);
- TaskContext mockContext = mock(TaskContext.class);
- when(mockContext.getSystemStreamPartitions()).thenReturn(this.inputPartitions);
- StreamOperatorAdaptorTask adaptorTask = new StreamOperatorAdaptorTask(this.joinTask);
- Field pipelineMapFld = StreamOperatorAdaptorTask.class.getDeclaredField("operatorChains");
- pipelineMapFld.setAccessible(true);
- Map<SystemStreamPartition, ChainedOperators> pipelineMap =
- (Map<SystemStreamPartition, ChainedOperators>) pipelineMapFld.get(adaptorTask);
-
- adaptorTask.init(mockConfig, mockContext);
- assertEquals(pipelineMap.size(), 4);
- this.inputPartitions.forEach(partition -> {
- assertNotNull(pipelineMap.get(partition));
- });
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/task/WindowOperatorTask.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/task/WindowOperatorTask.java b/samza-operator/src/test/java/org/apache/samza/task/WindowOperatorTask.java
deleted file mode 100644
index de7bba5..0000000
--- a/samza-operator/src/test/java/org/apache/samza/task/WindowOperatorTask.java
+++ /dev/null
@@ -1,71 +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.task;
-
-import org.apache.samza.operators.MessageStreams.SystemMessageStream;
-import org.apache.samza.operators.TriggerBuilder;
-import org.apache.samza.operators.Windows;
-import org.apache.samza.operators.data.Message;
-import org.apache.samza.operators.data.Offset;
-import org.apache.samza.operators.task.StreamOperatorTask;
-import org.apache.samza.system.SystemStreamPartition;
-
-import java.util.Collection;
-
-
-/**
- * Example implementation of a simple user-defined tasks w/ window operators
- *
- */
-public class WindowOperatorTask implements StreamOperatorTask {
- class MessageType {
- String field1;
- String field2;
- }
-
- class JsonMessage extends InputJsonSystemMessage<MessageType> {
-
- JsonMessage(String key, MessageType data, Offset offset, long timestamp, SystemStreamPartition partition) {
- super(key, data, offset, timestamp, partition);
- }
- }
-
- @Override public void initOperators(Collection<SystemMessageStream> sources) {
- sources.forEach(source ->
- source.map(m1 ->
- new JsonMessage(
- this.myMessageKeyFunction(m1),
- (MessageType) m1.getMessage(),
- m1.getOffset(),
- m1.getTimestamp(),
- m1.getSystemStreamPartition())).
- window(
- Windows.<JsonMessage, String>intoSessionCounter(
- m -> String.format("%s-%s", m.getMessage().field1, m.getMessage().field2)).
- setTriggers(TriggerBuilder.<JsonMessage, Integer>earlyTriggerWhenExceedWndLen(100).
- addTimeoutSinceLastMessage(30000)))
- );
- }
-
- String myMessageKeyFunction(Message<Object, Object> m) {
- return m.getKey().toString();
- }
-
-}
[2/6] samza git commit: SAMZA-1054: Refactor Operator APIs
Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..2ad6461
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java
@@ -0,0 +1,203 @@
+/*
+ * 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.functions.FilterFunction;
+import org.apache.samza.operators.functions.FlatMapFunction;
+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.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.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;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+
+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;
+
+
+public class TestMessageStreamImpl {
+
+ @Test
+ public void testMap() {
+ MessageStreamImpl<TestMessageEnvelope> inputStream = new MessageStreamImpl<>();
+ MapFunction<TestMessageEnvelope, TestOutputMessageEnvelope> xMap =
+ m -> new TestOutputMessageEnvelope(m.getKey(), m.getMessage().getValue().length() + 1);
+ MessageStream<TestOutputMessageEnvelope> outputStream = inputStream.map(xMap);
+ Collection<OperatorSpec> subs = inputStream.getRegisteredOperatorSpecs();
+ assertEquals(subs.size(), 1);
+ OperatorSpec<TestOutputMessageEnvelope> mapOp = subs.iterator().next();
+ assertTrue(mapOp instanceof StreamOperatorSpec);
+ assertEquals(mapOp.getOutputStream(), outputStream);
+ // assert that the transformation function is what we defined above
+ TestMessageEnvelope xTestMsg = mock(TestMessageEnvelope.class);
+ TestMessageEnvelope.MessageType mockInnerTestMessage = mock(TestMessageEnvelope.MessageType.class);
+ when(xTestMsg.getKey()).thenReturn("test-msg-key");
+ when(xTestMsg.getMessage()).thenReturn(mockInnerTestMessage);
+ when(mockInnerTestMessage.getValue()).thenReturn("123456789");
+
+ Collection<TestOutputMessageEnvelope> cOutputMsg = ((StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope>) mapOp).getTransformFn().apply(xTestMsg);
+ assertEquals(cOutputMsg.size(), 1);
+ TestOutputMessageEnvelope outputMessage = cOutputMsg.iterator().next();
+ assertEquals(outputMessage.getKey(), xTestMsg.getKey());
+ assertEquals(outputMessage.getMessage(), Integer.valueOf(xTestMsg.getMessage().getValue().length() + 1));
+ }
+
+ @Test
+ public void testFlatMap() {
+ MessageStreamImpl<TestMessageEnvelope> inputStream = new MessageStreamImpl<>();
+ Set<TestOutputMessageEnvelope> flatOuts = new HashSet<TestOutputMessageEnvelope>() { {
+ this.add(mock(TestOutputMessageEnvelope.class));
+ this.add(mock(TestOutputMessageEnvelope.class));
+ this.add(mock(TestOutputMessageEnvelope.class));
+ } };
+ FlatMapFunction<TestMessageEnvelope, TestOutputMessageEnvelope> xFlatMap = m -> flatOuts;
+ MessageStream<TestOutputMessageEnvelope> outputStream = inputStream.flatMap(xFlatMap);
+ Collection<OperatorSpec> subs = inputStream.getRegisteredOperatorSpecs();
+ assertEquals(subs.size(), 1);
+ OperatorSpec<TestOutputMessageEnvelope> flatMapOp = subs.iterator().next();
+ assertTrue(flatMapOp instanceof StreamOperatorSpec);
+ assertEquals(flatMapOp.getOutputStream(), outputStream);
+ // assert that the transformation function is what we defined above
+ assertEquals(((StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope>) flatMapOp).getTransformFn(), xFlatMap);
+ }
+
+ @Test
+ public void testFilter() {
+ MessageStreamImpl<TestMessageEnvelope> inputStream = new MessageStreamImpl<>();
+ FilterFunction<TestMessageEnvelope> xFilter = m -> m.getMessage().getEventTime() > 123456L;
+ MessageStream<TestMessageEnvelope> outputStream = inputStream.filter(xFilter);
+ Collection<OperatorSpec> subs = inputStream.getRegisteredOperatorSpecs();
+ assertEquals(subs.size(), 1);
+ OperatorSpec<TestMessageEnvelope> filterOp = subs.iterator().next();
+ assertTrue(filterOp instanceof StreamOperatorSpec);
+ assertEquals(filterOp.getOutputStream(), outputStream);
+ // assert that the transformation function is what we defined above
+ FlatMapFunction<TestMessageEnvelope, TestMessageEnvelope> txfmFn = ((StreamOperatorSpec<TestMessageEnvelope, TestMessageEnvelope>) filterOp).getTransformFn();
+ TestMessageEnvelope mockMsg = mock(TestMessageEnvelope.class);
+ TestMessageEnvelope.MessageType mockInnerTestMessage = mock(TestMessageEnvelope.MessageType.class);
+ when(mockMsg.getMessage()).thenReturn(mockInnerTestMessage);
+ when(mockInnerTestMessage.getEventTime()).thenReturn(11111L);
+ Collection<TestMessageEnvelope> output = txfmFn.apply(mockMsg);
+ assertTrue(output.isEmpty());
+ when(mockMsg.getMessage()).thenReturn(mockInnerTestMessage);
+ when(mockInnerTestMessage.getEventTime()).thenReturn(999999L);
+ output = txfmFn.apply(mockMsg);
+ assertEquals(output.size(), 1);
+ assertEquals(output.iterator().next(), mockMsg);
+ }
+
+ @Test
+ public void testSink() {
+ MessageStreamImpl<TestMessageEnvelope> inputStream = new MessageStreamImpl<>();
+ SinkFunction<TestMessageEnvelope> xSink = (m, mc, tc) -> {
+ mc.send(new OutgoingMessageEnvelope(new SystemStream("test-sys", "test-stream"), m.getMessage()));
+ tc.commit(TaskCoordinator.RequestScope.CURRENT_TASK);
+ };
+ inputStream.sink(xSink);
+ Collection<OperatorSpec> subs = inputStream.getRegisteredOperatorSpecs();
+ assertEquals(subs.size(), 1);
+ OperatorSpec<TestMessageEnvelope> sinkOp = subs.iterator().next();
+ assertTrue(sinkOp instanceof SinkOperatorSpec);
+ assertEquals(((SinkOperatorSpec) sinkOp).getSinkFn(), xSink);
+ assertNull(((SinkOperatorSpec) sinkOp).getOutputStream());
+ }
+
+ @Test
+ public void testWindow() {
+ MessageStreamImpl<TestMessageEnvelope> inputStream = new MessageStreamImpl<>();
+ SessionWindow<TestMessageEnvelope, String, Integer> window = mock(SessionWindow.class);
+ doReturn(mock(WindowFn.class)).when(window).getInternalWindowFn();
+ MessageStream<WindowOutput<String, Integer>> outStream = inputStream.window(window);
+ Collection<OperatorSpec> subs = inputStream.getRegisteredOperatorSpecs();
+ assertEquals(subs.size(), 1);
+ OperatorSpec<TestMessageEnvelope> wndOp = subs.iterator().next();
+ assertTrue(wndOp instanceof WindowOperatorSpec);
+ assertEquals(((WindowOperatorSpec) wndOp).getOutputStream(), outStream);
+ }
+
+ @Test
+ public void testJoin() {
+ MessageStreamImpl<TestMessageEnvelope> source1 = new MessageStreamImpl<>();
+ MessageStreamImpl<TestMessageEnvelope> source2 = new MessageStreamImpl<>();
+ JoinFunction<TestMessageEnvelope, TestMessageEnvelope, TestOutputMessageEnvelope> joiner =
+ (m1, m2) -> new TestOutputMessageEnvelope(m1.getKey(), m1.getMessage().getValue().length() + m2.getMessage().getValue().length());
+ MessageStream<TestOutputMessageEnvelope> joinOutput = source1.join(source2, joiner);
+ Collection<OperatorSpec> subs = source1.getRegisteredOperatorSpecs();
+ assertEquals(subs.size(), 1);
+ OperatorSpec<TestMessageEnvelope> joinOp1 = subs.iterator().next();
+ assertTrue(joinOp1 instanceof PartialJoinOperatorSpec);
+ assertEquals(((PartialJoinOperatorSpec) joinOp1).getOutputStream(), joinOutput);
+ subs = source2.getRegisteredOperatorSpecs();
+ assertEquals(subs.size(), 1);
+ OperatorSpec<TestMessageEnvelope> joinOp2 = subs.iterator().next();
+ assertTrue(joinOp2 instanceof PartialJoinOperatorSpec);
+ assertEquals(((PartialJoinOperatorSpec) joinOp2).getOutputStream(), joinOutput);
+ TestMessageEnvelope joinMsg1 = new TestMessageEnvelope("test-join-1", "join-msg-001", 11111L);
+ TestMessageEnvelope joinMsg2 = new TestMessageEnvelope("test-join-2", "join-msg-002", 22222L);
+ TestOutputMessageEnvelope xOut = (TestOutputMessageEnvelope) ((PartialJoinOperatorSpec) joinOp1).getTransformFn().apply(joinMsg1, joinMsg2);
+ assertEquals(xOut.getKey(), "test-join-1");
+ assertEquals(xOut.getMessage(), Integer.valueOf(24));
+ xOut = (TestOutputMessageEnvelope) ((PartialJoinOperatorSpec) joinOp2).getTransformFn().apply(joinMsg2, joinMsg1);
+ assertEquals(xOut.getKey(), "test-join-1");
+ assertEquals(xOut.getMessage(), Integer.valueOf(24));
+ }
+
+ @Test
+ public void testMerge() {
+ MessageStream<TestMessageEnvelope> merge1 = new MessageStreamImpl<>();
+ Collection<MessageStream<TestMessageEnvelope>> others = new ArrayList<MessageStream<TestMessageEnvelope>>() { {
+ this.add(new MessageStreamImpl<>());
+ this.add(new MessageStreamImpl<>());
+ } };
+ MessageStream<TestMessageEnvelope> mergeOutput = merge1.merge(others);
+ validateMergeOperator(merge1, mergeOutput);
+
+ others.forEach(merge -> validateMergeOperator(merge, mergeOutput));
+ }
+
+ private void validateMergeOperator(MessageStream<TestMessageEnvelope> mergeSource, MessageStream<TestMessageEnvelope> mergeOutput) {
+ Collection<OperatorSpec> subs = ((MessageStreamImpl<TestMessageEnvelope>) mergeSource).getRegisteredOperatorSpecs();
+ assertEquals(subs.size(), 1);
+ OperatorSpec<TestMessageEnvelope> mergeOp = subs.iterator().next();
+ assertTrue(mergeOp instanceof StreamOperatorSpec);
+ assertEquals(((StreamOperatorSpec) mergeOp).getOutputStream(), mergeOutput);
+ TestMessageEnvelope mockMsg = mock(TestMessageEnvelope.class);
+ Collection<TestMessageEnvelope> outputs = ((StreamOperatorSpec<TestMessageEnvelope, TestMessageEnvelope>) mergeOp).getTransformFn().apply(mockMsg);
+ assertEquals(outputs.size(), 1);
+ assertEquals(outputs.iterator().next(), mockMsg);
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/TestStateStoreImpl.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..8fa7ccc
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/TestStateStoreImpl.java
@@ -0,0 +1,72 @@
+/*
+ * 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.windows.StoreFunctions;
+import org.apache.samza.operators.windows.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<TestMessageEnvelope, String, WindowState> mockStoreFunctions = mock(StoreFunctions.class);
+ // test constructor
+ StateStoreImpl<TestMessageEnvelope, String, WindowState> storeImpl = new StateStoreImpl<>(mockStoreFunctions, "myStoreName");
+ TaskContext mockContext = mock(TaskContext.class);
+ KeyValueStore<String, WindowState> mockKvStore = mock(KeyValueStore.class);
+ when(mockContext.getStore("myStoreName")).thenReturn(mockKvStore);
+ // test init()
+ storeImpl.init(mockContext);
+ verify(mockContext, times(1)).getStore("myStoreName");
+ Function<TestMessageEnvelope, String> 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<String, WindowState> 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<String, WindowState> oldEntry = new Entry<>("myKey", mockState);
+ WindowState mockNewState = mock(WindowState.class);
+ BiFunction<TestMessageEnvelope, WindowState, WindowState> mockUpdaterFn = mock(BiFunction.class);
+ when(mockStoreFunctions.getStateUpdaterFn()).thenReturn(mockUpdaterFn);
+ when(mockUpdaterFn.apply(mockMsg, mockState)).thenReturn(mockNewState);
+ // test updateState()
+ Entry<String, WindowState> newEntry = storeImpl.updateState(mockMsg, oldEntry);
+ assertEquals(newEntry.getKey(), "myKey");
+ assertEquals(newEntry.getValue(), mockNewState);
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/WindowTask.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..f33510e
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/WindowTask.java
@@ -0,0 +1,70 @@
+/*
+ * 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.IncomingSystemMessageEnvelope;
+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.Windows;
+import org.apache.samza.system.SystemStreamPartition;
+
+import java.util.Map;
+
+
+/**
+ * Example implementation of a simple user-defined tasks w/ window operators
+ *
+ */
+public class WindowTask implements StreamOperatorTask {
+ class MessageType {
+ String field1;
+ String field2;
+ }
+
+ class JsonMessageEnvelope extends JsonIncomingSystemMessageEnvelope<MessageType> {
+
+ JsonMessageEnvelope(String key, MessageType data, Offset offset, SystemStreamPartition partition) {
+ super(key, data, offset, partition);
+ }
+ }
+
+ @Override public void transform(Map<SystemStreamPartition, MessageStream<IncomingSystemMessageEnvelope>> messageStreams) {
+ messageStreams.values().forEach(source ->
+ source.map(m1 ->
+ new JsonMessageEnvelope(
+ this.myMessageKeyFunction(m1),
+ (MessageType) m1.getMessage(),
+ m1.getOffset(),
+ m1.getSystemStreamPartition())).
+ window(
+ Windows.<JsonMessageEnvelope, String>intoSessionCounter(
+ m -> String.format("%s-%s", m.getMessage().field1, m.getMessage().field2)).
+ setTriggers(TriggerBuilder.<JsonMessageEnvelope, Integer>earlyTriggerWhenExceedWndLen(100).
+ addTimeoutSinceLastMessage(30000)))
+ );
+ }
+
+ String myMessageKeyFunction(MessageEnvelope<Object, Object> m) {
+ return m.getKey().toString();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/data/JsonIncomingSystemMessageEnvelope.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/data/JsonIncomingSystemMessageEnvelope.java b/samza-operator/src/test/java/org/apache/samza/operators/data/JsonIncomingSystemMessageEnvelope.java
new file mode 100644
index 0000000..9a425d1
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/data/JsonIncomingSystemMessageEnvelope.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.data;
+
+import org.apache.samza.system.SystemStreamPartition;
+
+
+/**
+ * Example input {@link MessageEnvelope} w/ Json message and string as the key.
+ */
+
+public class JsonIncomingSystemMessageEnvelope<T> implements MessageEnvelope<String, T> {
+
+ private final String key;
+ private final T data;
+ private final Offset offset;
+ private final SystemStreamPartition partition;
+
+ public JsonIncomingSystemMessageEnvelope(String key, T data, Offset offset, SystemStreamPartition partition) {
+ this.key = key;
+ this.data = data;
+ this.offset = offset;
+ this.partition = partition;
+ }
+
+ @Override
+ public T getMessage() {
+ return this.data;
+ }
+
+ @Override
+ public String getKey() {
+ return this.key;
+ }
+
+ public Offset getOffset() {
+ return this.offset;
+ }
+
+ public SystemStreamPartition getSystemStreamPartition() {
+ return this.partition;
+ }
+}
+
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/impl/TestChainedOperators.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestChainedOperators.java b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestChainedOperators.java
deleted file mode 100644
index e3a70e8..0000000
--- a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestChainedOperators.java
+++ /dev/null
@@ -1,129 +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.TestMessage;
-import org.apache.samza.operators.TestOutputMessage;
-import org.apache.samza.operators.Windows;
-import org.apache.samza.task.TaskContext;
-import org.junit.Before;
-import org.junit.Test;
-import org.reactivestreams.Subscriber;
-
-import java.lang.reflect.Field;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.Set;
-
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.mock;
-
-
-public class TestChainedOperators {
- Field subsField = null;
- Field opSubsField = null;
-
- @Before public void prep() throws NoSuchFieldException {
- subsField = ChainedOperators.class.getDeclaredField("subscribers");
- subsField.setAccessible(true);
- opSubsField = OperatorImpl.class.getDeclaredField("subscribers");
- opSubsField.setAccessible(true);
- }
-
- @Test public void testCreate() {
- // test creation of empty chain
- MessageStream<TestMessage> testStream = new MessageStream<>();
- TaskContext mockContext = mock(TaskContext.class);
- ChainedOperators<TestMessage> operatorChain = ChainedOperators.create(testStream, mockContext);
- assertTrue(operatorChain != null);
- }
-
- @Test public void testLinearChain() throws IllegalAccessException {
- // test creation of linear chain
- MessageStream<TestMessage> testInput = new MessageStream<>();
- TaskContext mockContext = mock(TaskContext.class);
- testInput.map(m -> m).window(Windows.intoSessionCounter(TestMessage::getKey));
- ChainedOperators<TestMessage> operatorChain = ChainedOperators.create(testInput, mockContext);
- Set<OperatorImpl> subsSet = (Set<OperatorImpl>) subsField.get(operatorChain);
- assertEquals(subsSet.size(), 1);
- OperatorImpl<TestMessage, TestMessage> firstOpImpl = subsSet.iterator().next();
- Set<Subscriber<? super ProcessorContext<TestMessage>>> subsOps = (Set<Subscriber<? super ProcessorContext<TestMessage>>>) opSubsField.get(firstOpImpl);
- assertEquals(subsOps.size(), 1);
- Subscriber<? super ProcessorContext<TestMessage>> wndOpImpl = subsOps.iterator().next();
- subsOps = (Set<Subscriber<? super ProcessorContext<TestMessage>>>) opSubsField.get(wndOpImpl);
- assertEquals(subsOps.size(), 0);
- }
-
- @Test public void testBroadcastChain() throws IllegalAccessException {
- // test creation of broadcast chain
- MessageStream<TestMessage> testInput = new MessageStream<>();
- TaskContext mockContext = mock(TaskContext.class);
- testInput.filter(m -> m.getTimestamp() > 123456L).flatMap(m -> new ArrayList() { { this.add(m); this.add(m); } });
- testInput.filter(m -> m.getTimestamp() < 123456L).map(m -> m);
- ChainedOperators<TestMessage> operatorChain = ChainedOperators.create(testInput, mockContext);
- Set<OperatorImpl> subsSet = (Set<OperatorImpl>) subsField.get(operatorChain);
- assertEquals(subsSet.size(), 2);
- Iterator<OperatorImpl> iter = subsSet.iterator();
- // check the first branch w/ flatMap
- OperatorImpl<TestMessage, TestMessage> opImpl = iter.next();
- Set<Subscriber<? super ProcessorContext<TestMessage>>> subsOps = (Set<Subscriber<? super ProcessorContext<TestMessage>>>) opSubsField.get(opImpl);
- assertEquals(subsOps.size(), 1);
- Subscriber<? super ProcessorContext<TestMessage>> flatMapImpl = subsOps.iterator().next();
- subsOps = (Set<Subscriber<? super ProcessorContext<TestMessage>>>) opSubsField.get(flatMapImpl);
- assertEquals(subsOps.size(), 0);
- // check the second branch w/ map
- opImpl = iter.next();
- subsOps = (Set<Subscriber<? super ProcessorContext<TestMessage>>>) opSubsField.get(opImpl);
- assertEquals(subsOps.size(), 1);
- Subscriber<? super ProcessorContext<TestMessage>> mapImpl = subsOps.iterator().next();
- subsOps = (Set<Subscriber<? super ProcessorContext<TestMessage>>>) opSubsField.get(mapImpl);
- assertEquals(subsOps.size(), 0);
- }
-
- @Test public void testJoinChain() throws IllegalAccessException {
- // test creation of join chain
- MessageStream<TestMessage> input1 = new MessageStream<>();
- MessageStream<TestMessage> input2 = new MessageStream<>();
- TaskContext mockContext = mock(TaskContext.class);
- input1.join(input2, (m1, m2) -> new TestOutputMessage(m1.getKey(), m1.getMessage().length() + m2.getMessage().length(), m1.getTimestamp())).map(m -> m);
- // now, we create chained operators from each input sources
- ChainedOperators<TestMessage> chain1 = ChainedOperators.create(input1, mockContext);
- ChainedOperators<TestMessage> chain2 = ChainedOperators.create(input2, mockContext);
- // check that those two chains will merge at map operator
- // first branch of the join
- Set<OperatorImpl> subsSet = (Set<OperatorImpl>) subsField.get(chain1);
- assertEquals(subsSet.size(), 1);
- OperatorImpl<TestMessage, TestOutputMessage> joinOp1 = subsSet.iterator().next();
- Set<Subscriber<? super ProcessorContext<TestOutputMessage>>> subsOps = (Set<Subscriber<? super ProcessorContext<TestOutputMessage>>>) opSubsField.get(joinOp1);
- assertEquals(subsOps.size(), 1);
- // the map operator consumes the common join output, where two branches merge
- Subscriber<? super ProcessorContext<TestOutputMessage>> mapImpl = subsOps.iterator().next();
- // second branch of the join
- subsSet = (Set<OperatorImpl>) subsField.get(chain2);
- assertEquals(subsSet.size(), 1);
- OperatorImpl<TestMessage, TestOutputMessage> joinOp2 = subsSet.iterator().next();
- assertNotSame(joinOp1, joinOp2);
- subsOps = (Set<Subscriber<? super ProcessorContext<TestOutputMessage>>>) opSubsField.get(joinOp2);
- assertEquals(subsOps.size(), 1);
- // make sure that the map operator is the same
- assertEquals(mapImpl, subsOps.iterator().next());
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorFactory.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorFactory.java b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorFactory.java
deleted file mode 100644
index cb4576c..0000000
--- a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorFactory.java
+++ /dev/null
@@ -1,93 +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.TestMessage;
-import org.apache.samza.operators.TestOutputMessage;
-import org.apache.samza.operators.data.Message;
-import org.apache.samza.operators.impl.join.PartialJoinOpImpl;
-import org.apache.samza.operators.impl.window.SessionWindowImpl;
-import org.apache.samza.operators.internal.Operators.PartialJoinOperator;
-import org.apache.samza.operators.internal.Operators.SinkOperator;
-import org.apache.samza.operators.internal.Operators.StreamOperator;
-import org.apache.samza.operators.internal.Operators.WindowOperator;
-import org.apache.samza.storage.kv.Entry;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskCoordinator;
-import org.junit.Test;
-
-import java.lang.reflect.Field;
-import java.util.Collection;
-import java.util.function.BiFunction;
-import java.util.function.Function;
-
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-
-public class TestOperatorFactory {
-
- @Test public void testGetOperator() throws NoSuchFieldException, IllegalAccessException {
- // get window operator
- WindowOperator mockWnd = mock(WindowOperator.class);
- Entry<OperatorImpl<TestMessage, ? extends Message>, Boolean>
- factoryEntry = OperatorFactory.<TestMessage, TestOutputMessage>getOperator(mockWnd);
- assertFalse(factoryEntry.getValue());
- OperatorImpl<TestMessage, TestOutputMessage> opImpl = (OperatorImpl<TestMessage, TestOutputMessage>) factoryEntry.getKey();
- assertTrue(opImpl instanceof SessionWindowImpl);
- Field sessWndField = SessionWindowImpl.class.getDeclaredField("sessWnd");
- sessWndField.setAccessible(true);
- WindowOperator sessWnd = (WindowOperator) sessWndField.get(opImpl);
- assertEquals(sessWnd, mockWnd);
-
- // get simple operator
- StreamOperator<TestMessage, TestOutputMessage> mockSimpleOp = mock(StreamOperator.class);
- Function<TestMessage, Collection<TestOutputMessage>> mockTxfmFn = mock(Function.class);
- when(mockSimpleOp.getFunction()).thenReturn(mockTxfmFn);
- factoryEntry = OperatorFactory.<TestMessage, TestOutputMessage>getOperator(mockSimpleOp);
- opImpl = (OperatorImpl<TestMessage, TestOutputMessage>) factoryEntry.getKey();
- assertTrue(opImpl instanceof SimpleOperatorImpl);
- Field txfmFnField = SimpleOperatorImpl.class.getDeclaredField("transformFn");
- txfmFnField.setAccessible(true);
- assertEquals(mockTxfmFn, txfmFnField.get(opImpl));
-
- // get sink operator
- MessageStream.VoidFunction3<TestMessage, MessageCollector, TaskCoordinator> sinkFn = (m, mc, tc) -> { };
- SinkOperator<TestMessage> sinkOp = mock(SinkOperator.class);
- when(sinkOp.getFunction()).thenReturn(sinkFn);
- factoryEntry = OperatorFactory.<TestMessage, TestOutputMessage>getOperator(sinkOp);
- opImpl = (OperatorImpl<TestMessage, TestOutputMessage>) factoryEntry.getKey();
- assertTrue(opImpl instanceof SinkOperatorImpl);
- Field sinkFnField = SinkOperatorImpl.class.getDeclaredField("sinkFunc");
- sinkFnField.setAccessible(true);
- assertEquals(sinkFn, sinkFnField.get(opImpl));
-
- // get join operator
- PartialJoinOperator<TestMessage, String, TestMessage, TestOutputMessage> joinOp = mock(PartialJoinOperator.class);
- TestOutputMessage mockOutput = mock(TestOutputMessage.class);
- BiFunction<TestMessage, TestMessage, TestOutputMessage> joinFn = (m1, m2) -> mockOutput;
- when(joinOp.getFunction()).thenReturn(joinFn);
- factoryEntry = OperatorFactory.<TestMessage, TestOutputMessage>getOperator(joinOp);
- opImpl = (OperatorImpl<TestMessage, TestOutputMessage>) factoryEntry.getKey();
- assertTrue(opImpl instanceof PartialJoinOpImpl);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java
index 4bd467d..361972e 100644
--- a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java
+++ b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java
@@ -18,52 +18,54 @@
*/
package org.apache.samza.operators.impl;
-import org.apache.samza.operators.TestMessage;
-import org.apache.samza.operators.TestOutputMessage;
+import org.apache.samza.operators.TestMessageEnvelope;
+import org.apache.samza.operators.TestOutputMessageEnvelope;
import org.apache.samza.task.MessageCollector;
import org.apache.samza.task.TaskCoordinator;
+import org.hamcrest.core.IsEqual;
import org.junit.Test;
-import org.mockito.ArgumentMatcher;
-import org.reactivestreams.Subscriber;
import static org.junit.Assert.assertEquals;
-import static org.mockito.Mockito.*;
+import static org.mockito.Mockito.argThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
public class TestOperatorImpl {
- TestMessage curInputMsg;
+ TestMessageEnvelope curInputMsg;
MessageCollector curCollector;
TaskCoordinator curCoordinator;
- @Test public void testSubscribers() {
+ @Test
+ public void testSubscribers() {
this.curInputMsg = null;
this.curCollector = null;
this.curCoordinator = null;
- OperatorImpl<TestMessage, TestOutputMessage> opImpl = new OperatorImpl<TestMessage, TestOutputMessage>() {
- @Override protected void onNext(TestMessage message, MessageCollector collector, TaskCoordinator coordinator) {
+ OperatorImpl<TestMessageEnvelope, TestOutputMessageEnvelope> opImpl = new OperatorImpl<TestMessageEnvelope, TestOutputMessageEnvelope>() {
+ @Override
+ public void onNext(TestMessageEnvelope message, MessageCollector collector, TaskCoordinator coordinator) {
TestOperatorImpl.this.curInputMsg = message;
TestOperatorImpl.this.curCollector = collector;
TestOperatorImpl.this.curCoordinator = coordinator;
}
};
- // verify subscribe() added the mockSub and nextProcessors() invoked the mockSub.onNext()
- Subscriber<ProcessorContext<TestOutputMessage>> mockSub = mock(Subscriber.class);
- opImpl.subscribe(mockSub);
- TestOutputMessage xOutput = mock(TestOutputMessage.class);
+ // verify registerNextOperator() added the mockSub and propagateResult() invoked the mockSub.onNext()
+ OperatorImpl mockSub = mock(OperatorImpl.class);
+ opImpl.registerNextOperator(mockSub);
+ TestOutputMessageEnvelope xOutput = mock(TestOutputMessageEnvelope.class);
MessageCollector mockCollector = mock(MessageCollector.class);
TaskCoordinator mockCoordinator = mock(TaskCoordinator.class);
- opImpl.nextProcessors(xOutput, mockCollector, mockCoordinator);
- verify(mockSub, times(1)).onNext(argThat(new ArgumentMatcher<ProcessorContext<TestOutputMessage>>() {
- @Override public boolean matches(Object argument) {
- ProcessorContext<TestOutputMessage> pCntx = (ProcessorContext<TestOutputMessage>) argument;
- return pCntx.getMessage().equals(xOutput) && pCntx.getCoordinator().equals(mockCoordinator) && pCntx.getCollector().equals(mockCollector);
- }
- }));
+ opImpl.propagateResult(xOutput, mockCollector, mockCoordinator);
+ verify(mockSub, times(1)).onNext(
+ argThat(new IsEqual<>(xOutput)),
+ argThat(new IsEqual<>(mockCollector)),
+ argThat(new IsEqual<>(mockCoordinator))
+ );
// verify onNext() is invoked correctly
- TestMessage mockInput = mock(TestMessage.class);
- ProcessorContext<TestMessage> inCntx = new ProcessorContext<>(mockInput, mockCollector, mockCoordinator);
- opImpl.onNext(inCntx);
+ TestMessageEnvelope mockInput = mock(TestMessageEnvelope.class);
+ opImpl.onNext(mockInput, mockCollector, mockCoordinator);
assertEquals(mockInput, this.curInputMsg);
assertEquals(mockCollector, this.curCollector);
assertEquals(mockCoordinator, this.curCoordinator);
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpls.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..1d0d547
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpls.java
@@ -0,0 +1,183 @@
+/*
+ * 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.TestOutputMessageEnvelope;
+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.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.Windows;
+import org.apache.samza.task.TaskContext;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.lang.reflect.Field;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+
+public class TestOperatorImpls {
+ Field nextOperatorsField = null;
+
+ @Before
+ public void prep() throws NoSuchFieldException {
+ nextOperatorsField = OperatorImpl.class.getDeclaredField("nextOperators");
+ nextOperatorsField.setAccessible(true);
+ }
+
+ @Test
+ public void testCreateOperator() throws NoSuchFieldException, IllegalAccessException {
+ // get window operator
+ WindowOperatorSpec mockWnd = mock(WindowOperatorSpec.class);
+ OperatorImpl<TestMessageEnvelope, ? extends MessageEnvelope> 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);
+
+ // get simple operator
+ StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope> mockSimpleOp = mock(StreamOperatorSpec.class);
+ FlatMapFunction<TestMessageEnvelope, TestOutputMessageEnvelope> mockTxfmFn = mock(FlatMapFunction.class);
+ when(mockSimpleOp.getTransformFn()).thenReturn(mockTxfmFn);
+ opImpl = OperatorImpls.createOperatorImpl(mockSimpleOp);
+ assertTrue(opImpl instanceof StreamOperatorImpl);
+ Field txfmFnField = StreamOperatorImpl.class.getDeclaredField("transformFn");
+ txfmFnField.setAccessible(true);
+ assertEquals(mockTxfmFn, txfmFnField.get(opImpl));
+
+ // get sink operator
+ SinkFunction<TestMessageEnvelope> sinkFn = (m, mc, tc) -> { };
+ SinkOperatorSpec<TestMessageEnvelope> sinkOp = mock(SinkOperatorSpec.class);
+ when(sinkOp.getSinkFn()).thenReturn(sinkFn);
+ opImpl = OperatorImpls.createOperatorImpl(sinkOp);
+ assertTrue(opImpl instanceof SinkOperatorImpl);
+ Field sinkFnField = SinkOperatorImpl.class.getDeclaredField("sinkFn");
+ sinkFnField.setAccessible(true);
+ assertEquals(sinkFn, sinkFnField.get(opImpl));
+
+ // get join operator
+ PartialJoinOperatorSpec<TestMessageEnvelope, String, TestMessageEnvelope, TestOutputMessageEnvelope> joinOp = mock(PartialJoinOperatorSpec.class);
+ TestOutputMessageEnvelope mockOutput = mock(TestOutputMessageEnvelope.class);
+ BiFunction<TestMessageEnvelope, TestMessageEnvelope, TestOutputMessageEnvelope> joinFn = (m1, m2) -> mockOutput;
+ when(joinOp.getTransformFn()).thenReturn(joinFn);
+ opImpl = OperatorImpls.createOperatorImpl(joinOp);
+ assertTrue(opImpl instanceof PartialJoinOperatorImpl);
+ }
+
+ @Test
+ public void testEmptyChain() {
+ // test creation of empty chain
+ MessageStreamImpl<TestMessageEnvelope> testStream = new MessageStreamImpl<>();
+ TaskContext mockContext = mock(TaskContext.class);
+ RootOperatorImpl operatorChain = OperatorImpls.createOperatorImpls(testStream, mockContext);
+ assertTrue(operatorChain != null);
+ }
+
+ @Test
+ public void testLinearChain() throws IllegalAccessException {
+ // test creation of linear chain
+ MessageStreamImpl<TestMessageEnvelope> testInput = new MessageStreamImpl<>();
+ TaskContext mockContext = mock(TaskContext.class);
+ testInput.map(m -> m).window(Windows.intoSessionCounter(TestMessageEnvelope::getKey));
+ RootOperatorImpl operatorChain = OperatorImpls.createOperatorImpls(testInput, mockContext);
+ Set<OperatorImpl> subsSet = (Set<OperatorImpl>) nextOperatorsField.get(operatorChain);
+ assertEquals(subsSet.size(), 1);
+ OperatorImpl<TestMessageEnvelope, TestMessageEnvelope> firstOpImpl = subsSet.iterator().next();
+ Set<OperatorImpl> subsOps = (Set<OperatorImpl>) nextOperatorsField.get(firstOpImpl);
+ assertEquals(subsOps.size(), 1);
+ OperatorImpl wndOpImpl = subsOps.iterator().next();
+ subsOps = (Set<OperatorImpl>) nextOperatorsField.get(wndOpImpl);
+ assertEquals(subsOps.size(), 0);
+ }
+
+ @Test
+ public void testBroadcastChain() throws IllegalAccessException {
+ // test creation of broadcast chain
+ MessageStreamImpl<TestMessageEnvelope> testInput = new MessageStreamImpl<>();
+ TaskContext mockContext = mock(TaskContext.class);
+ testInput.filter(m -> m.getMessage().getEventTime() > 123456L).flatMap(m -> new ArrayList() { { this.add(m); this.add(m); } });
+ testInput.filter(m -> m.getMessage().getEventTime() < 123456L).map(m -> m);
+ RootOperatorImpl operatorChain = OperatorImpls.createOperatorImpls(testInput, mockContext);
+ Set<OperatorImpl> subsSet = (Set<OperatorImpl>) nextOperatorsField.get(operatorChain);
+ assertEquals(subsSet.size(), 2);
+ Iterator<OperatorImpl> iter = subsSet.iterator();
+ // check the first branch w/ flatMap
+ OperatorImpl<TestMessageEnvelope, TestMessageEnvelope> opImpl = iter.next();
+ Set<OperatorImpl> subsOps = (Set<OperatorImpl>) nextOperatorsField.get(opImpl);
+ assertEquals(subsOps.size(), 1);
+ OperatorImpl flatMapImpl = subsOps.iterator().next();
+ subsOps = (Set<OperatorImpl>) nextOperatorsField.get(flatMapImpl);
+ assertEquals(subsOps.size(), 0);
+ // check the second branch w/ map
+ opImpl = iter.next();
+ subsOps = (Set<OperatorImpl>) nextOperatorsField.get(opImpl);
+ assertEquals(subsOps.size(), 1);
+ OperatorImpl mapImpl = subsOps.iterator().next();
+ subsOps = (Set<OperatorImpl>) nextOperatorsField.get(mapImpl);
+ assertEquals(subsOps.size(), 0);
+ }
+
+ @Test
+ public void testJoinChain() throws IllegalAccessException {
+ // test creation of join chain
+ MessageStreamImpl<TestMessageEnvelope> input1 = new MessageStreamImpl<>();
+ MessageStreamImpl<TestMessageEnvelope> input2 = new MessageStreamImpl<>();
+ TaskContext mockContext = mock(TaskContext.class);
+ input1
+ .join(input2, (m1, m2) ->
+ new TestOutputMessageEnvelope(m1.getKey(), m1.getMessage().getValue().length() + m2.getMessage().getValue().length()))
+ .map(m -> m);
+ // now, we create chained operators from each input sources
+ RootOperatorImpl chain1 = OperatorImpls.createOperatorImpls(input1, mockContext);
+ RootOperatorImpl chain2 = OperatorImpls.createOperatorImpls(input2, mockContext);
+ // check that those two chains will merge at map operator
+ // first branch of the join
+ Set<OperatorImpl> subsSet = (Set<OperatorImpl>) nextOperatorsField.get(chain1);
+ assertEquals(subsSet.size(), 1);
+ OperatorImpl<TestMessageEnvelope, TestOutputMessageEnvelope> joinOp1 = subsSet.iterator().next();
+ Set<OperatorImpl> subsOps = (Set<OperatorImpl>) nextOperatorsField.get(joinOp1);
+ assertEquals(subsOps.size(), 1);
+ // the map operator consumes the common join output, where two branches merge
+ OperatorImpl mapImpl = subsOps.iterator().next();
+ // second branch of the join
+ subsSet = (Set<OperatorImpl>) nextOperatorsField.get(chain2);
+ assertEquals(subsSet.size(), 1);
+ OperatorImpl<TestMessageEnvelope, TestOutputMessageEnvelope> joinOp2 = subsSet.iterator().next();
+ assertNotSame(joinOp1, joinOp2);
+ subsOps = (Set<OperatorImpl>) nextOperatorsField.get(joinOp2);
+ assertEquals(subsOps.size(), 1);
+ // make sure that the map operator is the same
+ assertEquals(mapImpl, subsOps.iterator().next());
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/impl/TestProcessorContext.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestProcessorContext.java b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestProcessorContext.java
deleted file mode 100644
index 224245e..0000000
--- a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestProcessorContext.java
+++ /dev/null
@@ -1,40 +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.TestMessage;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskCoordinator;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.mockito.Mockito.mock;
-
-
-public class TestProcessorContext {
- @Test public void testConstructor() {
- TestMessage mockMsg = mock(TestMessage.class);
- MessageCollector mockCollector = mock(MessageCollector.class);
- TaskCoordinator mockTaskCoordinator = mock(TaskCoordinator.class);
- ProcessorContext<TestMessage> pCntx = new ProcessorContext<>(mockMsg, mockCollector, mockTaskCoordinator);
- assertEquals(pCntx.getMessage(), mockMsg);
- assertEquals(pCntx.getCollector(), mockCollector);
- assertEquals(pCntx.getCoordinator(), mockTaskCoordinator);
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSessionWindowImpl.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..c302df7
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSessionWindowImpl.java
@@ -0,0 +1,111 @@
+/*
+ * 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.windows.StoreFunctions;
+import org.apache.samza.operators.windows.WindowOutput;
+import org.apache.samza.operators.windows.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<TestMessageEnvelope, String, WindowState<Integer>, WindowOutput<String, Integer>> wndOp = mock(WindowOperatorSpec.class);
+ SessionWindowOperatorImpl<TestMessageEnvelope, String, WindowState<Integer>, WindowOutput<String, Integer>> sessWnd = new SessionWindowOperatorImpl<>(wndOp);
+ assertEquals(wndOp, sessWndField.get(sessWnd));
+ }
+
+ @Test
+ public void testInitAndProcess() throws IllegalAccessException {
+ WindowOperatorSpec<TestMessageEnvelope, String, WindowState<Integer>, WindowOutput<String, Integer>> wndOp = mock(WindowOperatorSpec.class);
+ BiFunction<TestMessageEnvelope, Entry<String, WindowState<Integer>>, WindowOutput<String, Integer>> mockTxfmFn = mock(BiFunction.class);
+ SessionWindowOperatorImpl<TestMessageEnvelope, String, WindowState<Integer>, WindowOutput<String, Integer>> sessWnd = new SessionWindowOperatorImpl<>(wndOp);
+
+ // construct and init the SessionWindowOperatorImpl object
+ MessageStreamImpl<TestMessageEnvelope> mockInputStrm = mock(MessageStreamImpl.class);
+ StoreFunctions<TestMessageEnvelope, String, WindowState<Integer>> mockStoreFns = mock(StoreFunctions.class);
+ Function<TestMessageEnvelope, String> 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<String, WindowState<Integer>> 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<TestMessageEnvelope, WindowState<Integer>, WindowState<Integer>> stateUpdaterFn = mock(BiFunction.class);
+ when(mockStoreFns.getStateUpdaterFn()).thenReturn(stateUpdaterFn);
+ WindowState<Integer> mockNewState = mock(WindowState.class);
+ WindowState<Integer> 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<TestMessageEnvelope>() {
+ @Override public boolean matches(Object argument) {
+ TestMessageEnvelope xIn = (TestMessageEnvelope) argument;
+ return xIn.equals(mockMsg);
+ }
+ }), argThat(new ArgumentMatcher<Entry<String, WindowState<Integer>>>() {
+ @Override public boolean matches(Object argument) {
+ Entry<String, WindowState<Integer>> xIn = (Entry<String, WindowState<Integer>>) 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);
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSimpleOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSimpleOperatorImpl.java b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSimpleOperatorImpl.java
deleted file mode 100644
index de029ea..0000000
--- a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSimpleOperatorImpl.java
+++ /dev/null
@@ -1,55 +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.TestMessage;
-import org.apache.samza.operators.TestOutputMessage;
-import org.apache.samza.operators.internal.Operators.StreamOperator;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskCoordinator;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.function.Function;
-
-import static org.mockito.Mockito.*;
-
-
-public class TestSimpleOperatorImpl {
-
- @Test public void testSimpleOperator() {
- StreamOperator<TestMessage, TestOutputMessage> mockOp = mock(StreamOperator.class);
- Function<TestMessage, Collection<TestOutputMessage>> txfmFn = mock(Function.class);
- when(mockOp.getFunction()).thenReturn(txfmFn);
-
- SimpleOperatorImpl<TestMessage, TestOutputMessage> opImpl = spy(new SimpleOperatorImpl<>(mockOp));
- TestMessage inMsg = mock(TestMessage.class);
- TestOutputMessage outMsg = mock(TestOutputMessage.class);
- Collection<TestOutputMessage> mockOutputs = new ArrayList() { {
- this.add(outMsg);
- } };
- when(txfmFn.apply(inMsg)).thenReturn(mockOutputs);
- MessageCollector mockCollector = mock(MessageCollector.class);
- TaskCoordinator mockCoordinator = mock(TaskCoordinator.class);
- opImpl.onNext(inMsg, mockCollector, mockCoordinator);
- verify(txfmFn, times(1)).apply(inMsg);
- verify(opImpl, times(1)).nextProcessors(outMsg, mockCollector, mockCoordinator);
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSinkOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSinkOperatorImpl.java b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSinkOperatorImpl.java
index cdac3fc..ba5b6f8 100644
--- a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSinkOperatorImpl.java
+++ b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSinkOperatorImpl.java
@@ -18,25 +18,28 @@
*/
package org.apache.samza.operators.impl;
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.TestOutputMessage;
-import org.apache.samza.operators.internal.Operators.SinkOperator;
+import org.apache.samza.operators.TestOutputMessageEnvelope;
+import org.apache.samza.operators.functions.SinkFunction;
+import org.apache.samza.operators.spec.SinkOperatorSpec;
import org.apache.samza.task.MessageCollector;
import org.apache.samza.task.TaskCoordinator;
import org.junit.Test;
-import static org.mockito.Mockito.*;
+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 TestSinkOperatorImpl {
- @Test public void testSinkOperator() {
- SinkOperator<TestOutputMessage> sinkOp = mock(SinkOperator.class);
- MessageStream.VoidFunction3<TestOutputMessage, MessageCollector, TaskCoordinator> sinkFn = mock(
- MessageStream.VoidFunction3.class);
- when(sinkOp.getFunction()).thenReturn(sinkFn);
- SinkOperatorImpl<TestOutputMessage> sinkImpl = new SinkOperatorImpl<>(sinkOp);
- TestOutputMessage mockMsg = mock(TestOutputMessage.class);
+ @Test
+ public void testSinkOperator() {
+ SinkOperatorSpec<TestOutputMessageEnvelope> sinkOp = mock(SinkOperatorSpec.class);
+ SinkFunction<TestOutputMessageEnvelope> sinkFn = mock(SinkFunction.class);
+ when(sinkOp.getSinkFn()).thenReturn(sinkFn);
+ SinkOperatorImpl<TestOutputMessageEnvelope> sinkImpl = new SinkOperatorImpl<>(sinkOp);
+ TestOutputMessageEnvelope mockMsg = mock(TestOutputMessageEnvelope.class);
MessageCollector mockCollector = mock(MessageCollector.class);
TaskCoordinator mockCoordinator = mock(TaskCoordinator.class);
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/impl/TestStateStoreImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestStateStoreImpl.java b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestStateStoreImpl.java
deleted file mode 100644
index 5ede757..0000000
--- a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestStateStoreImpl.java
+++ /dev/null
@@ -1,69 +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.TestMessage;
-import org.apache.samza.operators.WindowState;
-import org.apache.samza.operators.internal.Operators.StoreFunctions;
-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.*;
-
-
-public class TestStateStoreImpl {
- @Test public void testStateStoreImpl() {
- StoreFunctions<TestMessage, String, WindowState> mockStoreFunctions = mock(StoreFunctions.class);
- // test constructor
- StateStoreImpl<TestMessage, String, WindowState> storeImpl = new StateStoreImpl<>(mockStoreFunctions, "myStoreName");
- TaskContext mockContext = mock(TaskContext.class);
- KeyValueStore<String, WindowState> mockKvStore = mock(KeyValueStore.class);
- when(mockContext.getStore("myStoreName")).thenReturn(mockKvStore);
- // test init()
- storeImpl.init(mockContext);
- verify(mockContext, times(1)).getStore("myStoreName");
- Function<TestMessage, String> wndKeyFn = mock(Function.class);
- when(mockStoreFunctions.getStoreKeyFinder()).thenReturn(wndKeyFn);
- TestMessage mockMsg = mock(TestMessage.class);
- when(wndKeyFn.apply(mockMsg)).thenReturn("myKey");
- WindowState mockState = mock(WindowState.class);
- when(mockKvStore.get("myKey")).thenReturn(mockState);
- // test getState()
- Entry<String, WindowState> 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<String, WindowState> oldEntry = new Entry<>("myKey", mockState);
- WindowState mockNewState = mock(WindowState.class);
- BiFunction<TestMessage, WindowState, WindowState> mockUpdaterFn = mock(BiFunction.class);
- when(mockStoreFunctions.getStateUpdater()).thenReturn(mockUpdaterFn);
- when(mockUpdaterFn.apply(mockMsg, mockState)).thenReturn(mockNewState);
- // test updateState()
- Entry<String, WindowState> newEntry = storeImpl.updateState(mockMsg, oldEntry);
- assertEquals(newEntry.getKey(), "myKey");
- assertEquals(newEntry.getValue(), mockNewState);
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/impl/TestStreamOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestStreamOperatorImpl.java b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestStreamOperatorImpl.java
new file mode 100644
index 0000000..5a3840c
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/impl/TestStreamOperatorImpl.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.impl;
+
+import org.apache.samza.operators.TestMessageEnvelope;
+import org.apache.samza.operators.TestOutputMessageEnvelope;
+import org.apache.samza.operators.functions.FlatMapFunction;
+import org.apache.samza.operators.spec.StreamOperatorSpec;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskCoordinator;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collection;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+
+public class TestStreamOperatorImpl {
+
+ @Test
+ public void testSimpleOperator() {
+ StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope> mockOp = mock(StreamOperatorSpec.class);
+ FlatMapFunction<TestMessageEnvelope, TestOutputMessageEnvelope> txfmFn = mock(FlatMapFunction.class);
+ when(mockOp.getTransformFn()).thenReturn(txfmFn);
+
+ StreamOperatorImpl<TestMessageEnvelope, TestOutputMessageEnvelope> opImpl = spy(new StreamOperatorImpl<>(mockOp));
+ TestMessageEnvelope inMsg = mock(TestMessageEnvelope.class);
+ TestOutputMessageEnvelope outMsg = mock(TestOutputMessageEnvelope.class);
+ Collection<TestOutputMessageEnvelope> mockOutputs = new ArrayList() { {
+ this.add(outMsg);
+ } };
+ when(txfmFn.apply(inMsg)).thenReturn(mockOutputs);
+ MessageCollector mockCollector = mock(MessageCollector.class);
+ TaskCoordinator mockCoordinator = mock(TaskCoordinator.class);
+ opImpl.onNext(inMsg, mockCollector, mockCoordinator);
+ verify(txfmFn, times(1)).apply(inMsg);
+ verify(opImpl, times(1)).propagateResult(outMsg, mockCollector, mockCoordinator);
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/impl/window/TestSessionWindowImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/impl/window/TestSessionWindowImpl.java b/samza-operator/src/test/java/org/apache/samza/operators/impl/window/TestSessionWindowImpl.java
deleted file mode 100644
index 719ab99..0000000
--- a/samza-operator/src/test/java/org/apache/samza/operators/impl/window/TestSessionWindowImpl.java
+++ /dev/null
@@ -1,105 +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.window;
-
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.TestMessage;
-import org.apache.samza.operators.WindowState;
-import org.apache.samza.operators.internal.Operators.StoreFunctions;
-import org.apache.samza.operators.internal.Operators.WindowOperator;
-import org.apache.samza.operators.internal.WindowOutput;
-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.*;
-
-
-public class TestSessionWindowImpl {
- Field wndStoreField = null;
- Field sessWndField = null;
-
- @Before public void prep() throws NoSuchFieldException {
- wndStoreField = SessionWindowImpl.class.getDeclaredField("wndStore");
- sessWndField = SessionWindowImpl.class.getDeclaredField("sessWnd");
- wndStoreField.setAccessible(true);
- sessWndField.setAccessible(true);
- }
-
- @Test public void testConstructor() throws IllegalAccessException, NoSuchFieldException {
- // test constructing a SessionWindowImpl w/ expected mock functions
- WindowOperator<TestMessage, String, WindowState<Integer>, WindowOutput<String, Integer>> wndOp = mock(WindowOperator.class);
- SessionWindowImpl<TestMessage, String, WindowState<Integer>, WindowOutput<String, Integer>> sessWnd = new SessionWindowImpl<>(wndOp);
- assertEquals(wndOp, sessWndField.get(sessWnd));
- }
-
- @Test public void testInitAndProcess() throws IllegalAccessException {
- WindowOperator<TestMessage, String, WindowState<Integer>, WindowOutput<String, Integer>> wndOp = mock(WindowOperator.class);
- BiFunction<TestMessage, Entry<String, WindowState<Integer>>, WindowOutput<String, Integer>> mockTxfmFn = mock(BiFunction.class);
- SessionWindowImpl<TestMessage, String, WindowState<Integer>, WindowOutput<String, Integer>> sessWnd = new SessionWindowImpl<>(wndOp);
-
- // construct and init the SessionWindowImpl object
- MessageStream<TestMessage> mockInputStrm = mock(MessageStream.class);
- StoreFunctions<TestMessage, String, WindowState<Integer>> mockStoreFns = mock(StoreFunctions.class);
- Function<TestMessage, String> wndKeyFn = m -> "test-msg-key";
- when(mockStoreFns.getStoreKeyFinder()).thenReturn(wndKeyFn);
- when(wndOp.getStoreFunctions()).thenReturn(mockStoreFns);
- when(wndOp.getStoreName(mockInputStrm)).thenReturn("test-wnd-store");
- when(wndOp.getFunction()).thenReturn(mockTxfmFn);
- TaskContext mockContext = mock(TaskContext.class);
- KeyValueStore<String, WindowState<Integer>> 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.
- TestMessage mockMsg = mock(TestMessage.class);
- MessageCollector mockCollector = mock(MessageCollector.class);
- TaskCoordinator mockCoordinator = mock(TaskCoordinator.class);
- BiFunction<TestMessage, WindowState<Integer>, WindowState<Integer>> stateUpdaterFn = mock(BiFunction.class);
- when(mockStoreFns.getStateUpdater()).thenReturn(stateUpdaterFn);
- WindowState<Integer> mockNewState = mock(WindowState.class);
- WindowState<Integer> 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<TestMessage>() {
- @Override public boolean matches(Object argument) {
- TestMessage xIn = (TestMessage) argument;
- return xIn.equals(mockMsg);
- }
- }), argThat(new ArgumentMatcher<Entry<String, WindowState<Integer>>>() {
- @Override public boolean matches(Object argument) {
- Entry<String, WindowState<Integer>> xIn = (Entry<String, WindowState<Integer>>) 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);
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/spec/TestOperatorSpecs.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..028bd67
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/spec/TestOperatorSpecs.java
@@ -0,0 +1,114 @@
+/*
+ * 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.TestMessageEnvelope;
+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.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;
+
+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.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+
+public class TestOperatorSpecs {
+ @Test
+ public void testGetStreamOperator() {
+ FlatMapFunction<MessageEnvelope, TestMessageEnvelope> transformFn = m -> new ArrayList<TestMessageEnvelope>() { {
+ this.add(new TestMessageEnvelope(m.getKey().toString(), m.getMessage().toString(), 12345L));
+ } };
+ StreamOperatorSpec<MessageEnvelope, TestMessageEnvelope> strmOp = OperatorSpecs.createStreamOperator(transformFn);
+ assertEquals(strmOp.getTransformFn(), transformFn);
+ assertTrue(strmOp.getOutputStream() instanceof MessageStreamImpl);
+ }
+
+ @Test
+ public void testGetSinkOperator() {
+ SinkFunction<TestMessageEnvelope> sinkFn = (m, c, t) -> { };
+ SinkOperatorSpec<TestMessageEnvelope> sinkOp = OperatorSpecs.createSinkOperator(sinkFn);
+ assertEquals(sinkOp.getSinkFn(), sinkFn);
+ assertTrue(sinkOp.getOutputStream() == null);
+ }
+
+ @Test
+ public void testGetWindowOperator() {
+ WindowFn<TestMessageEnvelope, String, WindowState<Integer>, WindowOutput<String, Integer>> windowFn = mock(WindowFn.class);
+ BiFunction<TestMessageEnvelope, Entry<String, WindowState<Integer>>, WindowOutput<String, Integer>> xFunction = (m, e) -> null;
+ StoreFunctions<TestMessageEnvelope, String, WindowState<Integer>> storeFns = mock(StoreFunctions.class);
+ Trigger<TestMessageEnvelope, WindowState<Integer>> trigger = mock(Trigger.class);
+ MessageStreamImpl<TestMessageEnvelope> 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<TestMessageEnvelope, String, WindowState<Integer>, WindowOutput<String, Integer>> 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()));
+ }
+
+ @Test
+ public void testGetPartialJoinOperator() {
+ BiFunction<MessageEnvelope<Object, ?>, MessageEnvelope<Object, ?>, TestMessageEnvelope> merger =
+ (m1, m2) -> new TestMessageEnvelope(m1.getKey().toString(), m2.getMessage().toString(), System.nanoTime());
+ MessageStreamImpl<TestMessageEnvelope> joinOutput = new MessageStreamImpl<>();
+ PartialJoinOperatorSpec<MessageEnvelope<Object, ?>, Object, MessageEnvelope<Object, ?>, TestMessageEnvelope> partialJoin =
+ OperatorSpecs.createPartialJoinOperator(merger, joinOutput);
+
+ assertEquals(partialJoin.getOutputStream(), joinOutput);
+ MessageEnvelope<Object, Object> m = mock(MessageEnvelope.class);
+ MessageEnvelope<Object, Object> 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
+ public void testGetMergeOperator() {
+ MessageStreamImpl<TestMessageEnvelope> output = new MessageStreamImpl<>();
+ StreamOperatorSpec<TestMessageEnvelope, TestMessageEnvelope> mergeOp = OperatorSpecs.createMergeOperator(output);
+ Function<TestMessageEnvelope, Collection<TestMessageEnvelope>> mergeFn = t -> new ArrayList<TestMessageEnvelope>() { {
+ this.add(t);
+ } };
+ TestMessageEnvelope t = mock(TestMessageEnvelope.class);
+ assertEquals(mergeOp.getTransformFn().apply(t), mergeFn.apply(t));
+ assertEquals(mergeOp.getOutputStream(), output);
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/task/BroadcastOperatorTask.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/task/BroadcastOperatorTask.java b/samza-operator/src/test/java/org/apache/samza/task/BroadcastOperatorTask.java
deleted file mode 100644
index d1b0a88..0000000
--- a/samza-operator/src/test/java/org/apache/samza/task/BroadcastOperatorTask.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.task;
-
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.MessageStreams.SystemMessageStream;
-import org.apache.samza.operators.TriggerBuilder;
-import org.apache.samza.operators.Windows;
-import org.apache.samza.operators.data.IncomingSystemMessage;
-import org.apache.samza.operators.data.Offset;
-import org.apache.samza.operators.task.StreamOperatorTask;
-import org.apache.samza.system.SystemStreamPartition;
-
-import java.util.Collection;
-
-
-/**
- * Example implementation of split stream tasks
- *
- */
-public class BroadcastOperatorTask implements StreamOperatorTask {
- class MessageType {
- String field1;
- String field2;
- String field3;
- String field4;
- String parKey;
- private long timestamp;
-
- public long getTimestamp() {
- return this.timestamp;
- }
- }
-
- class JsonMessage extends InputJsonSystemMessage<MessageType> {
-
- JsonMessage(String key, MessageType data, Offset offset, long timestamp, SystemStreamPartition partition) {
- super(key, data, offset, timestamp, partition);
- }
- }
-
- @Override public void initOperators(Collection<SystemMessageStream> sources) {
- sources.forEach(source -> {
- MessageStream<JsonMessage> inputStream = source.map(this::getInputMessage);
-
- inputStream.filter(this::myFilter1).
- window(Windows.<JsonMessage, String>intoSessionCounter(
- m -> String.format("%s-%s", m.getMessage().field1, m.getMessage().field2)).
- setTriggers(TriggerBuilder.<JsonMessage, Integer>earlyTriggerWhenExceedWndLen(100).
- addLateTriggerOnSizeLimit(10).
- addTimeoutSinceLastMessage(30000)));
-
- inputStream.filter(this::myFilter2).
- window(Windows.<JsonMessage, String>intoSessions(
- m -> String.format("%s-%s", m.getMessage().field3, m.getMessage().field4)).
- setTriggers(TriggerBuilder.<JsonMessage, Collection<JsonMessage>>earlyTriggerWhenExceedWndLen(100).
- addTimeoutSinceLastMessage(30000)));
-
- inputStream.filter(this::myFilter3).
- window(Windows.<JsonMessage, String, MessageType>intoSessions(
- m -> String.format("%s-%s", m.getMessage().field3, m.getMessage().field4), m -> m.getMessage()).
- setTriggers(TriggerBuilder.<JsonMessage, Collection<MessageType>>earlyTriggerOnEventTime(m -> m.getTimestamp(), 30000).
- addTimeoutSinceFirstMessage(60000)));
- }
- );
- }
-
- JsonMessage getInputMessage(IncomingSystemMessage m1) {
- return (JsonMessage) m1.getMessage();
- }
-
- boolean myFilter1(JsonMessage m1) {
- // Do user defined processing here
- return m1.getMessage().parKey.equals("key1");
- }
-
- boolean myFilter2(JsonMessage m1) {
- // Do user defined processing here
- return m1.getMessage().parKey.equals("key2");
- }
-
- boolean myFilter3(JsonMessage m1) {
- return m1.getMessage().parKey.equals("key3");
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/task/InputJsonSystemMessage.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/task/InputJsonSystemMessage.java b/samza-operator/src/test/java/org/apache/samza/task/InputJsonSystemMessage.java
deleted file mode 100644
index 88aa159..0000000
--- a/samza-operator/src/test/java/org/apache/samza/task/InputJsonSystemMessage.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.task;
-
-import org.apache.samza.operators.data.InputSystemMessage;
-import org.apache.samza.operators.data.Message;
-import org.apache.samza.operators.data.Offset;
-import org.apache.samza.system.SystemStreamPartition;
-
-
-/**
- * Example input message w/ Json message body and string as the key.
- */
-
-public class InputJsonSystemMessage<T> implements Message<String, T>, InputSystemMessage<Offset> {
-
- private final String key;
- private final T data;
- private final Offset offset;
- private final long timestamp;
- private final SystemStreamPartition partition;
-
- InputJsonSystemMessage(String key, T data, Offset offset, long timestamp, SystemStreamPartition partition) {
- this.key = key;
- this.data = data;
- this.offset = offset;
- this.timestamp = timestamp;
- this.partition = partition;
- }
-
- @Override public T getMessage() {
- return this.data;
- }
-
- @Override public String getKey() {
- return this.key;
- }
-
- @Override public long getTimestamp() {
- return this.timestamp;
- }
-
- @Override public Offset getOffset() {
- return this.offset;
- }
-
- @Override public SystemStreamPartition getSystemStreamPartition() {
- return this.partition;
- }
-}
-
[4/6] samza git commit: SAMZA-1054: Refactor Operator APIs
Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/test/java/org/apache/samza/operators/TestOutputMessage.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/TestOutputMessage.java b/samza-api/src/test/java/org/apache/samza/operators/TestOutputMessage.java
deleted file mode 100644
index 14e6562..0000000
--- a/samza-api/src/test/java/org/apache/samza/operators/TestOutputMessage.java
+++ /dev/null
@@ -1,47 +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.Message;
-
-
-public class TestOutputMessage implements Message<String, Integer> {
- private final String key;
- private final Integer value;
- private final long timestamp;
-
- public TestOutputMessage(String key, Integer value, long timestamp) {
- this.key = key;
- this.value = value;
- this.timestamp = timestamp;
- }
-
- @Override public Integer getMessage() {
- return this.value;
- }
-
- @Override public String getKey() {
- return this.key;
- }
-
- @Override public long getTimestamp() {
- return this.timestamp;
- }
-}
-
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/test/java/org/apache/samza/operators/TestOutputMessageEnvelope.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/TestOutputMessageEnvelope.java b/samza-api/src/test/java/org/apache/samza/operators/TestOutputMessageEnvelope.java
new file mode 100644
index 0000000..284b30b
--- /dev/null
+++ b/samza-api/src/test/java/org/apache/samza/operators/TestOutputMessageEnvelope.java
@@ -0,0 +1,43 @@
+/*
+ * 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;
+
+
+public class TestOutputMessageEnvelope implements MessageEnvelope<String, Integer> {
+ private final String key;
+ private final Integer value;
+
+ public TestOutputMessageEnvelope(String key, Integer value) {
+ this.key = key;
+ this.value = value;
+ }
+
+ @Override
+ public Integer getMessage() {
+ return this.value;
+ }
+
+ @Override
+ public String getKey() {
+ return this.key;
+ }
+}
+
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/test/java/org/apache/samza/operators/TestTriggerBuilder.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/TestTriggerBuilder.java b/samza-api/src/test/java/org/apache/samza/operators/TestTriggerBuilder.java
deleted file mode 100644
index 927b14b..0000000
--- a/samza-api/src/test/java/org/apache/samza/operators/TestTriggerBuilder.java
+++ /dev/null
@@ -1,214 +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.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<TestMessage, Collection<TestMessage>> builder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000);
- BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean> triggerField =
- (BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean>) this.earlyTriggerField.get(builder);
- WindowState<Collection<TestMessage>> 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<TestMessage, Boolean> tokenFunc = m -> true;
- builder = TriggerBuilder.earlyTriggerOnTokenMsg(tokenFunc);
- triggerField = (BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean>) this.earlyTriggerField.get(builder);
- TestMessage m = mock(TestMessage.class);
- assertTrue(triggerField.apply(m, mockState));
-
- builder = TriggerBuilder.earlyTriggerOnEventTime(TestMessage::getTimestamp, 30000L);
- triggerField = (BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean>) this.earlyTriggerField.get(builder);
- when(mockState.getEarliestEventTimeNs()).thenReturn(1000000000L);
- when(mockState.getLatestEventTimeNs()).thenReturn(20000000000L);
- when(m.getTimestamp()).thenReturn(19999000000L);
- assertFalse(triggerField.apply(m, mockState));
- when(m.getTimestamp()).thenReturn(32000000000L);
- assertTrue(triggerField.apply(m, mockState));
- when(m.getTimestamp()).thenReturn(1001000000L);
- when(mockState.getLatestEventTimeNs()).thenReturn(32000000000L);
- assertTrue(triggerField.apply(m, mockState));
-
- BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean> mockFunc = mock(BiFunction.class);
- builder = TriggerBuilder.earlyTrigger(mockFunc);
- triggerField = (BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean>) this.earlyTriggerField.get(builder);
- assertEquals(triggerField, mockFunc);
-
- builder = TriggerBuilder.timeoutSinceFirstMessage(10000L);
- Function<WindowState<Collection<TestMessage>>, Boolean> timerTrigger =
- (Function<WindowState<Collection<TestMessage>>, 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<WindowState<Collection<TestMessage>>, 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<TestMessage, Collection<TestMessage>> builder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000);
- builder.addTimeoutSinceFirstMessage(10000L);
- // exam that both earlyTrigger and timer triggers are set up
- BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean> triggerField =
- (BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean>) this.earlyTriggerField.get(builder);
- WindowState<Collection<TestMessage>> mockState = mock(WindowState.class);
- when(mockState.getNumberMessages()).thenReturn(200L);
- assertFalse(triggerField.apply(null, mockState));
- // check the timer trigger
- Function<WindowState<Collection<TestMessage>>, Boolean> timerTrigger =
- (Function<WindowState<Collection<TestMessage>>, 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<TestMessage, WindowState<Collection<TestMessage>>, 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<WindowState<Collection<TestMessage>>, 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<TestMessage, Collection<TestMessage>> builder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000);
- builder.addLateTriggerOnSizeLimit(10000L);
- // exam that both earlyTrigger and lateTriggers are set up
- BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean> earlyTrigger =
- (BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean>) this.earlyTriggerField.get(builder);
- WindowState<Collection<TestMessage>> mockState = mock(WindowState.class);
- when(mockState.getNumberMessages()).thenReturn(200L);
- assertFalse(earlyTrigger.apply(null, mockState));
- // check the late trigger
- BiFunction<TestMessage, WindowState<Collection<TestMessage>>, Boolean> lateTrigger =
- (BiFunction<TestMessage, WindowState<Collection<TestMessage>>, 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<TestMessage, WindowState<Collection<TestMessage>>, 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<TestMessage, WindowState<Collection<TestMessage>>, Boolean>) this.lateTriggerField.get(builder);
- assertFalse(lateTrigger.apply(null, mockState));
- List<TestMessage> 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<TestMessage, Collection<TestMessage>> builder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000);
- builder.onEarlyTrigger(c -> {
- c.clear();
- return c;
- });
- List<TestMessage> collection = new ArrayList<TestMessage>() { {
- for (int i = 0; i < 10; i++) {
- this.add(new TestMessage(String.format("key-%d", i), "string-value", System.nanoTime()));
- }
- } };
- // exam that earlyTriggerUpdater is set up
- Function<WindowState<Collection<TestMessage>>, WindowState<Collection<TestMessage>>> earlyTriggerUpdater =
- (Function<WindowState<Collection<TestMessage>>, WindowState<Collection<TestMessage>>>) this.earlyTriggerUpdater.get(builder);
- WindowState<Collection<TestMessage>> mockState = mock(WindowState.class);
- when(mockState.getOutputValue()).thenReturn(collection);
- earlyTriggerUpdater.apply(mockState);
- assertTrue(collection.isEmpty());
-
- collection.add(new TestMessage("key-to-stay", "string-to-stay", System.nanoTime()));
- collection.add(new TestMessage("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<Collection<TestMessage>>, WindowState<Collection<TestMessage>>> lateTriggerUpdater =
- (Function<WindowState<Collection<TestMessage>>, WindowState<Collection<TestMessage>>>) 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");
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/test/java/org/apache/samza/operators/TestWindows.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/TestWindows.java b/samza-api/src/test/java/org/apache/samza/operators/TestWindows.java
deleted file mode 100644
index 8a25a96..0000000
--- a/samza-api/src/test/java/org/apache/samza/operators/TestWindows.java
+++ /dev/null
@@ -1,106 +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.Windows.Window;
-import org.apache.samza.operators.internal.Trigger;
-import org.apache.samza.operators.internal.WindowOutput;
-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<TestMessage, String, Collection<TestMessage>, WindowOutput<String, Collection<TestMessage>>> testWnd = Windows.intoSessions(
- TestMessage::getKey);
- assertTrue(testWnd instanceof Windows.SessionWindow);
- Field wndKeyFuncField = Windows.SessionWindow.class.getDeclaredField("wndKeyFunction");
- Field aggregatorField = Windows.SessionWindow.class.getDeclaredField("aggregator");
- wndKeyFuncField.setAccessible(true);
- aggregatorField.setAccessible(true);
- Function<TestMessage, String> wndKeyFunc = (Function<TestMessage, String>) wndKeyFuncField.get(testWnd);
- assertEquals(wndKeyFunc.apply(new TestMessage("test-key", "test-value", 0)), "test-key");
- BiFunction<TestMessage, Collection<TestMessage>, Collection<TestMessage>> aggrFunc =
- (BiFunction<TestMessage, Collection<TestMessage>, Collection<TestMessage>>) aggregatorField.get(testWnd);
- TestMessage mockMsg = mock(TestMessage.class);
- Collection<TestMessage> collection = aggrFunc.apply(mockMsg, new ArrayList<>());
- assertTrue(collection.size() == 1);
- assertTrue(collection.contains(mockMsg));
-
- // test constructing the session window w/ customized session info
- Window<TestMessage, String, Collection<Character>, WindowOutput<String, Collection<Character>>> testWnd2 = Windows.intoSessions(
- m -> String.format("key-%d", m.getTimestamp()), m -> m.getMessage().charAt(0));
- assertTrue(testWnd2 instanceof Windows.SessionWindow);
- wndKeyFunc = (Function<TestMessage, String>) wndKeyFuncField.get(testWnd2);
- aggrFunc = (BiFunction<TestMessage, Collection<TestMessage>, Collection<TestMessage>>) aggregatorField.get(testWnd2);
- assertEquals(wndKeyFunc.apply(new TestMessage("test-key", "test-value", 0)), "key-0");
- when(mockMsg.getMessage()).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<TestMessage, String, Integer, WindowOutput<String, Integer>> testCounter = Windows.intoSessionCounter(
- m -> String.format("key-%d", m.getTimestamp()));
- assertTrue(testCounter instanceof Windows.SessionWindow);
- wndKeyFunc = (Function<TestMessage, String>) wndKeyFuncField.get(testCounter);
- BiFunction<TestMessage, Integer, Integer> counterFn = (BiFunction<TestMessage, Integer, Integer>) aggregatorField.get(testCounter);
- when(mockMsg.getTimestamp()).thenReturn(12345L);
- assertEquals(wndKeyFunc.apply(mockMsg), "key-12345");
- assertEquals(counterFn.apply(mockMsg, 1), Integer.valueOf(2));
- }
-
- @Test public void testSetTriggers() throws NoSuchFieldException, IllegalAccessException {
- Window<TestMessage, String, Integer, WindowOutput<String, Integer>> testCounter = Windows.intoSessionCounter(
- m -> String.format("key-%d", m.getTimestamp()));
- // test session window w/ a trigger
- TriggerBuilder<TestMessage, Integer> triggerBuilder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000L);
- testCounter.setTriggers(triggerBuilder);
- Trigger<TestMessage, WindowState<Integer>> expectedTrigger = triggerBuilder.build();
- Trigger<TestMessage, WindowState<Integer>> 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));
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/test/java/org/apache/samza/operators/data/TestIncomingSystemMessage.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/data/TestIncomingSystemMessage.java b/samza-api/src/test/java/org/apache/samza/operators/data/TestIncomingSystemMessage.java
index b734e87..7bd62a7 100644
--- a/samza-api/src/test/java/org/apache/samza/operators/data/TestIncomingSystemMessage.java
+++ b/samza-api/src/test/java/org/apache/samza/operators/data/TestIncomingSystemMessage.java
@@ -30,9 +30,10 @@ import static org.mockito.Mockito.when;
public class TestIncomingSystemMessage {
- @Test public void testConstructor() {
+ @Test
+ public void testConstructor() {
IncomingMessageEnvelope ime = mock(IncomingMessageEnvelope.class);
- IncomingSystemMessage ism = new IncomingSystemMessage(ime);
+ IncomingSystemMessageEnvelope ism = new IncomingSystemMessageEnvelope(ime);
Object mockKey = mock(Object.class);
Object mockValue = mock(Object.class);
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/test/java/org/apache/samza/operators/data/TestLongOffset.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/data/TestLongOffset.java b/samza-api/src/test/java/org/apache/samza/operators/data/TestLongOffset.java
index 943c47f..7838896 100644
--- a/samza-api/src/test/java/org/apache/samza/operators/data/TestLongOffset.java
+++ b/samza-api/src/test/java/org/apache/samza/operators/data/TestLongOffset.java
@@ -28,7 +28,8 @@ import static org.mockito.Mockito.mock;
public class TestLongOffset {
- @Test public void testConstructor() throws Exception {
+ @Test
+ public void testConstructor() throws Exception {
LongOffset o1 = new LongOffset("12345");
Field offsetField = LongOffset.class.getDeclaredField("offset");
offsetField.setAccessible(true);
@@ -47,7 +48,8 @@ public class TestLongOffset {
}
}
- @Test public void testComparator() {
+ @Test
+ public void testComparator() {
LongOffset o1 = new LongOffset("11111");
Offset other = mock(Offset.class);
try {
@@ -65,7 +67,8 @@ public class TestLongOffset {
assertEquals(o1.compareTo(o4), 0);
}
- @Test public void testEquals() {
+ @Test
+ public void testEquals() {
LongOffset o1 = new LongOffset("12345");
Offset other = mock(Offset.class);
assertFalse(o1.equals(other));
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/test/java/org/apache/samza/operators/internal/TestOperators.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/internal/TestOperators.java b/samza-api/src/test/java/org/apache/samza/operators/internal/TestOperators.java
deleted file mode 100644
index d994486..0000000
--- a/samza-api/src/test/java/org/apache/samza/operators/internal/TestOperators.java
+++ /dev/null
@@ -1,128 +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.internal;
-
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.WindowState;
-import org.apache.samza.operators.data.Message;
-import org.apache.samza.storage.kv.Entry;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskCoordinator;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.function.BiFunction;
-import java.util.function.Function;
-
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-
-public class TestOperators {
-
- private class TestMessage implements Message<String, Object> {
- private final long timestamp;
- private final String key;
- private final Object msg;
-
-
- TestMessage(String key, Object msg, long timestamp) {
- this.timestamp = timestamp;
- this.key = key;
- this.msg = msg;
- }
-
- @Override public Object getMessage() {
- return this.msg;
- }
-
- @Override public String getKey() {
- return this.key;
- }
-
- @Override public long getTimestamp() {
- return this.timestamp;
- }
- }
-
- @Test public void testGetStreamOperator() {
- Function<Message, Collection<TestMessage>> transformFn = m -> new ArrayList<TestMessage>() { {
- this.add(new TestMessage(m.getKey().toString(), m.getMessage(), 12345L));
- } };
- Operators.StreamOperator<Message, TestMessage> strmOp = Operators.getStreamOperator(transformFn);
- assertEquals(strmOp.getFunction(), transformFn);
- assertTrue(strmOp.getOutputStream() instanceof MessageStream);
- }
-
- @Test public void testGetSinkOperator() {
- MessageStream.VoidFunction3<TestMessage, MessageCollector, TaskCoordinator> sinkFn = (m, c, t) -> { };
- Operators.SinkOperator<TestMessage> sinkOp = Operators.getSinkOperator(sinkFn);
- assertEquals(sinkOp.getFunction(), sinkFn);
- assertTrue(sinkOp.getOutputStream() == null);
- }
-
- @Test public void testGetWindowOperator() {
- WindowFn<TestMessage, String, WindowState<Integer>, WindowOutput<String, Integer>> windowFn = mock(WindowFn.class);
- BiFunction<TestMessage, Entry<String, WindowState<Integer>>, WindowOutput<String, Integer>> xFunction = (m, e) -> null;
- Operators.StoreFunctions<TestMessage, String, WindowState<Integer>> storeFns = mock(Operators.StoreFunctions.class);
- Trigger<TestMessage, WindowState<Integer>> trigger = mock(Trigger.class);
- MessageStream<TestMessage> mockInput = mock(MessageStream.class);
- when(windowFn.getTransformFunc()).thenReturn(xFunction);
- when(windowFn.getStoreFuncs()).thenReturn(storeFns);
- when(windowFn.getTrigger()).thenReturn(trigger);
- when(mockInput.toString()).thenReturn("mockStream1");
-
- Operators.WindowOperator<TestMessage, String, WindowState<Integer>, WindowOutput<String, Integer>> windowOp = Operators.getWindowOperator(windowFn);
- assertEquals(windowOp.getFunction(), xFunction);
- assertEquals(windowOp.getStoreFunctions(), storeFns);
- assertEquals(windowOp.getTrigger(), trigger);
- assertEquals(windowOp.getStoreName(mockInput), String.format("input-mockStream1-wndop-%s", windowOp.toString()));
- }
-
- @Test public void testGetPartialJoinOperator() {
- BiFunction<Message<Object, ?>, Message<Object, ?>, TestMessage> merger =
- (m1, m2) -> new TestMessage(m1.getKey().toString(), m2.getMessage(),
- Math.max(m1.getTimestamp(), m2.getTimestamp()));
- MessageStream<TestMessage> joinOutput = new MessageStream<>();
- Operators.PartialJoinOperator<Message<Object, ?>, Object, Message<Object, ?>, TestMessage> partialJoin =
- Operators.getPartialJoinOperator(merger, joinOutput);
-
- assertEquals(partialJoin.getOutputStream(), joinOutput);
- Message<Object, Object> m = mock(Message.class);
- Message<Object, Object> s = mock(Message.class);
- assertEquals(partialJoin.getFunction(), merger);
- assertEquals(partialJoin.getSelfStoreFunctions().getStoreKeyFinder().apply(m), m.getKey());
- assertEquals(partialJoin.getSelfStoreFunctions().getStateUpdater().apply(m, s), m);
- assertEquals(partialJoin.getJoinStoreFunctions().getStoreKeyFinder().apply(m), m.getKey());
- assertNull(partialJoin.getJoinStoreFunctions().getStateUpdater());
- }
-
- @Test public void testGetMergeOperator() {
- MessageStream<TestMessage> output = new MessageStream<>();
- Operators.StreamOperator<TestMessage, TestMessage> mergeOp = Operators.getMergeOperator(output);
- Function<TestMessage, Collection<TestMessage>> mergeFn = t -> new ArrayList<TestMessage>() { {
- this.add(t);
- } };
- TestMessage t = mock(TestMessage.class);
- assertEquals(mergeOp.getFunction().apply(t), mergeFn.apply(t));
- assertEquals(mergeOp.getOutputStream(), output);
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/test/java/org/apache/samza/operators/internal/TestTrigger.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/internal/TestTrigger.java b/samza-api/src/test/java/org/apache/samza/operators/internal/TestTrigger.java
deleted file mode 100644
index 0f35a7c..0000000
--- a/samza-api/src/test/java/org/apache/samza/operators/internal/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.internal;
-
-import org.apache.samza.operators.WindowState;
-import org.apache.samza.operators.data.Message;
-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<Message<Object, Object>, WindowState<Integer>, Boolean> earlyTrigger = (m, s) -> s.getOutputValue() > 1000;
- BiFunction<Message<Object, Object>, WindowState<Integer>, Boolean> lateTrigger = (m, s) -> s.getOutputValue() > 1000;
- Function<WindowState<Integer>, Boolean> timerTrigger = s -> TimeUnit.NANOSECONDS.toMillis(s.getLastMessageTimeNs()) + 50000 < System.currentTimeMillis();
- Function<WindowState<Integer>, WindowState<Integer>> earlyTriggerUpdater = s -> {
- s.setOutputValue(0);
- return s;
- };
- Function<WindowState<Integer>, WindowState<Integer>> lateTriggerUpdater = s -> {
- s.setOutputValue(1);
- return s;
- };
-
- Trigger<Message<Object, Object>, WindowState<Integer>> 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));
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/test/java/org/apache/samza/operators/internal/TestWindowOutput.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/internal/TestWindowOutput.java b/samza-api/src/test/java/org/apache/samza/operators/internal/TestWindowOutput.java
deleted file mode 100644
index 268c9fc..0000000
--- a/samza-api/src/test/java/org/apache/samza/operators/internal/TestWindowOutput.java
+++ /dev/null
@@ -1,36 +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.internal;
-
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-
-
-public class TestWindowOutput {
-
- @Test public void testConstructor() {
- WindowOutput<String, Integer> wndOutput = WindowOutput.of("testMsg", 10);
- assertEquals(wndOutput.getKey(), "testMsg");
- assertEquals(wndOutput.getMessage(), Integer.valueOf(10));
- assertFalse(wndOutput.isDelete());
- assertEquals(wndOutput.getTimestamp(), 0);
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/test/java/org/apache/samza/operators/windows/TestTrigger.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..a91af24
--- /dev/null
+++ b/samza-api/src/test/java/org/apache/samza/operators/windows/TestTrigger.java
@@ -0,0 +1,68 @@
+/*
+ * 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<MessageEnvelope<Object, Object>, WindowState<Integer>, Boolean> earlyTrigger = (m, s) -> s.getOutputValue() > 1000;
+ BiFunction<MessageEnvelope<Object, Object>, WindowState<Integer>, Boolean> lateTrigger = (m, s) -> s.getOutputValue() > 1000;
+ Function<WindowState<Integer>, Boolean> timerTrigger = s -> TimeUnit.NANOSECONDS.toMillis(s.getLastMessageTimeNs()) + 50000 < System.currentTimeMillis();
+ Function<WindowState<Integer>, WindowState<Integer>> earlyTriggerUpdater = s -> {
+ s.setOutputValue(0);
+ return s;
+ };
+ Function<WindowState<Integer>, WindowState<Integer>> lateTriggerUpdater = s -> {
+ s.setOutputValue(1);
+ return s;
+ };
+
+ Trigger<MessageEnvelope<Object, Object>, WindowState<Integer>> 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));
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/test/java/org/apache/samza/operators/windows/TestTriggerBuilder.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..6a9b55d
--- /dev/null
+++ b/samza-api/src/test/java/org/apache/samza/operators/windows/TestTriggerBuilder.java
@@ -0,0 +1,226 @@
+/*
+ * 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<TestMessageEnvelope, Collection<TestMessageEnvelope>> builder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000);
+ BiFunction<TestMessageEnvelope, WindowState<Collection<TestMessageEnvelope>>, Boolean> triggerField =
+ (BiFunction<TestMessageEnvelope, WindowState<Collection<TestMessageEnvelope>>, Boolean>) this.earlyTriggerField.get(builder);
+ WindowState<Collection<TestMessageEnvelope>> 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<TestMessageEnvelope, Boolean> tokenFunc = m -> true;
+ builder = TriggerBuilder.earlyTriggerOnTokenMsg(tokenFunc);
+ triggerField = (BiFunction<TestMessageEnvelope, WindowState<Collection<TestMessageEnvelope>>, 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<TestMessageEnvelope, WindowState<Collection<TestMessageEnvelope>>, 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<TestMessageEnvelope, WindowState<Collection<TestMessageEnvelope>>, Boolean> mockFunc = mock(BiFunction.class);
+ builder = TriggerBuilder.earlyTrigger(mockFunc);
+ triggerField = (BiFunction<TestMessageEnvelope, WindowState<Collection<TestMessageEnvelope>>, Boolean>) this.earlyTriggerField.get(builder);
+ assertEquals(triggerField, mockFunc);
+
+ builder = TriggerBuilder.timeoutSinceFirstMessage(10000L);
+ Function<WindowState<Collection<TestMessageEnvelope>>, Boolean> timerTrigger =
+ (Function<WindowState<Collection<TestMessageEnvelope>>, 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<WindowState<Collection<TestMessageEnvelope>>, 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<TestMessageEnvelope, Collection<TestMessageEnvelope>> builder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000);
+ builder.addTimeoutSinceFirstMessage(10000L);
+ // exam that both earlyTrigger and timer triggers are set up
+ BiFunction<TestMessageEnvelope, WindowState<Collection<TestMessageEnvelope>>, Boolean> triggerField =
+ (BiFunction<TestMessageEnvelope, WindowState<Collection<TestMessageEnvelope>>, Boolean>) this.earlyTriggerField.get(builder);
+ WindowState<Collection<TestMessageEnvelope>> mockState = mock(WindowState.class);
+ when(mockState.getNumberMessages()).thenReturn(200L);
+ assertFalse(triggerField.apply(null, mockState));
+ // check the timer trigger
+ Function<WindowState<Collection<TestMessageEnvelope>>, Boolean> timerTrigger =
+ (Function<WindowState<Collection<TestMessageEnvelope>>, 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<TestMessageEnvelope, WindowState<Collection<TestMessageEnvelope>>, 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<WindowState<Collection<TestMessageEnvelope>>, 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<TestMessageEnvelope, Collection<TestMessageEnvelope>> builder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000);
+ builder.addLateTriggerOnSizeLimit(10000L);
+ // exam that both earlyTrigger and lateTriggers are set up
+ BiFunction<TestMessageEnvelope, WindowState<Collection<TestMessageEnvelope>>, Boolean> earlyTrigger =
+ (BiFunction<TestMessageEnvelope, WindowState<Collection<TestMessageEnvelope>>, Boolean>) this.earlyTriggerField.get(builder);
+ WindowState<Collection<TestMessageEnvelope>> mockState = mock(WindowState.class);
+ when(mockState.getNumberMessages()).thenReturn(200L);
+ assertFalse(earlyTrigger.apply(null, mockState));
+ // check the late trigger
+ BiFunction<TestMessageEnvelope, WindowState<Collection<TestMessageEnvelope>>, Boolean> lateTrigger =
+ (BiFunction<TestMessageEnvelope, WindowState<Collection<TestMessageEnvelope>>, 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<TestMessageEnvelope, WindowState<Collection<TestMessageEnvelope>>, 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<TestMessageEnvelope, WindowState<Collection<TestMessageEnvelope>>, Boolean>) this.lateTriggerField.get(builder);
+ assertFalse(lateTrigger.apply(null, mockState));
+ List<TestMessageEnvelope> 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<TestMessageEnvelope, Collection<TestMessageEnvelope>> builder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000);
+ builder.onEarlyTrigger(c -> {
+ c.clear();
+ return c;
+ });
+ List<TestMessageEnvelope> collection = new ArrayList<TestMessageEnvelope>() { {
+ 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<Collection<TestMessageEnvelope>>, WindowState<Collection<TestMessageEnvelope>>> earlyTriggerUpdater =
+ (Function<WindowState<Collection<TestMessageEnvelope>>, WindowState<Collection<TestMessageEnvelope>>>) this.earlyTriggerUpdater.get(builder);
+ WindowState<Collection<TestMessageEnvelope>> 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<Collection<TestMessageEnvelope>>, WindowState<Collection<TestMessageEnvelope>>> lateTriggerUpdater =
+ (Function<WindowState<Collection<TestMessageEnvelope>>, WindowState<Collection<TestMessageEnvelope>>>) 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");
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/test/java/org/apache/samza/operators/windows/TestWindowOutput.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/windows/TestWindowOutput.java b/samza-api/src/test/java/org/apache/samza/operators/windows/TestWindowOutput.java
new file mode 100644
index 0000000..7f81fc9
--- /dev/null
+++ b/samza-api/src/test/java/org/apache/samza/operators/windows/TestWindowOutput.java
@@ -0,0 +1,36 @@
+/*
+ * 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.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+
+public class TestWindowOutput {
+
+ @Test
+ public void testConstructor() {
+ WindowOutput<String, Integer> wndOutput = WindowOutput.of("testMsg", 10);
+ assertEquals(wndOutput.getKey(), "testMsg");
+ assertEquals(wndOutput.getMessage(), Integer.valueOf(10));
+ assertFalse(wndOutput.isDelete());
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/test/java/org/apache/samza/operators/windows/TestWindows.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..26af26e
--- /dev/null
+++ b/samza-api/src/test/java/org/apache/samza/operators/windows/TestWindows.java
@@ -0,0 +1,109 @@
+/*
+ * 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<TestMessageEnvelope, String, Collection<TestMessageEnvelope>, WindowOutput<String, Collection<TestMessageEnvelope>>> 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<TestMessageEnvelope, String> wndKeyFunc = (Function<TestMessageEnvelope, String>) wndKeyFuncField.get(testWnd);
+ assertEquals(wndKeyFunc.apply(new TestMessageEnvelope("test-key", "test-value", 0)), "test-key");
+ BiFunction<TestMessageEnvelope, Collection<TestMessageEnvelope>, Collection<TestMessageEnvelope>> aggrFunc =
+ (BiFunction<TestMessageEnvelope, Collection<TestMessageEnvelope>, Collection<TestMessageEnvelope>>) aggregatorField.get(testWnd);
+ TestMessageEnvelope mockMsg = mock(TestMessageEnvelope.class);
+ Collection<TestMessageEnvelope> collection = aggrFunc.apply(mockMsg, new ArrayList<>());
+ assertTrue(collection.size() == 1);
+ assertTrue(collection.contains(mockMsg));
+
+ // test constructing the session window w/ customized session info
+ Window<TestMessageEnvelope, String, Collection<Character>, WindowOutput<String, Collection<Character>>> testWnd2 = Windows.intoSessions(
+ m -> String.format("key-%d", m.getMessage().getEventTime()), m -> m.getMessage().getValue().charAt(0));
+ assertTrue(testWnd2 instanceof SessionWindow);
+ wndKeyFunc = (Function<TestMessageEnvelope, String>) wndKeyFuncField.get(testWnd2);
+ aggrFunc = (BiFunction<TestMessageEnvelope, Collection<TestMessageEnvelope>, Collection<TestMessageEnvelope>>) 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<TestMessageEnvelope, String, Integer, WindowOutput<String, Integer>> testCounter = Windows.intoSessionCounter(
+ m -> String.format("key-%d", m.getMessage().getEventTime()));
+ assertTrue(testCounter instanceof SessionWindow);
+ wndKeyFunc = (Function<TestMessageEnvelope, String>) wndKeyFuncField.get(testCounter);
+ BiFunction<TestMessageEnvelope, Integer, Integer> counterFn = (BiFunction<TestMessageEnvelope, Integer, Integer>) 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<TestMessageEnvelope, String, Integer, WindowOutput<String, Integer>> testCounter = Windows.intoSessionCounter(
+ m -> String.format("key-%d", m.getMessage().getEventTime()));
+ // test session window w/ a trigger
+ TriggerBuilder<TestMessageEnvelope, Integer> triggerBuilder = TriggerBuilder.earlyTriggerWhenExceedWndLen(1000L);
+ testCounter.setTriggers(triggerBuilder);
+ Trigger<TestMessageEnvelope, WindowState<Integer>> expectedTrigger = triggerBuilder.build();
+ Trigger<TestMessageEnvelope, WindowState<Integer>> 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));
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/MessageStreamImpl.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..231d3f5
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/MessageStreamImpl.java
@@ -0,0 +1,134 @@
+/*
+ * 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.functions.FilterFunction;
+import org.apache.samza.operators.functions.FlatMapFunction;
+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.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.windows.WindowState;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+
+/**
+ * The implementation for input/output {@link MessageStream}s to/from the operators.
+ * Users use the {@link MessageStream} API methods to describe and chain the operators specs.
+ *
+ * @param <M> type of {@link MessageEnvelope}s in this {@link MessageStream}
+ */
+public class MessageStreamImpl<M extends MessageEnvelope> implements MessageStream<M> {
+
+ /**
+ * The set of operators that consume the {@link MessageEnvelope}s in this {@link MessageStream}
+ */
+ private final Set<OperatorSpec> registeredOperatorSpecs = new HashSet<>();
+
+ @Override
+ public <OM extends MessageEnvelope> MessageStream<OM> map(MapFunction<M, OM> mapFn) {
+ OperatorSpec<OM> op = OperatorSpecs.<M, OM>createStreamOperator(m -> new ArrayList<OM>() { {
+ OM r = mapFn.apply(m);
+ if (r != null) {
+ this.add(r);
+ }
+ } });
+ this.registeredOperatorSpecs.add(op);
+ return op.getOutputStream();
+ }
+
+ @Override
+ public <OM extends MessageEnvelope> MessageStream<OM> flatMap(FlatMapFunction<M, OM> flatMapFn) {
+ OperatorSpec<OM> op = OperatorSpecs.createStreamOperator(flatMapFn);
+ this.registeredOperatorSpecs.add(op);
+ return op.getOutputStream();
+ }
+
+ @Override
+ public MessageStream<M> filter(FilterFunction<M> filterFn) {
+ OperatorSpec<M> op = OperatorSpecs.<M, M>createStreamOperator(t -> new ArrayList<M>() { {
+ if (filterFn.apply(t)) {
+ this.add(t);
+ }
+ } });
+ this.registeredOperatorSpecs.add(op);
+ return op.getOutputStream();
+ }
+
+ @Override
+ public void sink(SinkFunction<M> sinkFn) {
+ this.registeredOperatorSpecs.add(OperatorSpecs.createSinkOperator(sinkFn));
+ }
+
+ @Override
+ public <WK, WV, WS extends WindowState<WV>, WM extends WindowOutput<WK, WV>> MessageStream<WM> window(
+ Window<M, WK, WV, WM> window) {
+ OperatorSpec<WM> wndOp = OperatorSpecs.createWindowOperator((WindowFn<M, WK, WS, WM>) window.getInternalWindowFn());
+ this.registeredOperatorSpecs.add(wndOp);
+ return wndOp.getOutputStream();
+ }
+
+ @Override
+ public <K, JM extends MessageEnvelope<K, ?>, RM extends MessageEnvelope> MessageStream<RM> join(
+ MessageStream<JM> otherStream, JoinFunction<M, JM, RM> joinFn) {
+ MessageStreamImpl<RM> outputStream = new MessageStreamImpl<>();
+
+ BiFunction<M, JM, RM> parJoin1 = joinFn::apply;
+ BiFunction<JM, M, RM> parJoin2 = (m, t1) -> joinFn.apply(t1, m);
+
+ // TODO: need to add default store functions for the two partial join functions
+
+ ((MessageStreamImpl<JM>) otherStream).registeredOperatorSpecs.add(
+ OperatorSpecs.createPartialJoinOperator(parJoin2, outputStream));
+ this.registeredOperatorSpecs.add(OperatorSpecs.createPartialJoinOperator(parJoin1, outputStream));
+ return outputStream;
+ }
+
+ @Override
+ public MessageStream<M> merge(Collection<MessageStream<M>> otherStreams) {
+ MessageStreamImpl<M> outputStream = new MessageStreamImpl<>();
+
+ otherStreams.add(this);
+ otherStreams.forEach(other ->
+ ((MessageStreamImpl<M>) other).registeredOperatorSpecs.add(OperatorSpecs.createMergeOperator(outputStream)));
+ return outputStream;
+ }
+
+ /**
+ * Gets the operator specs registered to consume the output of this {@link MessageStream}. This is an internal API and
+ * should not be exposed to users.
+ *
+ * @return a collection containing all {@link OperatorSpec}s that are registered with this {@link MessageStream}.
+ */
+ public Collection<OperatorSpec> getRegisteredOperatorSpecs() {
+ return Collections.unmodifiableSet(this.registeredOperatorSpecs);
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/StateStoreImpl.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..2572f14
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/StateStoreImpl.java
@@ -0,0 +1,56 @@
+/*
+ * 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<M extends MessageEnvelope, SK, SS> {
+ private final String storeName;
+ private final StoreFunctions<M, SK, SS> storeFunctions;
+ private KeyValueStore<SK, SS> kvStore = null;
+
+ public StateStoreImpl(StoreFunctions<M, SK, SS> store, String storeName) {
+ this.storeFunctions = store;
+ this.storeName = storeName;
+ }
+
+ public void init(TaskContext context) {
+ this.kvStore = (KeyValueStore<SK, SS>) context.getStore(this.storeName);
+ }
+
+ public Entry<SK, SS> getState(M m) {
+ SK key = this.storeFunctions.getStoreKeyFn().apply(m);
+ SS state = this.kvStore.get(key);
+ return new Entry<>(key, state);
+ }
+
+ public Entry<SK, SS> updateState(M m, Entry<SK, SS> 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
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/StreamOperatorAdaptorTask.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/StreamOperatorAdaptorTask.java b/samza-operator/src/main/java/org/apache/samza/operators/StreamOperatorAdaptorTask.java
new file mode 100644
index 0000000..152cd92
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/StreamOperatorAdaptorTask.java
@@ -0,0 +1,105 @@
+/*
+ * 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.config.Config;
+import org.apache.samza.operators.data.IncomingSystemMessageEnvelope;
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.impl.OperatorImpl;
+import org.apache.samza.operators.impl.OperatorImpls;
+import org.apache.samza.system.IncomingMessageEnvelope;
+import org.apache.samza.system.SystemStreamPartition;
+import org.apache.samza.task.InitableTask;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.StreamTask;
+import org.apache.samza.task.TaskContext;
+import org.apache.samza.task.TaskCoordinator;
+import org.apache.samza.task.WindowableTask;
+
+import java.util.HashMap;
+import java.util.Map;
+
+
+/**
+ * An {@link StreamTask} implementation that receives {@link IncomingSystemMessageEnvelope}s and propagates them
+ * through the user's stream transformations defined in {@link StreamOperatorTask#transform(Map)} using the
+ * {@link MessageStream} APIs.
+ * <p>
+ * This class brings all the operator API implementation components together and feeds the
+ * {@link IncomingSystemMessageEnvelope}s into the transformation chains.
+ * <p>
+ * It accepts an instance of the user implemented {@link StreamOperatorTask}. When its own {@link #init(Config, TaskContext)}
+ * method is called during startup, it creates a {@link MessageStreamImpl} corresponding to each of its input
+ * {@link SystemStreamPartition}s and then calls the user's {@link StreamOperatorTask#transform(Map)} method.
+ * <p>
+ * When users invoke the methods on the {@link MessageStream} API to describe their stream transformations in the
+ * {@link StreamOperatorTask#transform(Map)} method, the underlying {@link MessageStreamImpl} creates the
+ * corresponding {@link org.apache.samza.operators.spec.OperatorSpec} to record information about the desired
+ * transformation, and returns the output {@link MessageStream} to allow further transform chaining.
+ * <p>
+ * Once the user's transformation DAGs have been described for all {@link MessageStream}s (i.e., when the
+ * {@link StreamOperatorTask#transform(Map)} call returns), it calls
+ * {@link OperatorImpls#createOperatorImpls(MessageStreamImpl, TaskContext)} for each of the input
+ * {@link MessageStreamImpl}. This instantiates the {@link org.apache.samza.operators.impl.OperatorImpl} DAG
+ * corresponding to the aforementioned {@link org.apache.samza.operators.spec.OperatorSpec} DAG and returns the
+ * root node of the DAG, which this class saves.
+ * <p>
+ * Now that it has the root for the DAG corresponding to each {@link SystemStreamPartition}, it can pass the message
+ * envelopes received in {@link StreamTask#process(IncomingMessageEnvelope, MessageCollector, TaskCoordinator)} along
+ * to the appropriate root nodes. From then on, each {@link org.apache.samza.operators.impl.OperatorImpl} propagates
+ * its transformed output to the next set of {@link org.apache.samza.operators.impl.OperatorImpl}s.
+ */
+public final class StreamOperatorAdaptorTask implements StreamTask, InitableTask, WindowableTask {
+
+ /**
+ * A mapping from each {@link SystemStreamPartition} to the root node of its operator chain DAG.
+ */
+ private final Map<SystemStreamPartition, OperatorImpl<IncomingSystemMessageEnvelope, ? extends MessageEnvelope>> operatorChains = new HashMap<>();
+
+ private final StreamOperatorTask userTask;
+
+ public StreamOperatorAdaptorTask(StreamOperatorTask userTask) {
+ this.userTask = userTask;
+ }
+
+ @Override
+ public final void init(Config config, TaskContext context) throws Exception {
+ if (this.userTask instanceof InitableTask) {
+ ((InitableTask) this.userTask).init(config, context);
+ }
+ Map<SystemStreamPartition, MessageStream<IncomingSystemMessageEnvelope>> messageStreams = new HashMap<>();
+ context.getSystemStreamPartitions().forEach(ssp -> messageStreams.put(ssp, new MessageStreamImpl<>()));
+ this.userTask.transform(messageStreams);
+ messageStreams.forEach((ssp, ms) ->
+ operatorChains.put(ssp, OperatorImpls.createOperatorImpls((MessageStreamImpl<IncomingSystemMessageEnvelope>) ms, context)));
+ }
+
+ @Override
+ public final void process(IncomingMessageEnvelope ime, MessageCollector collector, TaskCoordinator coordinator) {
+ this.operatorChains.get(ime.getSystemStreamPartition())
+ .onNext(new IncomingSystemMessageEnvelope(ime), collector, coordinator);
+ }
+
+ @Override
+ public final void window(MessageCollector collector, TaskCoordinator coordinator) throws Exception {
+ if (this.userTask instanceof WindowableTask) {
+ ((WindowableTask) this.userTask).window(collector, coordinator);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/impl/ChainedOperators.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/ChainedOperators.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/ChainedOperators.java
deleted file mode 100644
index 1eee2dc..0000000
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/ChainedOperators.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.impl;
-
-import org.apache.samza.storage.kv.Entry;
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.data.Message;
-import org.apache.samza.operators.internal.Operators.Operator;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskContext;
-import org.apache.samza.task.TaskCoordinator;
-
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Set;
-
-
-/**
- * Implementation class for a chain of operators from the single input {@code source}
- *
- * @param <M> type of message in the input stream {@code source}
- */
-public class ChainedOperators<M extends Message> {
-
- private final Set<OperatorImpl> subscribers = new HashSet<>();
-
- /**
- * Private constructor
- *
- * @param source the input source {@link MessageStream}
- * @param context the {@link TaskContext} object that we need to instantiate the state stores
- */
- private ChainedOperators(MessageStream<M> source, TaskContext context) {
- // create the pipeline/topology starting from source
- source.getSubscribers().forEach(sub -> {
- // pass in the context s.t. stateful stream operators can initialize their stores
- OperatorImpl subImpl = this.createAndSubscribe(sub, source, context);
- this.subscribers.add(subImpl);
- });
- }
-
- /**
- * Private function to recursively instantiate the implementation of operators and the chains
- *
- * @param operator the operator that subscribe to {@code source}
- * @param source the source {@link MessageStream}
- * @param context the context of the task
- * @return the implementation object of the corresponding {@code operator}
- */
- private OperatorImpl<M, ? extends Message> createAndSubscribe(Operator operator, MessageStream source,
- TaskContext context) {
- Entry<OperatorImpl<M, ? extends Message>, Boolean> factoryEntry = OperatorFactory.getOperator(operator);
- if (factoryEntry.getValue()) {
- // The operator has already been instantiated and we do not need to traverse and create the subscribers any more.
- return factoryEntry.getKey();
- }
- OperatorImpl<M, ? extends Message> opImpl = factoryEntry.getKey();
- MessageStream outStream = operator.getOutputStream();
- Collection<Operator> subs = outStream.getSubscribers();
- subs.forEach(sub -> {
- OperatorImpl subImpl = this.createAndSubscribe(sub, operator.getOutputStream(), context);
- opImpl.subscribe(subImpl);
- });
- // initialize the operator's state store
- opImpl.init(source, context);
- return opImpl;
- }
-
- /**
- * Static method to create a {@link ChainedOperators} from the {@code source} stream
- *
- * @param source the input source {@link MessageStream}
- * @param context the {@link TaskContext} object used to initialize the {@link StateStoreImpl}
- * @param <M> the type of input {@link Message}
- * @return a {@link ChainedOperators} object takes the {@code source} as input
- */
- public static <M extends Message> ChainedOperators create(MessageStream<M> source, TaskContext context) {
- return new ChainedOperators<>(source, context);
- }
-
- /**
- * Method to navigate the incoming {@code message} through the processing chains
- *
- * @param message the incoming message to this {@link ChainedOperators}
- * @param collector the {@link MessageCollector} object within the process context
- * @param coordinator the {@link TaskCoordinator} object within the process context
- */
- public void onNext(M message, MessageCollector collector, TaskCoordinator coordinator) {
- this.subscribers.forEach(sub -> sub.onNext(message, collector, coordinator));
- }
-
- /**
- * Method to handle timer events
- *
- * @param collector the {@link MessageCollector} object within the process context
- * @param coordinator the {@link TaskCoordinator} object within the process context
- */
- public void onTimer(MessageCollector collector, TaskCoordinator coordinator) {
- long nanoTime = System.nanoTime();
- this.subscribers.forEach(sub -> sub.onTimer(nanoTime, collector, coordinator));
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorFactory.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorFactory.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorFactory.java
deleted file mode 100644
index ea90878..0000000
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorFactory.java
+++ /dev/null
@@ -1,85 +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.storage.kv.Entry;
-import org.apache.samza.operators.WindowState;
-import org.apache.samza.operators.data.Message;
-import org.apache.samza.operators.internal.Operators.*;
-import org.apache.samza.operators.internal.WindowOutput;
-import org.apache.samza.operators.impl.join.PartialJoinOpImpl;
-import org.apache.samza.operators.impl.window.SessionWindowImpl;
-
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-
-/**
- * The factory class that instantiates all implementation of {@link OperatorImpl} classes.
- */
-public class OperatorFactory {
-
- /**
- * the static operatorMap that includes all operator implementation instances
- */
- private static final Map<Operator, OperatorImpl<? extends Message, ? extends Message>> OPERATOR_MAP = new ConcurrentHashMap<>();
-
- /**
- * The method to actually create the implementation instances of operators
- *
- * @param operator the immutable definition of {@link Operator}
- * @param <M> type of input {@link Message}
- * @param <RM> type of output {@link Message}
- * @return the implementation object of {@link OperatorImpl}
- */
- private static <M extends Message, RM extends Message> OperatorImpl<M, ? extends Message> createOperator(Operator<RM> operator) {
- if (operator instanceof StreamOperator) {
- return new SimpleOperatorImpl<>((StreamOperator<M, RM>) operator);
- } else if (operator instanceof SinkOperator) {
- return new SinkOperatorImpl<>((SinkOperator<M>) operator);
- } else if (operator instanceof WindowOperator) {
- return new SessionWindowImpl<>((WindowOperator<M, ?, ? extends WindowState, ? extends WindowOutput>) operator);
- } else if (operator instanceof PartialJoinOperator) {
- return new PartialJoinOpImpl<>((PartialJoinOperator) operator);
- }
- throw new IllegalArgumentException(
- String.format("The type of operator is not supported. Operator class name: %s", operator.getClass().getName()));
- }
-
- /**
- * The method to get the unique implementation instance of {@link Operator}
- *
- * @param operator the {@link Operator} to instantiate
- * @param <M> type of input {@link Message}
- * @param <RM> type of output {@link Message}
- * @return A pair of entry that include the unique implementation instance to the {@code operator} and a boolean value indicating whether
- * the operator instance has already been created or not. True means the operator instance has already created, false means the operator
- * was not created.
- */
- public static <M extends Message, RM extends Message> Entry<OperatorImpl<M, ? extends Message>, Boolean> getOperator(Operator<RM> operator) {
- if (!OPERATOR_MAP.containsKey(operator)) {
- OperatorImpl<M, ? extends Message> operatorImpl = OperatorFactory.createOperator(operator);
- if (OPERATOR_MAP.putIfAbsent(operator, operatorImpl) == null) {
- return new Entry<OperatorImpl<M, ? extends Message>, Boolean>(operatorImpl, false) { };
- }
- }
- return new Entry<OperatorImpl<M, ? extends Message>, Boolean>((OperatorImpl<M, ? extends Message>) OPERATOR_MAP.get(operator), true) { };
- }
-
-}
[5/6] samza git commit: SAMZA-1054: Refactor Operator APIs
Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/internal/Operators.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/internal/Operators.java b/samza-api/src/main/java/org/apache/samza/operators/internal/Operators.java
deleted file mode 100644
index f06387c..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/internal/Operators.java
+++ /dev/null
@@ -1,469 +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.internal;
-
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.WindowState;
-import org.apache.samza.operators.data.Message;
-import org.apache.samza.storage.kv.Entry;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskCoordinator;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.UUID;
-import java.util.function.BiFunction;
-import java.util.function.Function;
-
-
-/**
- * This class defines all basic stream operator classes used by internal implementation only. All classes defined in
- * this file are immutable.
- *
- * NOTE: Programmers should not use the operators defined in this class directly. All {@link Operator} objects
- * should be initiated via {@link MessageStream} API methods
- */
-public class Operators {
- /**
- * Private constructor to prevent instantiation of the {@link Operators} class
- */
- private Operators() {}
-
- private static String getOperatorId() {
- // TODO: need to change the IDs to be a consistent, durable IDs that can be recovered across container and job restarts
- return UUID.randomUUID().toString();
- }
-
- /**
- * Private interface for stream operator functions. The interface class defines the output of the stream operator function.
- *
- */
- public interface Operator<OM extends Message> {
- MessageStream<OM> getOutputStream();
- }
-
- /**
- * Linear stream operator function that takes 1 input {@link Message} and output a collection of output {@link Message}s.
- *
- * @param <M> the type of input {@link Message}
- * @param <OM> the type of output {@link Message}
- */
- public static class StreamOperator<M extends Message, OM extends Message> implements Operator<OM> {
- /**
- * The output {@link MessageStream}
- */
- private final MessageStream<OM> outputStream;
-
- /**
- * The transformation function
- */
- private final Function<M, Collection<OM>> txfmFunction;
-
- /**
- * Constructor of {@link StreamOperator}. Make it private s.t. it can only be created within {@link Operators}.
- *
- * @param transformFn the transformation function to be applied that transforms 1 input {@link Message} into a collection
- * of output {@link Message}s
- */
- private StreamOperator(Function<M, Collection<OM>> transformFn) {
- this(transformFn, new MessageStream<>());
- }
-
- /**
- * Constructor of {@link StreamOperator} which allows the user to define the output {@link MessageStream}
- *
- * @param transformFn the transformation function
- * @param outputStream the output {@link MessageStream}
- */
- private StreamOperator(Function<M, Collection<OM>> transformFn, MessageStream<OM> outputStream) {
- this.outputStream = outputStream;
- this.txfmFunction = transformFn;
- }
-
- @Override
- public MessageStream<OM> getOutputStream() {
- return this.outputStream;
- }
-
- /**
- * Method to get the transformation function.
- *
- * @return the {@code txfmFunction}
- */
- public Function<M, Collection<OM>> getFunction() {
- return this.txfmFunction;
- }
-
- }
-
- /**
- * A sink operator function that allows customized code to send the output to external system. This is the terminal
- * operator that does not have any output {@link MessageStream} that allows further processing in the same
- * {@link org.apache.samza.operators.task.StreamOperatorTask}
- *
- * @param <M> the type of input {@link Message}
- */
- public static class SinkOperator<M extends Message> implements Operator {
-
- /**
- * The user-defined sink function
- */
- private final MessageStream.VoidFunction3<M, MessageCollector, TaskCoordinator> sink;
-
- /**
- * Default constructor for {@link SinkOperator}. Make it private s.t. it can only be created within {@link Operators}.
- *
- * @param sink the user-defined sink function
- */
- private SinkOperator(MessageStream.VoidFunction3<M, MessageCollector, TaskCoordinator> sink) {
- this.sink = sink;
- }
-
- @Override
- public MessageStream getOutputStream() {
- return null;
- }
-
- /**
- * Method to get the user-defined function implements the {@link SinkOperator}
- *
- * @return a {@link MessageStream.VoidFunction3} function that allows the caller to pass in an input message, {@link MessageCollector}
- * and {@link TaskCoordinator} to the sink function
- */
- public MessageStream.VoidFunction3<M, MessageCollector, TaskCoordinator> getFunction() {
- return this.sink;
- }
- }
-
- /**
- * The store functions that are used by {@link WindowOperator} and {@link PartialJoinOperator} to store and retrieve
- * buffered messages and partial aggregation results
- *
- * @param <SK> the type of key used to store the operator states
- * @param <SS> the type of operator state. e.g. could be the partial aggregation result for a window, or a buffered
- * input message from the join stream for a join
- */
- public static class StoreFunctions<M extends Message, SK, SS> {
- /**
- * Function to define the key to query in the operator state store, according to the incoming {@link Message}
- * This method only supports finding the unique key for the incoming message, which supports use case of non-overlapping
- * windows and unique-key-based join.
- *
- * 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 to map from a single input
- * message to a range of keys in the store.
- */
- private final Function<M, SK> storeKeyFinder;
-
- /**
- * Function to update the store entry based on the current state and the incoming {@link Message}
- *
- * TODO: this is assuming a 1:1 mapping from the input message 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.
- */
- private final BiFunction<M, SS, SS> stateUpdater;
-
- /**
- * Constructor of state store functions.
- *
- */
- private StoreFunctions(Function<M, SK> keyFinder,
- BiFunction<M, SS, SS> stateUpdater) {
- this.storeKeyFinder = keyFinder;
- this.stateUpdater = stateUpdater;
- }
-
- /**
- * Method to get the {@code storeKeyFinder} function
- *
- * @return the function to calculate the key from an input {@link Message}
- */
- public Function<M, SK> getStoreKeyFinder() {
- return this.storeKeyFinder;
- }
-
- /**
- * Method to get the {@code stateUpdater} function
- *
- * @return the function to update the corresponding state according to an input {@link Message}
- */
- public BiFunction<M, SS, SS> getStateUpdater() {
- return this.stateUpdater;
- }
- }
-
- /**
- * Defines a window operator function that takes one {@link MessageStream} as an input, accumulate the window state, and generate
- * an output {@link MessageStream} w/ output type {@code WM} which extends {@link WindowOutput}
- *
- * @param <M> the type of input {@link Message}
- * @param <WK> the type of key in the output {@link Message} from the {@link WindowOperator} function
- * @param <WS> the type of window state in the {@link WindowOperator} function
- * @param <WM> the type of window output {@link Message}
- */
- public static class WindowOperator<M extends Message, WK, WS extends WindowState, WM extends WindowOutput<WK, ?>> implements Operator<WM> {
- /**
- * The output {@link MessageStream}
- */
- private final MessageStream<WM> outputStream;
-
- /**
- * The main window transformation function that takes {@link Message}s from one input stream, aggregates w/ the window
- * state(s) from the window state store, and generate output {@link Message}s to the output stream.
- */
- private final BiFunction<M, Entry<WK, WS>, WM> txfmFunction;
-
- /**
- * The state store functions for the {@link WindowOperator}
- */
- private final StoreFunctions<M, WK, WS> storeFunctions;
-
- /**
- * The window trigger function
- */
- private final Trigger<M, WS> trigger;
-
- /**
- * The unique ID of stateful operators
- */
- private final String opId;
-
- /**
- * Constructor for {@link WindowOperator}. Make it private s.t. it can only be created within {@link Operators}.
- *
- * @param windowFn description of the window function
- * @param operatorId auto-generated unique ID of the operator
- */
- private WindowOperator(WindowFn<M, WK, WS, WM> windowFn, String operatorId) {
- this.outputStream = new MessageStream<>();
- this.txfmFunction = windowFn.getTransformFunc();
- this.storeFunctions = windowFn.getStoreFuncs();
- this.trigger = windowFn.getTrigger();
- this.opId = operatorId;
- }
-
- @Override
- public String toString() {
- return this.opId;
- }
-
- @Override
- public MessageStream<WM> getOutputStream() {
- return this.outputStream;
- }
-
- /**
- * Method to get the window's {@link StoreFunctions}.
- *
- * @return the window operator's {@code storeFunctions}
- */
- public StoreFunctions<M, WK, WS> getStoreFunctions() {
- return this.storeFunctions;
- }
-
- /**
- * Method to get the window operator's main function
- *
- * @return the window operator's {@code txfmFunction}
- */
- public BiFunction<M, Entry<WK, WS>, WM> getFunction() {
- return this.txfmFunction;
- }
-
- /**
- * Method to get the trigger functions
- *
- * @return the {@link Trigger} for this {@link WindowOperator}
- */
- public Trigger<M, WS> getTrigger() {
- return this.trigger;
- }
-
- /**
- * Method to generate the window operator's state store name
- *
- * @param inputStream the input {@link MessageStream} to this state store
- * @return the persistent store name of the window operator
- */
- public String getStoreName(MessageStream<M> 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());
- }
- }
-
- /**
- * The partial join operator that takes {@link Message}s from one input stream and join w/ buffered {@link Message}s from
- * another stream and generate join output to {@code output}
- *
- * @param <M> the type of input {@link Message}
- * @param <K> the type of join key
- * @param <JM> the type of message of {@link Message} in the other join stream
- * @param <RM> the type of message of {@link Message} in the join output stream
- */
- public static class PartialJoinOperator<M extends Message<K, ?>, K, JM extends Message<K, ?>, RM extends Message> implements Operator<RM> {
-
- private final MessageStream<RM> joinOutput;
-
- /**
- * The main transformation function of {@link PartialJoinOperator} that takes a type {@code M} input message,
- * join w/ a stream of buffered {@link Message}s from another stream w/ type {@code JM}, and generate joined type {@code RM}.
- */
- private final BiFunction<M, JM, RM> txfmFunction;
-
- /**
- * The message store functions that read the buffered messages from the other stream in the join
- */
- private final StoreFunctions<JM, K, JM> joinStoreFunctions;
-
- /**
- * The message store functions that save the buffered messages of this {@link MessageStream} in the join
- */
- private final StoreFunctions<M, K, M> selfStoreFunctions;
-
- /**
- * The unique ID for the stateful operator
- */
- private final String opId;
-
- /**
- * Default constructor to create a {@link PartialJoinOperator} object
- *
- * @param partialJoin partial join function that take type {@code M} of input {@link Message} and join w/ type
- * {@code JM} of buffered {@link Message} from another stream
- * @param joinOutput the output {@link MessageStream} of the join results
- */
- private PartialJoinOperator(BiFunction<M, JM, RM> partialJoin, MessageStream<RM> joinOutput, String opId) {
- this.joinOutput = joinOutput;
- this.txfmFunction = partialJoin;
- // Read-only join store, no creator/updater functions specified
- this.joinStoreFunctions = new StoreFunctions<>(m -> m.getKey(), null);
- // Buffered message store for this input stream
- this.selfStoreFunctions = new StoreFunctions<>(m -> m.getKey(), (m, s1) -> m);
- this.opId = opId;
- }
-
- @Override
- public String toString() {
- return this.opId;
- }
-
- @Override
- public MessageStream<RM> getOutputStream() {
- return this.joinOutput;
- }
-
- /**
- * Method to get {@code joinStoreFunctions}
- *
- * @return {@code joinStoreFunctions}
- */
- public StoreFunctions<JM, K, JM> getJoinStoreFunctions() {
- return this.joinStoreFunctions;
- }
-
- /**
- * Method to get {@code selfStoreFunctions}
- *
- * @return {@code selfStoreFunctions}
- */
- public StoreFunctions<M, K, M> getSelfStoreFunctions() {
- return this.selfStoreFunctions;
- }
-
- /**
- * Method to get {@code txfmFunction}
- *
- * @return {@code txfmFunction}
- */
- public BiFunction<M, JM, RM> getFunction() {
- return this.txfmFunction;
- }
- }
-
- /**
- * The method only to be used internally in {@link MessageStream} to create {@link StreamOperator}
- *
- * @param transformFn the corresponding transformation function
- * @param <M> type of input {@link Message}
- * @param <OM> type of output {@link Message}
- * @return the {@link StreamOperator}
- */
- public static <M extends Message, OM extends Message> StreamOperator<M, OM> getStreamOperator(Function<M, Collection<OM>> transformFn) {
- return new StreamOperator<>(transformFn);
- }
-
- /**
- * The method only to be used internally in {@link MessageStream} to create {@link SinkOperator}
- *
- * @param sinkFn the sink function
- * @param <M> type of input {@link Message}
- * @return the {@link SinkOperator}
- */
- public static <M extends Message> SinkOperator<M> getSinkOperator(MessageStream.VoidFunction3<M, MessageCollector, TaskCoordinator> sinkFn) {
- return new SinkOperator<>(sinkFn);
- }
-
- /**
- * The method only to be used internally in {@link MessageStream} to create {@link WindowOperator}
- *
- * @param windowFn the {@link WindowFn} function
- * @param <M> type of input {@link Message}
- * @param <WK> type of window key
- * @param <WS> type of {@link WindowState}
- * @param <WM> type of output {@link WindowOutput}
- * @return the {@link WindowOperator}
- */
- public static <M extends Message, WK, WS extends WindowState, WM extends WindowOutput<WK, ?>> WindowOperator<M, WK, WS, WM> getWindowOperator(
- WindowFn<M, WK, WS, WM> windowFn) {
- return new WindowOperator<>(windowFn, Operators.getOperatorId());
- }
-
- /**
- * The method only to be used internally in {@link MessageStream} to create {@link WindowOperator}
- *
- * @param joiner the {@link WindowFn} function
- * @param joinOutput the output {@link MessageStream}
- * @param <M> type of input {@link Message}
- * @param <K> type of join key
- * @param <JM> the type of message in the {@link Message} from the other join stream
- * @param <RM> the type of message in the {@link Message} from the join function
- * @return the {@link PartialJoinOperator}
- */
- public static <M extends Message<K, ?>, K, JM extends Message<K, ?>, RM extends Message> PartialJoinOperator<M, K, JM, RM> getPartialJoinOperator(
- BiFunction<M, JM, RM> joiner, MessageStream<RM> joinOutput) {
- return new PartialJoinOperator<>(joiner, joinOutput, Operators.getOperatorId());
- }
-
- /**
- * The method only to be used internally in {@link MessageStream} to create {@link StreamOperator} as a merger function
- *
- * @param mergeOutput the common output {@link MessageStream} from the merger
- * @param <M> the type of input {@link Message}
- * @return the {@link StreamOperator} for merge
- */
- public static <M extends Message> StreamOperator<M, M> getMergeOperator(MessageStream<M> mergeOutput) {
- return new StreamOperator<M, M>(t ->
- new ArrayList<M>() { {
- this.add(t);
- } },
- mergeOutput);
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/internal/Trigger.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/internal/Trigger.java b/samza-api/src/main/java/org/apache/samza/operators/internal/Trigger.java
deleted file mode 100644
index 3b50e2b..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/internal/Trigger.java
+++ /dev/null
@@ -1,95 +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.internal;
-
-import org.apache.samza.operators.WindowState;
-import org.apache.samza.operators.data.Message;
-
-import java.util.function.BiFunction;
-import java.util.function.Function;
-
-/**
- * Defines the trigger functions for {@link Operators.WindowOperator}. This class is immutable.
- *
- * @param <M> the type of message from the input stream
- * @param <S> the type of state variable in the window's state store
- */
-public class Trigger<M extends Message, S extends WindowState> {
-
- /**
- * System timer based trigger condition. This is the only guarantee that the {@link Operators.WindowOperator} will proceed forward
- */
- private final Function<S, Boolean> timerTrigger;
-
- /**
- * early trigger condition that determines when to send the first output from the {@link Operators.WindowOperator}
- */
- private final BiFunction<M, S, Boolean> earlyTrigger;
-
- /**
- * late trigger condition that determines when to send the updated output after the first one from a {@link Operators.WindowOperator}
- */
- private final BiFunction<M, S, Boolean> lateTrigger;
-
- /**
- * the function to updated the window state when the first output is triggered
- */
- private final Function<S, S> earlyTriggerUpdater;
-
- /**
- * the function to updated the window state when the late output is triggered
- */
- private final Function<S, S> 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<S, Boolean> timerTrigger, BiFunction<M, S, Boolean> earlyTrigger, BiFunction<M, S, Boolean> lateTrigger,
- Function<S, S> earlyTriggerUpdater, Function<S, S> 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 <M> the type of input {@link Message}
- * @param <S> the type of window state extends {@link WindowState}
- * @return the {@link Trigger} function
- */
- public static <M extends Message, S extends WindowState> Trigger<M, S> createTrigger(Function<S, Boolean> timerTrigger,
- BiFunction<M, S, Boolean> earlyTrigger, BiFunction<M, S, Boolean> lateTrigger, Function<S, S> earlyTriggerUpdater,
- Function<S, S> lateTriggerUpdater) {
- return new Trigger(timerTrigger, earlyTrigger, lateTrigger, earlyTriggerUpdater, lateTriggerUpdater);
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/internal/WindowFn.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/internal/WindowFn.java b/samza-api/src/main/java/org/apache/samza/operators/internal/WindowFn.java
deleted file mode 100644
index 489e5b8..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/internal/WindowFn.java
+++ /dev/null
@@ -1,60 +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.internal;
-
-import org.apache.samza.operators.WindowState;
-import org.apache.samza.operators.data.Message;
-import org.apache.samza.storage.kv.Entry;
-
-import java.util.function.BiFunction;
-
-
-/**
- * Defines an internal representation of a window function. This class SHOULD NOT be used by the programmer directly. It is used
- * by the internal representation and implementation classes in operators.
- *
- * @param <M> type of input stream {@link Message} for window
- * @param <WK> type of window key in the output {@link Message}
- * @param <WS> type of {@link WindowState} variable in the state store
- * @param <WM> type of the message in the output stream
- */
-public interface WindowFn<M extends Message, WK, WS extends WindowState, WM extends WindowOutput<WK, ?>> {
-
- /**
- * get the transformation function of the {@link WindowFn}
- *
- * @return the transformation function takes type {@code M} message and the window state entry, then transform to an {@link WindowOutput}
- */
- BiFunction<M, Entry<WK, WS>, WM> getTransformFunc();
-
- /**
- * get the state store functions for this {@link WindowFn}
- *
- * @return the collection of state store methods
- */
- Operators.StoreFunctions<M, WK, WS> getStoreFuncs();
-
- /**
- * get the trigger conditions for this {@link WindowFn}
- *
- * @return the trigger condition for the {@link WindowFn} function
- */
- Trigger<M, WS> getTrigger();
-
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/internal/WindowOutput.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/internal/WindowOutput.java b/samza-api/src/main/java/org/apache/samza/operators/internal/WindowOutput.java
deleted file mode 100644
index 643b703..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/internal/WindowOutput.java
+++ /dev/null
@@ -1,55 +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.internal;
-
-import org.apache.samza.operators.data.Message;
-
-
-/**
- * This class defines the specific type of output messages from a {@link Operators.WindowOperator} function
- *
- * @param <K> the type of key in the output window result
- * @param <M> the type of value in the output window result
- */
-public final class WindowOutput<K, M> implements Message<K, M> {
- private final K key;
- private final M value;
-
- WindowOutput(K key, M aggregated) {
- this.key = key;
- this.value = aggregated;
- }
-
- @Override public M getMessage() {
- return this.value;
- }
-
- @Override public K getKey() {
- return this.key;
- }
-
- @Override public long getTimestamp() {
- return 0;
- }
-
- static public <K, M> WindowOutput<K, M> of(K key, M result) {
- return new WindowOutput<>(key, result);
- }
-}
-
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/task/StreamOperatorTask.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/task/StreamOperatorTask.java b/samza-api/src/main/java/org/apache/samza/operators/task/StreamOperatorTask.java
deleted file mode 100644
index 42c8f74..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/task/StreamOperatorTask.java
+++ /dev/null
@@ -1,45 +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.task;
-
-import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.operators.MessageStreams.SystemMessageStream;
-
-import java.util.Collection;
-
-/**
- * This interface defines the methods that user needs to implement via the operator programming APIs.
- */
-@InterfaceStability.Unstable
-public interface StreamOperatorTask {
-
- /**
- * Defines the method for users to initialize the operator chains consuming from all {@link SystemMessageStream}s.
- * Users have to implement this function to define their transformation logic on each of the incoming
- * {@link SystemMessageStream}.
- *
- * Note that each {@link SystemMessageStream} corresponds to an input {@link org.apache.samza.system.SystemStreamPartition}
- *
- * @param sources the collection of {@link SystemMessageStream}s that takes {@link org.apache.samza.operators.data.IncomingSystemMessage}
- * from a {@link org.apache.samza.system.SystemStreamPartition}
- */
- void initOperators(Collection<SystemMessageStream> sources);
-
-
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/windows/SessionWindow.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..287025c
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/windows/SessionWindow.java
@@ -0,0 +1,102 @@
+/*
+ * 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 <M> the type of input {@link MessageEnvelope}
+ * @param <WK> the type of session key in the session window
+ * @param <WV> the type of output value in each session window
+ */
+public class SessionWindow<M extends MessageEnvelope, WK, WV> implements Window<M, WK, WV, WindowOutput<WK, WV>> {
+
+ /**
+ * 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<M, WK> sessionKeyFunction, BiFunction<M, WV, WV> aggregator) {
+ this.wndKeyFunction = sessionKeyFunction;
+ this.aggregator = aggregator;
+ }
+
+ /**
+ * function to calculate the window key from input {@link MessageEnvelope}
+ */
+ private final Function<M, WK> wndKeyFunction;
+
+ /**
+ * function to calculate the output value from the input {@link MessageEnvelope} and the current output value
+ */
+ private final BiFunction<M, WV, WV> aggregator;
+
+ /**
+ * trigger condition that determines when to send the {@link WindowOutput}
+ */
+ private Trigger<M, WindowState<WV>> trigger = null;
+
+ //TODO: need to create a set of {@link StoreFunctions} that is default to input {@link MessageEnvelope} type for {@link Window}
+ private StoreFunctions<M, WK, WindowState<WV>> 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<M, WK, WV, WindowOutput<WK, WV>> setTriggers(TriggerBuilder<M, WV> wndTrigger) {
+ this.trigger = wndTrigger.build();
+ return this;
+ }
+
+ private BiFunction<M, Entry<WK, WindowState<WV>>, WindowOutput<WK, WV>> getTransformFunc() {
+ // TODO: actual implementation of the main session window logic, based on the wndKeyFunction, aggregator, and triggers;
+ return null;
+ }
+
+ public WindowFn<M, WK, WindowState<WV>, WindowOutput<WK, WV>> getInternalWindowFn() {
+ return new WindowFn<M, WK, WindowState<WV>, WindowOutput<WK, WV>>() {
+
+ @Override public BiFunction<M, Entry<WK, WindowState<WV>>, WindowOutput<WK, WV>> getTransformFn() {
+ return SessionWindow.this.getTransformFunc();
+ }
+
+ @Override public StoreFunctions<M, WK, WindowState<WV>> getStoreFns() {
+ return SessionWindow.this.storeFunctions;
+ }
+
+ @Override public Trigger<M, WindowState<WV>> getTrigger() {
+ return SessionWindow.this.trigger;
+ }
+ };
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/windows/StoreFunctions.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..0d40761
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/windows/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.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 <SK> the type of key used to store the operator state
+ * @param <SS> 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<M extends MessageEnvelope, SK, SS> {
+ /**
+ * 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<M, SK> 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<M, SS, SS> stateUpdaterFn;
+
+ public StoreFunctions(Function<M, SK> storeKeyFn, BiFunction<M, SS, SS> stateUpdaterFn) {
+ this.storeKeyFn = storeKeyFn;
+ this.stateUpdaterFn = stateUpdaterFn;
+ }
+
+ public Function<M, SK> getStoreKeyFn() {
+ return this.storeKeyFn;
+ }
+
+ public BiFunction<M, SS, SS> getStateUpdaterFn() {
+ return this.stateUpdaterFn;
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/windows/Trigger.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..c8b0edb
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/windows/Trigger.java
@@ -0,0 +1,94 @@
+/*
+ * 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 <M> the type of {@link MessageEnvelope} in the input stream
+ * @param <S> the type of state variable in the window's state store
+ */
+public class Trigger<M extends MessageEnvelope, S extends WindowState> {
+
+ /**
+ * System timer based trigger condition. This is the only guarantee that the window operator will proceed forward
+ */
+ private final Function<S, Boolean> timerTrigger;
+
+ /**
+ * early trigger condition that determines when to send the first output from the window operator
+ */
+ private final BiFunction<M, S, Boolean> earlyTrigger;
+
+ /**
+ * late trigger condition that determines when to send the updated output after the first one from a window operator
+ */
+ private final BiFunction<M, S, Boolean> lateTrigger;
+
+ /**
+ * the function to updated the window state when the first output is triggered
+ */
+ private final Function<S, S> earlyTriggerUpdater;
+
+ /**
+ * the function to updated the window state when the late output is triggered
+ */
+ private final Function<S, S> 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<S, Boolean> timerTrigger, BiFunction<M, S, Boolean> earlyTrigger, BiFunction<M, S, Boolean> lateTrigger,
+ Function<S, S> earlyTriggerUpdater, Function<S, S> 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 <M> the type of input {@link MessageEnvelope}
+ * @param <S> the type of window state extends {@link WindowState}
+ * @return the {@link Trigger} function
+ */
+ public static <M extends MessageEnvelope, S extends WindowState> Trigger<M, S> createTrigger(Function<S, Boolean> timerTrigger,
+ BiFunction<M, S, Boolean> earlyTrigger, BiFunction<M, S, Boolean> lateTrigger, Function<S, S> earlyTriggerUpdater,
+ Function<S, S> lateTriggerUpdater) {
+ return new Trigger(timerTrigger, earlyTrigger, lateTrigger, earlyTriggerUpdater, lateTriggerUpdater);
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/windows/TriggerBuilder.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..6336a50
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/windows/TriggerBuilder.java
@@ -0,0 +1,320 @@
+/*
+ * 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:
+ *
+ * <p>
+ * 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
+ * </p>
+ *
+ * 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 <M> the type of input {@link MessageEnvelope} to the {@link Window}
+ * @param <V> the type of output value from the {@link Window}
+ */
+@InterfaceStability.Unstable
+public final class TriggerBuilder<M extends MessageEnvelope, V> {
+
+ /**
+ * Predicate helper to OR multiple trigger conditions
+ */
+ static class PredicateHelper {
+ static <M, S> BiFunction<M, S, Boolean> or(BiFunction<M, S, Boolean> lhs, BiFunction<M, S, Boolean> rhs) {
+ return (m, s) -> lhs.apply(m, s) || rhs.apply(m, s);
+ }
+
+ static <S> Function<S, Boolean> or(Function<S, Boolean> lhs, Function<S, Boolean> rhs) {
+ return s -> lhs.apply(s) || rhs.apply(s);
+ }
+ }
+
+ /**
+ * The early trigger condition that determines the first output from the {@link Window}
+ */
+ private BiFunction<M, WindowState<V>, Boolean> earlyTrigger = null;
+
+ /**
+ * The late trigger condition that determines the late output(s) from the {@link Window}
+ */
+ private BiFunction<M, WindowState<V>, Boolean> lateTrigger = null;
+
+ /**
+ * The system timer based trigger conditions that guarantees the {@link Window} proceeds forward
+ */
+ private Function<WindowState<V>, Boolean> timerTrigger = null;
+
+ /**
+ * The state updater function to be applied after the first output is triggered
+ */
+ private Function<WindowState<V>, WindowState<V>> earlyTriggerUpdater = Function.identity();
+
+ /**
+ * The state updater function to be applied after the late output is triggered
+ */
+ private Function<WindowState<V>, WindowState<V>> 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<M, WindowState<V>, Boolean> addTrigger(BiFunction<M, WindowState<V>, Boolean> currentTrigger,
+ BiFunction<M, WindowState<V>, 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<WindowState<V>, Boolean> addTimerTrigger(Function<WindowState<V>, Boolean> currentTimer,
+ Function<WindowState<V>, 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<M, Long> 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<M, Boolean> 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<M, WindowState<V>> 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<M, V> onEarlyTrigger(Function<V, V> 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<M, V> onLateTrigger(Function<V, V> 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<M, V> 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<M, V> 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<M, V> 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<M, V> addLateTrigger(BiFunction<M, WindowState<V>, 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 <M> the type of input {@link MessageEnvelope}
+ * @param <V> the type of {@link Window} output value
+ * @return the {@link TriggerBuilder} object
+ */
+ public static <M extends MessageEnvelope, V> TriggerBuilder<M, V> earlyTriggerWhenExceedWndLen(long sizeLimit) {
+ return new TriggerBuilder<M, V>(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 <M> the type of input {@link MessageEnvelope}
+ * @param <V> the type of {@link Window} output value
+ * @return the {@link TriggerBuilder} object
+ */
+ public static <M extends MessageEnvelope, V> TriggerBuilder<M, V> earlyTriggerOnEventTime(Function<M, Long> eventTimeFunc, long eventTimeWndSizeMs) {
+ return new TriggerBuilder<M, V>(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 <M> the type of input {@link MessageEnvelope}
+ * @param <V> the type of {@link Window} output value
+ * @return the {@link TriggerBuilder} object
+ */
+ public static <M extends MessageEnvelope, V> TriggerBuilder<M, V> earlyTriggerOnTokenMsg(Function<M, Boolean> tokenFunc) {
+ return new TriggerBuilder<M, V>(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 <M> the input {@link MessageEnvelope} type
+ * @param <V> the output value from the window
+ * @return the {@link TriggerBuilder} object
+ */
+ public static <M extends MessageEnvelope, V> TriggerBuilder<M, V> earlyTrigger(BiFunction<M, WindowState<V>, Boolean> earlyTrigger) {
+ TriggerBuilder<M, V> newTriggers = new TriggerBuilder<M, V>();
+ 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 <M> the type of input {@link MessageEnvelope}
+ * @param <V> the type of {@link Window} output value
+ * @return the {@link TriggerBuilder} object
+ */
+ public static <M extends MessageEnvelope, V> TriggerBuilder<M, V> timeoutSinceLastMessage(long timeoutMs) {
+ return new TriggerBuilder<M, V>().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 <M> the type of input {@link MessageEnvelope}
+ * @param <V> the type of {@link Window} output value
+ * @return the {@link TriggerBuilder} object
+ */
+ public static <M extends MessageEnvelope, V> TriggerBuilder<M, V> timeoutSinceFirstMessage(long timeoutMs) {
+ return new TriggerBuilder<M, V>().addTimeoutSinceFirstMessage(timeoutMs);
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/windows/Window.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
new file mode 100644
index 0000000..56a307d
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/windows/Window.java
@@ -0,0 +1,49 @@
+/*
+ * 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 <M> the type of input {@link MessageEnvelope}
+ * @param <WK> the type of key to the {@link Window}
+ * @param <WV> the type of output value in the {@link WindowOutput}
+ * @param <WM> the type of {@link MessageEnvelope} in the window output stream
+ */
+public interface Window<M extends MessageEnvelope, WK, WV, WM extends WindowOutput<WK, WV>> {
+
+ /**
+ * 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<M, WK, WV, WM> setTriggers(TriggerBuilder<M, WV> wndTrigger);
+
+ /**
+ * Internal implementation helper to get the functions associated with this Window.
+ *
+ * <b>NOTE:</b> This is purely an internal API and should not be used directly by users.
+ *
+ * @return the functions associated with this Window.
+ */
+ WindowFn<M, WK, WindowState<WV>, WindowOutput<WK, WV>> getInternalWindowFn();
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/windows/WindowFn.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..8878bf9
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/windows/WindowFn.java
@@ -0,0 +1,59 @@
+/*
+ * 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 <M> type of the input {@link MessageEnvelope} for the window
+ * @param <WK> type of the window key in the output {@link MessageEnvelope}
+ * @param <WS> type of the {@link WindowState} in the state store
+ * @param <WM> type of the {@link MessageEnvelope} in the output stream
+ */
+public interface WindowFn<M extends MessageEnvelope, WK, WS extends WindowState, WM extends WindowOutput<WK, ?>> {
+
+ /**
+ * 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<M, Entry<WK, WS>, WM> getTransformFn();
+
+ /**
+ * Get the state store functions for this {@link WindowFn}.
+ *
+ * @return the state store functions
+ */
+ StoreFunctions<M, WK, WS> getStoreFns();
+
+ /**
+ * Get the trigger conditions for this {@link WindowFn}.
+ *
+ * @return the trigger condition for this {@link WindowFn}
+ */
+ Trigger<M, WS> getTrigger();
+
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/windows/WindowOutput.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/WindowOutput.java b/samza-api/src/main/java/org/apache/samza/operators/windows/WindowOutput.java
new file mode 100644
index 0000000..63e34c8
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/windows/WindowOutput.java
@@ -0,0 +1,51 @@
+/*
+ * 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 type of output {@link MessageEnvelope}s in a window operator output stream.
+ *
+ * @param <K> the type of key in the window output
+ * @param <M> the type of value in the window output
+ */
+public final class WindowOutput<K, M> implements MessageEnvelope<K, M> {
+ private final K key;
+ private final M value;
+
+ WindowOutput(K key, M value) {
+ this.key = key;
+ this.value = value;
+ }
+
+ @Override public M getMessage() {
+ return this.value;
+ }
+
+ @Override public K getKey() {
+ return this.key;
+ }
+
+ static public <K, M> WindowOutput<K, M> of(K key, M result) {
+ return new WindowOutput<>(key, result);
+ }
+}
+
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/windows/WindowState.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/windows/WindowState.java b/samza-api/src/main/java/org/apache/samza/operators/windows/WindowState.java
new file mode 100644
index 0000000..835d749
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/windows/WindowState.java
@@ -0,0 +1,85 @@
+/*
+ * 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;
+
+
+/**
+ * This interface defines the methods a window state class has to implement. The programmers are allowed to implement
+ * customized window state to be stored in window state stores by implementing this interface class.
+ *
+ * @param <WV> the type for window output value
+ */
+@InterfaceStability.Unstable
+public interface WindowState<WV> {
+ /**
+ * Method to get the system time when the first {@link org.apache.samza.operators.data.MessageEnvelope}
+ * in the window is received
+ *
+ * @return nano-second of system time for the first {@link org.apache.samza.operators.data.MessageEnvelope}
+ * received in the window
+ */
+ long getFirstMessageTimeNs();
+
+ /**
+ * Method to get the system time when the last {@link org.apache.samza.operators.data.MessageEnvelope}
+ * in the window is received
+ *
+ * @return nano-second of system time for the last {@link org.apache.samza.operators.data.MessageEnvelope}
+ * received in the window
+ */
+ long getLastMessageTimeNs();
+
+ /**
+ * Method to get the earliest event time in the window
+ *
+ * @return the earliest event time in nano-second in the window
+ */
+ long getEarliestEventTimeNs();
+
+ /**
+ * Method to get the latest event time in the window
+ *
+ * @return the latest event time in nano-second in the window
+ */
+ long getLatestEventTimeNs();
+
+ /**
+ * Method to get the total number of {@link org.apache.samza.operators.data.MessageEnvelope}s received in the window
+ *
+ * @return number of {@link org.apache.samza.operators.data.MessageEnvelope}s in the window
+ */
+ long getNumberMessages();
+
+ /**
+ * Method to get the corresponding window's output value
+ *
+ * @return the corresponding window's output value
+ */
+ WV getOutputValue();
+
+ /**
+ * Method to set the corresponding window's output value
+ *
+ * @param value the corresponding window's output value
+ */
+ void setOutputValue(WV value);
+
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/windows/Windows.java
----------------------------------------------------------------------
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 0000000..1a4ed8f
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/windows/Windows.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.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 <M extends MessageEnvelope, WK, WV, WS extends WindowState<WV>, WM extends WindowOutput<WK, WV>> WindowFn<M, WK, WS, WM> getInternalWindowFn(
+ Window<M, WK, WV, WM> window) {
+ if (window instanceof SessionWindow) {
+ SessionWindow<M, WK, WV> sessionWindow = (SessionWindow<M, WK, WV>) window;
+ return (WindowFn<M, WK, WS, WM>) 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 <M> type of input {@link MessageEnvelope}
+ * @param <WK> type of the session window key
+ * @return the {@link Window} function for the session
+ */
+ public static <M extends MessageEnvelope, WK> Window<M, WK, Collection<M>, WindowOutput<WK, Collection<M>>> intoSessions(Function<M, WK> 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 <M> type of the input {@link MessageEnvelope}
+ * @param <WK> type of the session window key
+ * @param <SI> 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 <M extends MessageEnvelope, WK, SI> Window<M, WK, Collection<SI>, WindowOutput<WK, Collection<SI>>> intoSessions(Function<M, WK> sessionKeyFunction,
+ Function<M, SI> 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 <M> type of the input {@link MessageEnvelope}
+ * @param <WK> type of the session window key
+ * @return the {@link Window} function for the session
+ */
+ public static <M extends MessageEnvelope, WK> Window<M, WK, Integer, WindowOutput<WK, Integer>> intoSessionCounter(Function<M, WK> sessionKeyFunction) {
+ return new SessionWindow<>(sessionKeyFunction, (m, c) -> c + 1);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/test/java/org/apache/samza/operators/TestMessage.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/TestMessage.java b/samza-api/src/test/java/org/apache/samza/operators/TestMessage.java
deleted file mode 100644
index 8c56287..0000000
--- a/samza-api/src/test/java/org/apache/samza/operators/TestMessage.java
+++ /dev/null
@@ -1,47 +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.Message;
-
-
-public class TestMessage implements Message<String, String> {
-
- private final String key;
- private final String value;
- private final long timestamp;
-
- TestMessage(String key, String value, long timestamp) {
- this.key = key;
- this.value = value;
- this.timestamp = timestamp;
- }
-
- @Override public String getMessage() {
- return this.value;
- }
-
- @Override public String getKey() {
- return this.key;
- }
-
- @Override public long getTimestamp() {
- return this.timestamp;
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/test/java/org/apache/samza/operators/TestMessageEnvelope.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/TestMessageEnvelope.java b/samza-api/src/test/java/org/apache/samza/operators/TestMessageEnvelope.java
new file mode 100644
index 0000000..dfa69ac
--- /dev/null
+++ b/samza-api/src/test/java/org/apache/samza/operators/TestMessageEnvelope.java
@@ -0,0 +1,61 @@
+/*
+ * 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;
+
+
+public class TestMessageEnvelope implements MessageEnvelope<String, TestMessageEnvelope.MessageType> {
+
+ private final String key;
+ private final MessageType value;
+
+ public TestMessageEnvelope(String key, String value, long eventTime) {
+ this.key = key;
+ this.value = new MessageType(value, eventTime);
+ }
+
+ @Override
+ public MessageType getMessage() {
+ return this.value;
+ }
+
+ @Override
+ public String getKey() {
+ return this.key;
+ }
+
+ public class MessageType {
+ private final String value;
+ private final long eventTime;
+
+ public MessageType(String value, long eventTime) {
+ this.value = value;
+ this.eventTime = eventTime;
+ }
+
+ public long getEventTime() {
+ return eventTime;
+ }
+
+ public String getValue() {
+ return value;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/test/java/org/apache/samza/operators/TestMessageStream.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/TestMessageStream.java b/samza-api/src/test/java/org/apache/samza/operators/TestMessageStream.java
deleted file mode 100644
index 4dbe233..0000000
--- a/samza-api/src/test/java/org/apache/samza/operators/TestMessageStream.java
+++ /dev/null
@@ -1,180 +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.internal.Operators.*;
-import org.apache.samza.operators.internal.WindowOutput;
-import org.apache.samza.system.OutgoingMessageEnvelope;
-import org.apache.samza.system.SystemStream;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskCoordinator;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Set;
-import java.util.function.BiFunction;
-import java.util.function.Function;
-
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-
-public class TestMessageStream {
-
- @Test public void testMap() {
- MessageStream<TestMessage> inputStream = new MessageStream<>();
- Function<TestMessage, TestOutputMessage> xMap = m -> new TestOutputMessage(m.getKey(), m.getMessage().length() + 1, m.getTimestamp() + 2);
- MessageStream<TestOutputMessage> outputStream = inputStream.map(xMap);
- Collection<Operator> subs = inputStream.getSubscribers();
- assertEquals(subs.size(), 1);
- Operator<TestOutputMessage> mapOp = subs.iterator().next();
- assertTrue(mapOp instanceof StreamOperator);
- assertEquals(mapOp.getOutputStream(), outputStream);
- // assert that the transformation function is what we defined above
- TestMessage xTestMsg = mock(TestMessage.class);
- when(xTestMsg.getKey()).thenReturn("test-msg-key");
- when(xTestMsg.getMessage()).thenReturn("123456789");
- when(xTestMsg.getTimestamp()).thenReturn(12345L);
- Collection<TestOutputMessage> cOutputMsg = ((StreamOperator<TestMessage, TestOutputMessage>) mapOp).getFunction().apply(xTestMsg);
- assertEquals(cOutputMsg.size(), 1);
- TestOutputMessage outputMessage = cOutputMsg.iterator().next();
- assertEquals(outputMessage.getKey(), xTestMsg.getKey());
- assertEquals(outputMessage.getMessage(), Integer.valueOf(xTestMsg.getMessage().length() + 1));
- assertEquals(outputMessage.getTimestamp(), xTestMsg.getTimestamp() + 2);
- }
-
- @Test public void testFlatMap() {
- MessageStream<TestMessage> inputStream = new MessageStream<>();
- Set<TestOutputMessage> flatOuts = new HashSet<TestOutputMessage>() { {
- this.add(mock(TestOutputMessage.class));
- this.add(mock(TestOutputMessage.class));
- this.add(mock(TestOutputMessage.class));
- } };
- Function<TestMessage, Collection<TestOutputMessage>> xFlatMap = m -> flatOuts;
- MessageStream<TestOutputMessage> outputStream = inputStream.flatMap(xFlatMap);
- Collection<Operator> subs = inputStream.getSubscribers();
- assertEquals(subs.size(), 1);
- Operator<TestOutputMessage> flatMapOp = subs.iterator().next();
- assertTrue(flatMapOp instanceof StreamOperator);
- assertEquals(flatMapOp.getOutputStream(), outputStream);
- // assert that the transformation function is what we defined above
- assertEquals(((StreamOperator<TestMessage, TestOutputMessage>) flatMapOp).getFunction(), xFlatMap);
- }
-
- @Test public void testFilter() {
- MessageStream<TestMessage> inputStream = new MessageStream<>();
- Function<TestMessage, Boolean> xFilter = m -> m.getTimestamp() > 123456L;
- MessageStream<TestMessage> outputStream = inputStream.filter(xFilter);
- Collection<Operator> subs = inputStream.getSubscribers();
- assertEquals(subs.size(), 1);
- Operator<TestMessage> filterOp = subs.iterator().next();
- assertTrue(filterOp instanceof StreamOperator);
- assertEquals(filterOp.getOutputStream(), outputStream);
- // assert that the transformation function is what we defined above
- Function<TestMessage, Collection<TestMessage>> txfmFn = ((StreamOperator<TestMessage, TestMessage>) filterOp).getFunction();
- TestMessage mockMsg = mock(TestMessage.class);
- when(mockMsg.getTimestamp()).thenReturn(11111L);
- Collection<TestMessage> output = txfmFn.apply(mockMsg);
- assertTrue(output.isEmpty());
- when(mockMsg.getTimestamp()).thenReturn(999999L);
- output = txfmFn.apply(mockMsg);
- assertEquals(output.size(), 1);
- assertEquals(output.iterator().next(), mockMsg);
- }
-
- @Test public void testSink() {
- MessageStream<TestMessage> inputStream = new MessageStream<>();
- MessageStream.VoidFunction3<TestMessage, MessageCollector, TaskCoordinator> xSink = (m, mc, tc) -> {
- mc.send(new OutgoingMessageEnvelope(new SystemStream("test-sys", "test-stream"), m.getMessage()));
- tc.commit(TaskCoordinator.RequestScope.CURRENT_TASK);
- };
- inputStream.sink(xSink);
- Collection<Operator> subs = inputStream.getSubscribers();
- assertEquals(subs.size(), 1);
- Operator<TestMessage> sinkOp = subs.iterator().next();
- assertTrue(sinkOp instanceof SinkOperator);
- assertEquals(((SinkOperator) sinkOp).getFunction(), xSink);
- assertNull(((SinkOperator) sinkOp).getOutputStream());
- }
-
- @Test public void testWindow() {
- MessageStream<TestMessage> inputStream = new MessageStream<>();
- Windows.SessionWindow<TestMessage, String, Integer> window = mock(Windows.SessionWindow.class);
- MessageStream<WindowOutput<String, Integer>> outStream = inputStream.window(window);
- Collection<Operator> subs = inputStream.getSubscribers();
- assertEquals(subs.size(), 1);
- Operator<TestMessage> wndOp = subs.iterator().next();
- assertTrue(wndOp instanceof WindowOperator);
- assertEquals(((WindowOperator) wndOp).getOutputStream(), outStream);
- }
-
- @Test public void testJoin() {
- MessageStream<TestMessage> source1 = new MessageStream<>();
- MessageStream<TestMessage> source2 = new MessageStream<>();
- BiFunction<TestMessage, TestMessage, TestOutputMessage> joiner = (m1, m2) -> new TestOutputMessage(m1.getKey(), m1.getMessage().length() + m2.getMessage().length(), m1.getTimestamp());
- MessageStream<TestOutputMessage> joinOutput = source1.join(source2, joiner);
- Collection<Operator> subs = source1.getSubscribers();
- assertEquals(subs.size(), 1);
- Operator<TestMessage> joinOp1 = subs.iterator().next();
- assertTrue(joinOp1 instanceof PartialJoinOperator);
- assertEquals(((PartialJoinOperator) joinOp1).getOutputStream(), joinOutput);
- subs = source2.getSubscribers();
- assertEquals(subs.size(), 1);
- Operator<TestMessage> joinOp2 = subs.iterator().next();
- assertTrue(joinOp2 instanceof PartialJoinOperator);
- assertEquals(((PartialJoinOperator) joinOp2).getOutputStream(), joinOutput);
- TestMessage joinMsg1 = new TestMessage("test-join-1", "join-msg-001", 11111L);
- TestMessage joinMsg2 = new TestMessage("test-join-2", "join-msg-002", 22222L);
- TestOutputMessage xOut = (TestOutputMessage) ((PartialJoinOperator) joinOp1).getFunction().apply(joinMsg1, joinMsg2);
- assertEquals(xOut.getKey(), "test-join-1");
- assertEquals(xOut.getMessage(), Integer.valueOf(24));
- assertEquals(xOut.getTimestamp(), 11111L);
- xOut = (TestOutputMessage) ((PartialJoinOperator) joinOp2).getFunction().apply(joinMsg2, joinMsg1);
- assertEquals(xOut.getKey(), "test-join-1");
- assertEquals(xOut.getMessage(), Integer.valueOf(24));
- assertEquals(xOut.getTimestamp(), 11111L);
- }
-
- @Test public void testMerge() {
- MessageStream<TestMessage> merge1 = new MessageStream<>();
- Collection<MessageStream<TestMessage>> others = new ArrayList<MessageStream<TestMessage>>() { {
- this.add(new MessageStream<>());
- this.add(new MessageStream<>());
- } };
- MessageStream<TestMessage> mergeOutput = merge1.merge(others);
- validateMergeOperator(merge1, mergeOutput);
-
- others.forEach(merge -> validateMergeOperator(merge, mergeOutput));
- }
-
- private void validateMergeOperator(MessageStream<TestMessage> mergeSource, MessageStream<TestMessage> mergeOutput) {
- Collection<Operator> subs = mergeSource.getSubscribers();
- assertEquals(subs.size(), 1);
- Operator<TestMessage> mergeOp = subs.iterator().next();
- assertTrue(mergeOp instanceof StreamOperator);
- assertEquals(((StreamOperator) mergeOp).getOutputStream(), mergeOutput);
- TestMessage mockMsg = mock(TestMessage.class);
- Collection<TestMessage> outputs = ((StreamOperator<TestMessage, TestMessage>) mergeOp).getFunction().apply(mockMsg);
- assertEquals(outputs.size(), 1);
- assertEquals(outputs.iterator().next(), mockMsg);
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/test/java/org/apache/samza/operators/TestMessageStreams.java
----------------------------------------------------------------------
diff --git a/samza-api/src/test/java/org/apache/samza/operators/TestMessageStreams.java b/samza-api/src/test/java/org/apache/samza/operators/TestMessageStreams.java
deleted file mode 100644
index c5fcceb..0000000
--- a/samza-api/src/test/java/org/apache/samza/operators/TestMessageStreams.java
+++ /dev/null
@@ -1,35 +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.Partition;
-import org.apache.samza.system.SystemStreamPartition;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-
-
-public class TestMessageStreams {
-
- @Test public void testInput() {
- SystemStreamPartition ssp = new SystemStreamPartition("my-system", "my-stream", new Partition(0));
- MessageStreams.SystemMessageStream mSysStream = MessageStreams.input(ssp);
- assertEquals(mSysStream.getSystemStreamPartition(), ssp);
- }
-}
[6/6] samza git commit: SAMZA-1054: Refactor Operator APIs
Posted by ni...@apache.org.
SAMZA-1054: Refactor Operator APIs
Some suggestions for an Operator API refactor and misc. cleanup. It does contain some implementation changes, mostly due to deleted, extracted or merged classes. (e.g. OperatorFactory + ChainedOperators == OperatorImpls).
Since git marked several moved classes as (delete + new) instead, it's probably best to apply the diff locally and browse the code in an IDE.
Some of the changes, in no particular order:
* Extracted XFunction interfaces into a .functions package in -api.
* -api's internal.Operators is now the -operators's spec.* package. Extracted interfaces and classes. Factory methods are now in OperatorSpecs.
* -api's MessageStreams is now -api's MessageStream interface and -operators's MessageStreamImpl.
* -api's internal.Windows classes are now in -api's .window package. Extracted interfaces and classes, but no implementation changes.
* OperatorFactory + ChainedOperators is now OperatorImpls, which is used from StreamOperatorAdaptorTask.
* Added a NoOpOperatorImpl, which acts as the root node for the OperatorImpl DAG returned by OperatorImpls.
* Removed usages of reactivestreams APIs since current code looks simpler without them. We can add them back when we need features like backpressure etc.
* Removed the InputSystemMessage interface.
* Made field names consistent (e.g Fn suffix for functions everywhere etc.).
* Some method/class visibility changes due to moved classes.
* General documentation changes, mostly to make public APIs clearer.
There are additional questions/tasks that we can address in future RBs:
* Updating Window and Trigger APIs.
* Merging samza-operator into samza-core.
* Questions about Message timestamp and Offset comparison semantics.
* Questions about OperatorSpec serialization (e.g. ID generation).
* Questions about StateStoreImpl and StoreFunctions.
Author: Prateek Maheshwari <pm...@linkedin.com>
Reviewers: Yi Pan <ni...@gmail.com>, Jagadish <ja...@gmail.com>
Closes #25 from prateekm/master
Project: http://git-wip-us.apache.org/repos/asf/samza/repo
Commit: http://git-wip-us.apache.org/repos/asf/samza/commit/00543804
Tree: http://git-wip-us.apache.org/repos/asf/samza/tree/00543804
Diff: http://git-wip-us.apache.org/repos/asf/samza/diff/00543804
Branch: refs/heads/master
Commit: 00543804b3c32f1cbea0212e4a94e360b5a324cc
Parents: a980c96
Author: Prateek Maheshwari <pm...@linkedin.com>
Authored: Thu Dec 1 14:50:52 2016 -0800
Committer: Yi Pan (Data Infrastructure) <ni...@gmail.com>
Committed: Thu Dec 1 14:50:52 2016 -0800
----------------------------------------------------------------------
.../apache/samza/operators/MessageStream.java | 197 +++-----
.../apache/samza/operators/MessageStreams.java | 81 ----
.../samza/operators/StreamOperatorTask.java | 51 ++
.../apache/samza/operators/TriggerBuilder.java | 323 -------------
.../org/apache/samza/operators/WindowState.java | 81 ----
.../org/apache/samza/operators/Windows.java | 203 --------
.../operators/data/IncomingSystemMessage.java | 76 ---
.../data/IncomingSystemMessageEnvelope.java | 63 +++
.../operators/data/InputSystemMessage.java | 45 --
.../apache/samza/operators/data/Message.java | 64 ---
.../samza/operators/data/MessageEnvelope.java | 54 +++
.../operators/functions/FilterFunction.java | 40 ++
.../operators/functions/FlatMapFunction.java | 44 ++
.../samza/operators/functions/JoinFunction.java | 44 ++
.../samza/operators/functions/MapFunction.java | 41 ++
.../samza/operators/functions/SinkFunction.java | 46 ++
.../samza/operators/internal/Operators.java | 469 -------------------
.../samza/operators/internal/Trigger.java | 95 ----
.../samza/operators/internal/WindowFn.java | 60 ---
.../samza/operators/internal/WindowOutput.java | 55 ---
.../operators/task/StreamOperatorTask.java | 45 --
.../samza/operators/windows/SessionWindow.java | 102 ++++
.../samza/operators/windows/StoreFunctions.java | 67 +++
.../apache/samza/operators/windows/Trigger.java | 94 ++++
.../samza/operators/windows/TriggerBuilder.java | 320 +++++++++++++
.../apache/samza/operators/windows/Window.java | 49 ++
.../samza/operators/windows/WindowFn.java | 59 +++
.../samza/operators/windows/WindowOutput.java | 51 ++
.../samza/operators/windows/WindowState.java | 85 ++++
.../apache/samza/operators/windows/Windows.java | 100 ++++
.../org/apache/samza/operators/TestMessage.java | 47 --
.../samza/operators/TestMessageEnvelope.java | 61 +++
.../samza/operators/TestMessageStream.java | 180 -------
.../samza/operators/TestMessageStreams.java | 35 --
.../samza/operators/TestOutputMessage.java | 47 --
.../operators/TestOutputMessageEnvelope.java | 43 ++
.../samza/operators/TestTriggerBuilder.java | 214 ---------
.../org/apache/samza/operators/TestWindows.java | 106 -----
.../data/TestIncomingSystemMessage.java | 5 +-
.../samza/operators/data/TestLongOffset.java | 9 +-
.../samza/operators/internal/TestOperators.java | 128 -----
.../samza/operators/internal/TestTrigger.java | 68 ---
.../operators/internal/TestWindowOutput.java | 36 --
.../samza/operators/windows/TestTrigger.java | 68 +++
.../operators/windows/TestTriggerBuilder.java | 226 +++++++++
.../operators/windows/TestWindowOutput.java | 36 ++
.../samza/operators/windows/TestWindows.java | 109 +++++
.../samza/operators/MessageStreamImpl.java | 134 ++++++
.../apache/samza/operators/StateStoreImpl.java | 56 +++
.../operators/StreamOperatorAdaptorTask.java | 105 +++++
.../samza/operators/impl/ChainedOperators.java | 119 -----
.../samza/operators/impl/OperatorFactory.java | 85 ----
.../samza/operators/impl/OperatorImpl.java | 76 +--
.../samza/operators/impl/OperatorImpls.java | 125 +++++
.../operators/impl/PartialJoinOperatorImpl.java | 46 ++
.../samza/operators/impl/ProcessorContext.java | 53 ---
.../samza/operators/impl/RootOperatorImpl.java | 36 ++
.../impl/SessionWindowOperatorImpl.java | 67 +++
.../operators/impl/SimpleOperatorImpl.java | 49 --
.../samza/operators/impl/SinkOperatorImpl.java | 22 +-
.../samza/operators/impl/StateStoreImpl.java | 56 ---
.../operators/impl/StreamOperatorImpl.java | 47 ++
.../operators/impl/join/PartialJoinOpImpl.java | 44 --
.../impl/window/SessionWindowImpl.java | 65 ---
.../samza/operators/spec/OperatorSpec.java | 37 ++
.../samza/operators/spec/OperatorSpecs.java | 116 +++++
.../operators/spec/PartialJoinOperatorSpec.java | 104 ++++
.../samza/operators/spec/SinkOperatorSpec.java | 62 +++
.../operators/spec/StreamOperatorSpec.java | 67 +++
.../operators/spec/WindowOperatorSpec.java | 119 +++++
.../samza/task/StreamOperatorAdaptorTask.java | 85 ----
.../apache/samza/operators/BroadcastTask.java | 101 ++++
.../org/apache/samza/operators/JoinTask.java | 77 +++
.../operators/TestFluentStreamAdaptorTask.java | 85 ++++
.../samza/operators/TestFluentStreamTasks.java | 112 +++++
.../samza/operators/TestMessageStreamImpl.java | 203 ++++++++
.../samza/operators/TestStateStoreImpl.java | 72 +++
.../org/apache/samza/operators/WindowTask.java | 70 +++
.../data/JsonIncomingSystemMessageEnvelope.java | 60 +++
.../operators/impl/TestChainedOperators.java | 129 -----
.../operators/impl/TestOperatorFactory.java | 93 ----
.../samza/operators/impl/TestOperatorImpl.java | 48 +-
.../samza/operators/impl/TestOperatorImpls.java | 183 ++++++++
.../operators/impl/TestProcessorContext.java | 40 --
.../operators/impl/TestSessionWindowImpl.java | 111 +++++
.../operators/impl/TestSimpleOperatorImpl.java | 55 ---
.../operators/impl/TestSinkOperatorImpl.java | 25 +-
.../operators/impl/TestStateStoreImpl.java | 69 ---
.../operators/impl/TestStreamOperatorImpl.java | 60 +++
.../impl/window/TestSessionWindowImpl.java | 105 -----
.../samza/operators/spec/TestOperatorSpecs.java | 114 +++++
.../samza/task/BroadcastOperatorTask.java | 102 ----
.../samza/task/InputJsonSystemMessage.java | 67 ---
.../org/apache/samza/task/JoinOperatorTask.java | 80 ----
.../task/TestStreamOperatorAdaptorTask.java | 80 ----
.../samza/task/TestStreamOperatorTasks.java | 105 -----
.../apache/samza/task/WindowOperatorTask.java | 71 ---
97 files changed, 4374 insertions(+), 4240 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/MessageStream.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 dede631..d18536b 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
@@ -16,173 +16,112 @@
* specific language governing permissions and limitations
* under the License.
*/
-
package org.apache.samza.operators;
import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.operators.data.Message;
-import org.apache.samza.operators.internal.Operators;
-import org.apache.samza.operators.internal.Operators.Operator;
-import org.apache.samza.operators.internal.WindowOutput;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskCoordinator;
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.functions.FilterFunction;
+import org.apache.samza.operators.functions.FlatMapFunction;
+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.ArrayList;
import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Set;
-import java.util.function.BiFunction;
-import java.util.function.Function;
/**
- * This class defines either the input or output streams to/from the operators. Users use the API methods defined here to
- * directly program the stream processing stages that processes a stream and generate another one.
+ * Represents a stream of {@link MessageEnvelope}s.
+ * <p>
+ * A {@link MessageStream} can be transformed into another {@link MessageStream} by applying the transforms in this API.
*
- * @param <M> Type of message in this stream
+ * @param <M> type of {@link MessageEnvelope}s in this stream
*/
@InterfaceStability.Unstable
-public class MessageStream<M extends Message> {
-
- private final Set<Operator> subscribers = new HashSet<>();
-
- /**
- * Helper method to get the corresponding list of subscribers to a specific {@link MessageStream}.
- *
- * NOTE: This is purely an internal API and should not be used directly by programmers.
- *
- * @return A unmodifiable set containing all {@link Operator}s that subscribe to this {@link MessageStream} object
- */
- public Collection<Operator> getSubscribers() {
- return Collections.unmodifiableSet(this.subscribers);
- }
-
- /**
- * Public API methods start here
- */
+public interface MessageStream<M extends MessageEnvelope> {
/**
- * Defines a function API that takes three input parameters w/ types {@code A}, {@code B}, and {@code C} and w/o a return value
+ * Applies the provided 1:1 {@link MapFunction} to {@link MessageEnvelope}s in this {@link MessageStream} and returns the
+ * transformed {@link MessageStream}.
*
- * @param <A> the type of input {@code a}
- * @param <B> the type of input {@code b}
- * @param <C> the type of input {@code c}
+ * @param mapFn the function to transform a {@link MessageEnvelope} to another {@link MessageEnvelope}
+ * @param <TM> the type of {@link MessageEnvelope}s in the transformed {@link MessageStream}
+ * @return the transformed {@link MessageStream}
*/
- @FunctionalInterface
- public interface VoidFunction3<A, B, C> {
- public void apply(A a, B b, C c);
- }
+ <TM extends MessageEnvelope> MessageStream<TM> map(MapFunction<M, TM> mapFn);
/**
- * Method to apply a map function (1:1) on a {@link MessageStream}
+ * Applies the provided 1:n {@link FlatMapFunction} to transform a {@link MessageEnvelope} in this {@link MessageStream}
+ * to n {@link MessageEnvelope}s in the transformed {@link MessageStream}
*
- * @param mapper the mapper function to map one input {@link Message} to one output {@link Message}
- * @param <OM> the type of the output {@link Message} in the output {@link MessageStream}
- * @return the output {@link MessageStream} by applying the map function on the input {@link MessageStream}
+ * @param flatMapFn the function to transform a {@link MessageEnvelope} to zero or more {@link MessageEnvelope}s
+ * @param <TM> the type of {@link MessageEnvelope}s in the transformed {@link MessageStream}
+ * @return the transformed {@link MessageStream}
*/
- public <OM extends Message> MessageStream<OM> map(Function<M, OM> mapper) {
- Operator<OM> op = Operators.<M, OM>getStreamOperator(m -> new ArrayList<OM>() { {
- OM r = mapper.apply(m);
- if (r != null) {
- this.add(r);
- }
- } });
- this.subscribers.add(op);
- return op.getOutputStream();
- }
+ <TM extends MessageEnvelope> MessageStream<TM> flatMap(FlatMapFunction<M, TM> flatMapFn);
/**
- * Method to apply a flatMap function (1:n) on a {@link MessageStream}
+ * Applies the provided {@link FilterFunction} to {@link MessageEnvelope}s in this {@link MessageStream} and returns the
+ * transformed {@link MessageStream}.
+ * <p>
+ * The {@link FilterFunction} is a predicate which determines whether a {@link MessageEnvelope} in this {@link MessageStream}
+ * should be retained in the transformed {@link MessageStream}.
*
- * @param flatMapper the flat mapper function to map one input {@link Message} to zero or more output {@link Message}s
- * @param <OM> the type of the output {@link Message} in the output {@link MessageStream}
- * @return the output {@link MessageStream} by applying the map function on the input {@link MessageStream}
+ * @param filterFn the predicate to filter {@link MessageEnvelope}s from this {@link MessageStream}
+ * @return the transformed {@link MessageStream}
*/
- public <OM extends Message> MessageStream<OM> flatMap(Function<M, Collection<OM>> flatMapper) {
- Operator<OM> op = Operators.getStreamOperator(flatMapper);
- this.subscribers.add(op);
- return op.getOutputStream();
- }
+ MessageStream<M> filter(FilterFunction<M> filterFn);
/**
- * Method to apply a filter function on a {@link MessageStream}
+ * Allows sending {@link MessageEnvelope}s in this {@link MessageStream} to an output
+ * {@link org.apache.samza.system.SystemStream} using the provided {@link SinkFunction}.
*
- * @param filter the filter function to filter input {@link Message}s from the input {@link MessageStream}
- * @return the output {@link MessageStream} after applying the filter function on the input {@link MessageStream}
+ * @param sinkFn the function to send {@link MessageEnvelope}s in this stream to output systems
*/
- public MessageStream<M> filter(Function<M, Boolean> filter) {
- Operator<M> op = Operators.<M, M>getStreamOperator(t -> new ArrayList<M>() { {
- if (filter.apply(t)) {
- this.add(t);
- }
- } });
- this.subscribers.add(op);
- return op.getOutputStream();
- }
+ void sink(SinkFunction<M> sinkFn);
/**
- * Method to send an input {@link MessageStream} to an output {@link org.apache.samza.system.SystemStream}, and allows the output {@link MessageStream}
- * to be consumed by downstream stream operators again.
+ * 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.
+ * <p>
+ * Use the {@link org.apache.samza.operators.windows.Windows} helper methods to create the appropriate windows.
*
- * @param sink the user-defined sink function to send the input {@link Message}s to the external output systems
+ * @param window the {@link Window} to group and process {@link MessageEnvelope}s from this {@link MessageStream}
+ * @param <WK> the type of key in the {@link WindowOutput} from the {@link Window}
+ * @param <WV> the type of value in the {@link WindowOutput} from the {@link Window}
+ * @param <WS> the type of window state kept in the {@link Window}
+ * @param <WM> the type of {@link WindowOutput} in the transformed {@link MessageStream}
+ * @return the transformed {@link MessageStream}
*/
- public void sink(VoidFunction3<M, MessageCollector, TaskCoordinator> sink) {
- this.subscribers.add(Operators.getSinkOperator(sink));
- }
+ <WK, WV, WS extends WindowState<WV>, WM extends WindowOutput<WK, WV>> MessageStream<WM> window(
+ Window<M, WK, WV, WM> window);
/**
- * Method to perform a window function (i.e. a group-by, aggregate function) on a {@link MessageStream}
+ * Joins this {@link MessageStream} with another {@link MessageStream} using the provided pairwise {@link JoinFunction}.
+ * <p>
+ * We currently only support 2-way joins.
*
- * @param window the window function to group and aggregate the input {@link Message}s from the input {@link MessageStream}
- * @param <WK> the type of key in the output {@link Message} from the {@link Windows.Window} function
- * @param <WV> the type of output value from
- * @param <WS> the type of window state kept in the {@link Windows.Window} function
- * @param <WM> the type of {@link org.apache.samza.operators.internal.WindowOutput} message from the {@link Windows.Window} function
- * @return the output {@link MessageStream} after applying the window function on the input {@link MessageStream}
- */
- public <WK, WV, WS extends WindowState<WV>, WM extends WindowOutput<WK, WV>> MessageStream<WM> window(Windows.Window<M, WK, WV, WM> window) {
- Operator<WM> wndOp = Operators.getWindowOperator(Windows.getInternalWindowFn(window));
- this.subscribers.add(wndOp);
- return wndOp.getOutputStream();
- }
-
- /**
- * Method to add an input {@link MessageStream} to a join function. Note that we currently only support 2-way joins.
- *
- * @param other the other stream to be joined w/
- * @param merger the common function to merge messages from this {@link MessageStream} and {@code other}
+ * @param otherStream the other {@link MessageStream} to be joined with
+ * @param joinFn the function to join {@link MessageEnvelope}s from this and the other {@link MessageStream}
* @param <K> the type of join key
- * @param <JM> the type of message in the {@link Message} from the other join stream
- * @param <RM> the type of message in the {@link Message} from the join function
- * @return the output {@link MessageStream} from the join function {@code joiner}
+ * @param <OM> the type of {@link MessageEnvelope}s in the other stream
+ * @param <RM> the type of {@link MessageEnvelope}s resulting from the {@code joinFn}
+ * @return the joined {@link MessageStream}
*/
- public <K, JM extends Message<K, ?>, RM extends Message> MessageStream<RM> join(MessageStream<JM> other,
- BiFunction<M, JM, RM> merger) {
- MessageStream<RM> outputStream = new MessageStream<>();
-
- BiFunction<M, JM, RM> parJoin1 = merger::apply;
- BiFunction<JM, M, RM> parJoin2 = (m, t1) -> merger.apply(t1, m);
-
- // TODO: need to add default store functions for the two partial join functions
-
- other.subscribers.add(Operators.<JM, K, M, RM>getPartialJoinOperator(parJoin2, outputStream));
- this.subscribers.add(Operators.<M, K, JM, RM>getPartialJoinOperator(parJoin1, outputStream));
- return outputStream;
- }
+ <K, OM extends MessageEnvelope<K, ?>, RM extends MessageEnvelope> MessageStream<RM> join(MessageStream<OM> otherStream,
+ JoinFunction<M, OM, RM> joinFn);
/**
- * Method to merge all {@code others} streams w/ this {@link MessageStream}. The merging streams must have the same type {@code M}
+ * Merge all {@code otherStreams} with this {@link MessageStream}.
+ * <p>
+ * The merging streams must have the same {@link MessageEnvelope} type {@code M}.
*
- * @param others other streams to be merged w/ this one
- * @return the merged output stream
+ * @param otherStreams other {@link MessageStream}s to be merged with this {@link MessageStream}
+ * @return the merged {@link MessageStream}
*/
- public MessageStream<M> merge(Collection<MessageStream<M>> others) {
- MessageStream<M> outputStream = new MessageStream<>();
-
- others.add(this);
- others.forEach(other -> other.subscribers.add(Operators.getMergeOperator(outputStream)));
- return outputStream;
- }
-
+ MessageStream<M> merge(Collection<MessageStream<M>> otherStreams);
+
}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/MessageStreams.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/MessageStreams.java b/samza-api/src/main/java/org/apache/samza/operators/MessageStreams.java
deleted file mode 100644
index 51bf482..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/MessageStreams.java
+++ /dev/null
@@ -1,81 +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.annotation.InterfaceStability;
-import org.apache.samza.operators.data.IncomingSystemMessage;
-import org.apache.samza.system.SystemStreamPartition;
-
-
-/**
- * This class defines all methods to create a {@link MessageStream} object. Users can use this to create an {@link MessageStream}
- * from a specific input source.
- *
- */
-@InterfaceStability.Unstable
-public final class MessageStreams {
-
- /**
- * private constructor to prevent instantiation
- */
- private MessageStreams() {}
-
- /**
- * private class for system input/output {@link MessageStream}
- */
- public static final class SystemMessageStream extends MessageStream<IncomingSystemMessage> {
- /**
- * The corresponding {@link org.apache.samza.system.SystemStream}
- */
- private final SystemStreamPartition ssp;
-
- /**
- * Constructor for input system stream
- *
- * @param ssp the input {@link SystemStreamPartition} for the input {@link SystemMessageStream}
- */
- private SystemMessageStream(SystemStreamPartition ssp) {
- this.ssp = ssp;
- }
-
- /**
- * Getter for the {@link SystemStreamPartition} of the input
- *
- * @return the input {@link SystemStreamPartition}
- */
- public SystemStreamPartition getSystemStreamPartition() {
- return this.ssp;
- }
- }
-
- /**
- * Public static API methods start here
- */
-
- /**
- * Static API method to create a {@link MessageStream} from a system input stream
- *
- * @param ssp the input {@link SystemStreamPartition}
- * @return the {@link MessageStream} object takes {@code ssp} as the input
- */
- public static SystemMessageStream input(SystemStreamPartition ssp) {
- return new SystemMessageStream(ssp);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/StreamOperatorTask.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/StreamOperatorTask.java b/samza-api/src/main/java/org/apache/samza/operators/StreamOperatorTask.java
new file mode 100644
index 0000000..16cf27a
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/StreamOperatorTask.java
@@ -0,0 +1,51 @@
+/*
+ * 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.annotation.InterfaceStability;
+import org.apache.samza.operators.data.IncomingSystemMessageEnvelope;
+import org.apache.samza.system.SystemStreamPartition;
+
+import java.util.Map;
+
+
+/**
+ * A {@link StreamOperatorTask} is the basic interface to implement for processing {@link MessageStream}s.
+ * Implementations can describe the transformation steps for each {@link MessageStream} in the
+ * {@link #transform} method using {@link MessageStream} APIs.
+ * <p>
+ * Implementations may be augmented by implementing {@link org.apache.samza.task.InitableTask},
+ * {@link org.apache.samza.task.WindowableTask} and {@link org.apache.samza.task.ClosableTask} interfaces,
+ * but should not implement {@link org.apache.samza.task.StreamTask} or {@link org.apache.samza.task.AsyncStreamTask}
+ * interfaces.
+ */
+@InterfaceStability.Unstable
+public interface StreamOperatorTask {
+
+ /**
+ * Describe the transformation steps for each {@link MessageStream}s for this task using the
+ * {@link MessageStream} APIs. Each {@link MessageStream} corresponds to one {@link SystemStreamPartition}
+ * in the input system.
+ *
+ * @param messageStreams the {@link MessageStream}s that receive {@link IncomingSystemMessageEnvelope}s
+ * from their corresponding {@link org.apache.samza.system.SystemStreamPartition}
+ */
+ void transform(Map<SystemStreamPartition, MessageStream<IncomingSystemMessageEnvelope>> messageStreams);
+
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/TriggerBuilder.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/TriggerBuilder.java b/samza-api/src/main/java/org/apache/samza/operators/TriggerBuilder.java
deleted file mode 100644
index 5b3f4d0..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/TriggerBuilder.java
+++ /dev/null
@@ -1,323 +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.annotation.InterfaceStability;
-import org.apache.samza.operators.data.Message;
-import org.apache.samza.operators.internal.Trigger;
-
-import java.util.concurrent.TimeUnit;
-import java.util.function.BiFunction;
-import java.util.function.Function;
-
-
-/**
- * This class defines a builder of {@link org.apache.samza.operators.internal.Trigger} object for a {@link Windows.Window}. The triggers are categorized into
- * three types:
- *
- * <p>
- * 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
- * </p>
- *
- * If multiple conditions are defined for a specific type of trigger, the aggregated trigger is the disjunction of the each individual trigger (i.e. OR).
- *
- * NOTE: Programmers should not use classes defined in {@link org.apache.samza.operators.internal} to create triggers
- *
- *
- * @param <M> the type of input {@link Message} to the {@link Windows.Window}
- * @param <V> the type of output value from the {@link Windows.Window}
- */
-@InterfaceStability.Unstable
-public final class TriggerBuilder<M extends Message, V> {
-
- /**
- * Predicate helper to OR multiple trigger conditions
- */
- static class PredicateHelper {
- static <M, S> BiFunction<M, S, Boolean> or(BiFunction<M, S, Boolean> lhs, BiFunction<M, S, Boolean> rhs) {
- return (m, s) -> lhs.apply(m, s) || rhs.apply(m, s);
- }
-
- static <S> Function<S, Boolean> or(Function<S, Boolean> lhs, Function<S, Boolean> rhs) {
- return s -> lhs.apply(s) || rhs.apply(s);
- }
- }
-
- /**
- * The early trigger condition that determines the first output from the {@link Windows.Window}
- */
- private BiFunction<M, WindowState<V>, Boolean> earlyTrigger = null;
-
- /**
- * The late trigger condition that determines the late output(s) from the {@link Windows.Window}
- */
- private BiFunction<M, WindowState<V>, Boolean> lateTrigger = null;
-
- /**
- * The system timer based trigger conditions that guarantees the {@link Windows.Window} proceeds forward
- */
- private Function<WindowState<V>, Boolean> timerTrigger = null;
-
- /**
- * The state updater function to be applied after the first output is triggered
- */
- private Function<WindowState<V>, WindowState<V>> earlyTriggerUpdater = Function.identity();
-
- /**
- * The state updater function to be applied after the late output is triggered
- */
- private Function<WindowState<V>, WindowState<V>> 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<M, WindowState<V>, Boolean> addTrigger(BiFunction<M, WindowState<V>, Boolean> currentTrigger,
- BiFunction<M, WindowState<V>, 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<WindowState<V>, Boolean> addTimerTrigger(Function<WindowState<V>, Boolean> currentTimer,
- Function<WindowState<V>, 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 messages 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 Message}
- * @param wndLenMs the window length in event time in milli-second
- */
- private TriggerBuilder(Function<M, Long> 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 message as the early trigger
- *
- * @param tokenFunc the function that checks whether an input {@link Message} is a token message that triggers window output
- */
- private TriggerBuilder(Function<M, Boolean> tokenFunc) {
- this.earlyTrigger = (m, s) -> tokenFunc.apply(m);
- }
-
- /**
- * Build method that creates an {@link org.apache.samza.operators.internal.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 org.apache.samza.operators.internal.Trigger} object
- */
- Trigger<M, WindowState<V>> 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<M, V> onEarlyTrigger(Function<V, V> 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<M, V> onLateTrigger(Function<V, V> 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 message received in the window
- *
- * @param timeoutMs the timeout in ms after the last message received in the window
- * @return the {@link TriggerBuilder} object
- */
- public TriggerBuilder<M, V> 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 message received in the window
- *
- * @param timeoutMs the timeout in ms after the first message received in the window
- * @return the {@link TriggerBuilder} object
- */
- public TriggerBuilder<M, V> 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 messages in window
- * @return the {@link TriggerBuilder} object
- */
- public TriggerBuilder<M, V> 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 message and the window state
- *
- * @param lateTrigger the late trigger condition based on input {@link Message} and the current {@link WindowState}
- * @return the {@link TriggerBuilder} object
- */
- public TriggerBuilder<M, V> addLateTrigger(BiFunction<M, WindowState<V>, 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 <M> the type of input {@link Message}
- * @param <V> the type of {@link Windows.Window} output value
- * @return the {@link TriggerBuilder} object
- */
- public static <M extends Message, V> TriggerBuilder<M, V> earlyTriggerWhenExceedWndLen(long sizeLimit) {
- return new TriggerBuilder<M, V>(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 message
- * @param eventTimeWndSizeMs the event time window size in Ms
- * @param <M> the type of input {@link Message}
- * @param <V> the type of {@link Windows.Window} output value
- * @return the {@link TriggerBuilder} object
- */
- public static <M extends Message, V> TriggerBuilder<M, V> earlyTriggerOnEventTime(Function<M, Long> eventTimeFunc, long eventTimeWndSizeMs) {
- return new TriggerBuilder<M, V>(eventTimeFunc, eventTimeWndSizeMs);
- }
-
- /**
- * Static API method to create a {@link TriggerBuilder} w/ early trigger condition based on token messages
- *
- * @param tokenFunc the function to determine whether an input message is a window token or not
- * @param <M> the type of input {@link Message}
- * @param <V> the type of {@link Windows.Window} output value
- * @return the {@link TriggerBuilder} object
- */
- public static <M extends Message, V> TriggerBuilder<M, V> earlyTriggerOnTokenMsg(Function<M, Boolean> tokenFunc) {
- return new TriggerBuilder<M, V>(tokenFunc);
- }
-
- /**
- * Static API method to allow customized early trigger condition based on input {@link Message} and the corresponding {@link WindowState}
- *
- * @param earlyTrigger the user defined early trigger condition
- * @param <M> the input message type
- * @param <V> the output value from the window
- * @return the {@link TriggerBuilder} object
- */
- public static <M extends Message, V> TriggerBuilder<M, V> earlyTrigger(BiFunction<M, WindowState<V>, Boolean> earlyTrigger) {
- TriggerBuilder<M, V> newTriggers = new TriggerBuilder<M, V>();
- newTriggers.earlyTrigger = newTriggers.addTrigger(newTriggers.earlyTrigger, earlyTrigger);
- return newTriggers;
- }
-
- /**
- * Static API method to create a {@link TriggerBuilder} w/ system timeout after the last message received in the window
- *
- * @param timeoutMs timeout in ms after the last message received
- * @param <M> the type of input {@link Message}
- * @param <V> the type of {@link Windows.Window} output value
- * @return the {@link TriggerBuilder} object
- */
- public static <M extends Message, V> TriggerBuilder<M, V> timeoutSinceLastMessage(long timeoutMs) {
- return new TriggerBuilder<M, V>().addTimeoutSinceLastMessage(timeoutMs);
- }
-
- /**
- * Static API method to create a {@link TriggerBuilder} w/ system timeout after the first message received in the window
- *
- * @param timeoutMs timeout in ms after the first message received
- * @param <M> the type of input {@link Message}
- * @param <V> the type of {@link Windows.Window} output value
- * @return the {@link TriggerBuilder} object
- */
- public static <M extends Message, V> TriggerBuilder<M, V> timeoutSinceFirstMessage(long timeoutMs) {
- return new TriggerBuilder<M, V>().addTimeoutSinceFirstMessage(timeoutMs);
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/WindowState.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/WindowState.java b/samza-api/src/main/java/org/apache/samza/operators/WindowState.java
deleted file mode 100644
index 759f2d8..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/WindowState.java
+++ /dev/null
@@ -1,81 +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.annotation.InterfaceStability;
-
-
-/**
- * This interface defines the methods a window state class has to implement. The programmers are allowed to implement
- * customized window state to be stored in window state stores by implementing this interface class.
- *
- * @param <WV> the type for window output value
- */
-@InterfaceStability.Unstable
-public interface WindowState<WV> {
- /**
- * Method to get the system time when the first message in the window is received
- *
- * @return nano-second of system time for the first message received in the window
- */
- long getFirstMessageTimeNs();
-
- /**
- * Method to get the system time when the last message in the window is received
- *
- * @return nano-second of system time for the last message received in the window
- */
- long getLastMessageTimeNs();
-
- /**
- * Method to get the earliest event time in the window
- *
- * @return the earliest event time in nano-second in the window
- */
- long getEarliestEventTimeNs();
-
- /**
- * Method to get the latest event time in the window
- *
- * @return the latest event time in nano-second in the window
- */
- long getLatestEventTimeNs();
-
- /**
- * Method to get the total number of messages received in the window
- *
- * @return number of messages in the window
- */
- long getNumberMessages();
-
- /**
- * Method to get the corresponding window's output value
- *
- * @return the corresponding window's output value
- */
- WV getOutputValue();
-
- /**
- * Method to set the corresponding window's output value
- *
- * @param value the corresponding window's output value
- */
- void setOutputValue(WV value);
-
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/Windows.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/Windows.java b/samza-api/src/main/java/org/apache/samza/operators/Windows.java
deleted file mode 100644
index 6619f41..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/Windows.java
+++ /dev/null
@@ -1,203 +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.annotation.InterfaceStability;
-import org.apache.samza.operators.data.Message;
-import org.apache.samza.operators.internal.Operators;
-import org.apache.samza.operators.internal.Trigger;
-import org.apache.samza.operators.internal.WindowFn;
-import org.apache.samza.operators.internal.WindowOutput;
-import org.apache.samza.storage.kv.Entry;
-
-import java.util.Collection;
-import java.util.function.BiFunction;
-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.
- *
- */
-@InterfaceStability.Unstable
-public final class Windows {
-
- /**
- * private constructor to prevent instantiation
- */
- private Windows() {}
-
- /**
- * This class defines a session window function class
- *
- * @param <M> the type of input {@link Message}
- * @param <WK> the type of session key in the session window
- * @param <WV> the type of output value in each session window
- */
- static class SessionWindow<M extends Message, WK, WV> implements Window<M, WK, WV, WindowOutput<WK, WV>> {
-
- /**
- * 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 Message}
- * @param aggregator function to calculate the output value based on the input {@link Message} and current output value
- */
- private SessionWindow(Function<M, WK> sessionKeyFunction, BiFunction<M, WV, WV> aggregator) {
- this.wndKeyFunction = sessionKeyFunction;
- this.aggregator = aggregator;
- }
-
- /**
- * function to calculate the window key from input message
- */
- private final Function<M, WK> wndKeyFunction;
-
- /**
- * function to calculate the output value from the input message and the current output value
- */
- private final BiFunction<M, WV, WV> aggregator;
-
- /**
- * trigger condition that determines when to send out the output value in a {@link WindowOutput} message
- */
- private Trigger<M, WindowState<WV>> trigger = null;
-
- //TODO: need to create a set of {@link StoreFunctions} that is default to input {@link Message} type for {@link Window}
- private Operators.StoreFunctions<M, WK, WindowState<WV>> 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<M, WK, WV, WindowOutput<WK, WV>> setTriggers(TriggerBuilder<M, WV> wndTrigger) {
- this.trigger = wndTrigger.build();
- return this;
- }
-
- private BiFunction<M, Entry<WK, WindowState<WV>>, WindowOutput<WK, WV>> getTransformFunc() {
- // TODO: actual implementation of the main session window logic, based on the wndKeyFunction, aggregator, and triggers;
- return null;
- }
-
- private WindowFn<M, WK, WindowState<WV>, WindowOutput<WK, WV>> getInternalWindowFn() {
- return new WindowFn<M, WK, WindowState<WV>, WindowOutput<WK, WV>>() {
-
- @Override public BiFunction<M, Entry<WK, WindowState<WV>>, WindowOutput<WK, WV>> getTransformFunc() {
- return SessionWindow.this.getTransformFunc();
- }
-
- @Override public Operators.StoreFunctions<M, WK, WindowState<WV>> getStoreFuncs() {
- return SessionWindow.this.storeFunctions;
- }
-
- @Override public Trigger<M, WindowState<WV>> getTrigger() {
- return SessionWindow.this.trigger;
- }
- };
- }
- }
-
- static <M extends Message, WK, WV, WS extends WindowState<WV>, WM extends WindowOutput<WK, WV>> WindowFn<M, WK, WS, WM> getInternalWindowFn(
- Window<M, WK, WV, WM> window) {
- if (window instanceof SessionWindow) {
- SessionWindow<M, WK, WV> sessionWindow = (SessionWindow<M, WK, WV>) window;
- return (WindowFn<M, WK, WS, WM>) sessionWindow.getInternalWindowFn();
- }
- throw new IllegalArgumentException("Input window type not supported.");
- }
-
- /**
- * Public static API methods start here
- *
- */
-
- /**
- * The public programming interface class for window function
- *
- * @param <M> the type of input {@link Message}
- * @param <WK> the type of key to the {@link Window}
- * @param <WV> the type of output value in the {@link WindowOutput}
- * @param <WM> the type of message in the window output stream
- */
- @InterfaceStability.Unstable
- public interface Window<M extends Message, WK, WV, WM extends WindowOutput<WK, WV>> {
-
- /**
- * 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<M, WK, WV, WM> setTriggers(TriggerBuilder<M, WV> wndTrigger);
- }
-
- /**
- * Static API method to create a {@link SessionWindow} in which the output value is simply the collection of input messages
- *
- * @param sessionKeyFunction function to calculate session window key
- * @param <M> type of input {@link Message}
- * @param <WK> type of the session window key
- * @return the {@link Window} function for the session
- */
- public static <M extends Message, WK> Window<M, WK, Collection<M>, WindowOutput<WK, Collection<M>>> intoSessions(Function<M, WK> 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 messages
- *
- * @param sessionKeyFunction function to calculate session window key
- * @param sessionInfoExtractor function to retrieve session info of type {@code SI} from the input message of type {@code M}
- * @param <M> type of the input {@link Message}
- * @param <WK> type of the session window key
- * @param <SI> type of the session information retrieved from each input message of type {@code M}
- * @return the {@link Window} function for the session
- */
- public static <M extends Message, WK, SI> Window<M, WK, Collection<SI>, WindowOutput<WK, Collection<SI>>> intoSessions(Function<M, WK> sessionKeyFunction,
- Function<M, SI> 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 messages
- *
- * @param sessionKeyFunction function to calculate session window key
- * @param <M> type of the input {@link Message}
- * @param <WK> type of the session window key
- * @return the {@link Window} function for the session
- */
- public static <M extends Message, WK> Window<M, WK, Integer, WindowOutput<WK, Integer>> intoSessionCounter(Function<M, WK> sessionKeyFunction) {
- return new SessionWindow<>(sessionKeyFunction, (m, c) -> c + 1);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/data/IncomingSystemMessage.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/data/IncomingSystemMessage.java b/samza-api/src/main/java/org/apache/samza/operators/data/IncomingSystemMessage.java
deleted file mode 100644
index 3c9874d..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/data/IncomingSystemMessage.java
+++ /dev/null
@@ -1,76 +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.data;
-
-import org.apache.samza.system.IncomingMessageEnvelope;
-import org.apache.samza.system.SystemStreamPartition;
-
-
-/**
- * This class implements a {@link Message} that encapsulates an {@link IncomingMessageEnvelope} from the system
- *
- */
-public class IncomingSystemMessage implements Message<Object, Object>, InputSystemMessage<Offset> {
- /**
- * Incoming message envelope
- */
- private final IncomingMessageEnvelope imsg;
-
- /**
- * The receive time of this incoming message
- */
- private final long recvTimeNano;
-
- /**
- * Ctor to create a {@code IncomingSystemMessage} from {@link IncomingMessageEnvelope}
- *
- * @param imsg The incoming system message
- */
- public IncomingSystemMessage(IncomingMessageEnvelope imsg) {
- this.imsg = imsg;
- this.recvTimeNano = System.nanoTime();
- }
-
- @Override
- public Object getMessage() {
- return this.imsg.getMessage();
- }
-
- @Override
- public Object getKey() {
- return this.imsg.getKey();
- }
-
- @Override
- public long getTimestamp() {
- return this.recvTimeNano;
- }
-
- @Override
- public Offset getOffset() {
- // TODO: need to add offset factory to generate different types of offset. This is just a placeholder,
- // assuming incoming message carries long value as offset (i.e. Kafka case)
- return new LongOffset(this.imsg.getOffset());
- }
-
- @Override
- public SystemStreamPartition getSystemStreamPartition() {
- return imsg.getSystemStreamPartition();
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/data/IncomingSystemMessageEnvelope.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/data/IncomingSystemMessageEnvelope.java b/samza-api/src/main/java/org/apache/samza/operators/data/IncomingSystemMessageEnvelope.java
new file mode 100644
index 0000000..a65809c
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/data/IncomingSystemMessageEnvelope.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.data;
+
+import org.apache.samza.system.IncomingMessageEnvelope;
+import org.apache.samza.system.SystemStreamPartition;
+
+
+/**
+ * A {@link MessageEnvelope} that provides additional information about its input {@link SystemStreamPartition}
+ * and its {@link Offset} within the {@link SystemStreamPartition}.
+ * <p>
+ * Note: the {@link Offset} is only unique and comparable within its {@link SystemStreamPartition}.
+ */
+public class IncomingSystemMessageEnvelope implements MessageEnvelope<Object, Object> {
+
+ private final IncomingMessageEnvelope ime;
+
+ /**
+ * Creates an {@code IncomingSystemMessageEnvelope} from the {@link IncomingMessageEnvelope}.
+ *
+ * @param ime the {@link IncomingMessageEnvelope} from the input system.
+ */
+ public IncomingSystemMessageEnvelope(IncomingMessageEnvelope ime) {
+ this.ime = ime;
+ }
+
+ @Override
+ public Object getKey() {
+ return this.ime.getKey();
+ }
+
+ @Override
+ public Object getMessage() {
+ return this.ime.getMessage();
+ }
+
+ public Offset getOffset() {
+ // TODO: need to add offset factory to generate different types of offset. This is just a placeholder,
+ // assuming incoming message envelope carries long value as offset (i.e. Kafka case)
+ return new LongOffset(this.ime.getOffset());
+ }
+
+ public SystemStreamPartition getSystemStreamPartition() {
+ return this.ime.getSystemStreamPartition();
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/data/InputSystemMessage.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/data/InputSystemMessage.java b/samza-api/src/main/java/org/apache/samza/operators/data/InputSystemMessage.java
deleted file mode 100644
index 5c23e74..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/data/InputSystemMessage.java
+++ /dev/null
@@ -1,45 +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.data;
-
-import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.system.SystemStreamPartition;
-
-
-/**
- * This interface defines additional methods a message from an system input should implement, including the methods to
- * get {@link SystemStreamPartition} and the {@link Offset} of the input system message.
- */
-@InterfaceStability.Unstable
-public interface InputSystemMessage<O extends Offset> {
-
- /**
- * Get the input message's {@link SystemStreamPartition}
- *
- * @return the {@link SystemStreamPartition} this message is coming from
- */
- SystemStreamPartition getSystemStreamPartition();
-
- /**
- * Get the offset of the message in the input stream. This should be used to uniquely identify a message in an input stream.
- *
- * @return The offset of the message in the input stream.
- */
- O getOffset();
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/data/Message.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/data/Message.java b/samza-api/src/main/java/org/apache/samza/operators/data/Message.java
deleted file mode 100644
index 8441682..0000000
--- a/samza-api/src/main/java/org/apache/samza/operators/data/Message.java
+++ /dev/null
@@ -1,64 +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.data;
-
-import org.apache.samza.annotation.InterfaceStability;
-
-
-/**
- * This class defines the generic interface of {@link Message}, which is a entry in the input/output stream.
- *
- * <p>The {@link Message} models the basic operatible unit in streaming SQL processes in Samza.
- *
- */
-@InterfaceStability.Unstable
-public interface Message<K, M> {
-
- /**
- * Access method to get the corresponding message body in {@link Message}
- *
- * @return Message object in this {@link Message}
- */
- M getMessage();
-
- /**
- * Method to indicate whether this {@link Message} indicates deletion of a message w/ the message key
- *
- * @return A boolean value indicates whether the current message is a delete or insert message
- */
- default boolean isDelete() {
- return false;
- };
-
- /**
- * Access method to the key of the message
- *
- * @return The key of the message
- */
- K getKey();
-
- /**
- * Get the message creation timestamp of the message.
- *
- * @return The message's timestamp in nano seconds.
- */
- long getTimestamp();
-
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/data/MessageEnvelope.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/data/MessageEnvelope.java b/samza-api/src/main/java/org/apache/samza/operators/data/MessageEnvelope.java
new file mode 100644
index 0000000..ad64231
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/data/MessageEnvelope.java
@@ -0,0 +1,54 @@
+/*
+ * 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.data;
+
+import org.apache.samza.annotation.InterfaceStability;
+
+
+/**
+ * An entry in the input/output {@link org.apache.samza.operators.MessageStream}s.
+ */
+@InterfaceStability.Unstable
+public interface MessageEnvelope<K, M> {
+
+ /**
+ * Get the key for this {@link MessageEnvelope}.
+ *
+ * @return the key for this {@link MessageEnvelope}
+ */
+ K getKey();
+
+ /**
+ * Get the message in this {@link MessageEnvelope}.
+ *
+ * @return the message in this {@link MessageEnvelope}
+ */
+ M getMessage();
+
+ /**
+ * Whether this {@link MessageEnvelope} indicates deletion of a previous message with this key.
+ *
+ * @return true if the current {@link MessageEnvelope} indicates deletion of a previous message with this key
+ */
+ default boolean isDelete() {
+ return false;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/functions/FilterFunction.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/FilterFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/FilterFunction.java
new file mode 100644
index 0000000..e611cd0
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/functions/FilterFunction.java
@@ -0,0 +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.functions;
+
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.operators.data.MessageEnvelope;
+
+
+/**
+ * A function that specifies whether a {@link MessageEnvelope} should be retained for further processing or filtered out.
+ * @param <M> type of the input {@link MessageEnvelope}
+ */
+@InterfaceStability.Unstable
+@FunctionalInterface
+public interface FilterFunction<M extends MessageEnvelope> {
+
+ /**
+ * Returns a boolean indicating whether this {@link MessageEnvelope} should be retained or filtered out.
+ * @param message the {@link MessageEnvelope} to be checked
+ * @return true if {@link MessageEnvelope} should be retained
+ */
+ boolean apply(M message);
+
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/functions/FlatMapFunction.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/FlatMapFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/FlatMapFunction.java
new file mode 100644
index 0000000..dbc0bd9
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/functions/FlatMapFunction.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.functions;
+
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.operators.data.MessageEnvelope;
+
+import java.util.Collection;
+
+
+/**
+ * A function that transforms a {@link MessageEnvelope} into a collection of 0 or more {@link MessageEnvelope}s,
+ * possibly of a different type.
+ * @param <M> type of the input {@link MessageEnvelope}
+ * @param <OM> type of the transformed {@link MessageEnvelope}s
+ */
+@InterfaceStability.Unstable
+@FunctionalInterface
+public interface FlatMapFunction<M extends MessageEnvelope, OM extends MessageEnvelope> {
+
+ /**
+ * Transforms the provided {@link MessageEnvelope} into a collection of 0 or more {@link MessageEnvelope}s.
+ * @param message the {@link MessageEnvelope} to be transformed
+ * @return a collection of 0 or more transformed {@link MessageEnvelope}s
+ */
+ Collection<OM> apply(M message);
+
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/functions/JoinFunction.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/JoinFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/JoinFunction.java
new file mode 100644
index 0000000..8cb1fce
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/functions/JoinFunction.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.functions;
+
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.operators.data.MessageEnvelope;
+
+
+/**
+ * A function that joins {@link MessageEnvelope}s from two {@link org.apache.samza.operators.MessageStream}s and produces
+ * a joined {@link MessageEnvelope}.
+ * @param <M> type of the input {@link MessageEnvelope}
+ * @param <JM> type of the {@link MessageEnvelope} to join with
+ * @param <RM> type of the joined {@link MessageEnvelope}
+ */
+@InterfaceStability.Unstable
+@FunctionalInterface
+public interface JoinFunction<M extends MessageEnvelope, JM extends MessageEnvelope, RM extends MessageEnvelope> {
+
+ /**
+ * Join the provided {@link MessageEnvelope}s and produces the joined {@link MessageEnvelope}.
+ * @param message the input {@link MessageEnvelope}
+ * @param otherMessage the {@link MessageEnvelope} to join with
+ * @return the joined {@link MessageEnvelope}
+ */
+ RM apply(M message, JM otherMessage);
+
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/functions/MapFunction.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/MapFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/MapFunction.java
new file mode 100644
index 0000000..04919a7
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/functions/MapFunction.java
@@ -0,0 +1,41 @@
+/*
+ * 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.functions;
+
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.operators.data.MessageEnvelope;
+
+
+/**
+ * A function that transforms a {@link MessageEnvelope} into another {@link MessageEnvelope}, possibly of a different type.
+ * @param <M> type of the input {@link MessageEnvelope}
+ * @param <OM> type of the transformed {@link MessageEnvelope}
+ */
+@InterfaceStability.Unstable
+@FunctionalInterface
+public interface MapFunction<M extends MessageEnvelope, OM extends MessageEnvelope> {
+
+ /**
+ * Transforms the provided {@link MessageEnvelope} into another {@link MessageEnvelope}
+ * @param message the {@link MessageEnvelope} to be transformed
+ * @return the transformed {@link MessageEnvelope}
+ */
+ OM apply(M message);
+
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-api/src/main/java/org/apache/samza/operators/functions/SinkFunction.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/SinkFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/SinkFunction.java
new file mode 100644
index 0000000..505da92
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/operators/functions/SinkFunction.java
@@ -0,0 +1,46 @@
+/*
+ * 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.functions;
+
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskCoordinator;
+
+
+/**
+ * A function that allows sending a {@link MessageEnvelope} to an output system.
+ * @param <M> type of the input {@link MessageEnvelope}
+ */
+@InterfaceStability.Unstable
+@FunctionalInterface
+public interface SinkFunction<M extends MessageEnvelope> {
+
+ /**
+ * Allows sending the provided {@link MessageEnvelope} to an output {@link org.apache.samza.system.SystemStream} using
+ * the provided {@link MessageCollector}. Also provides access to the {@link TaskCoordinator} to request commits
+ * or shut the container down.
+ *
+ * @param message the {@link MessageEnvelope} to be sent to an output {@link org.apache.samza.system.SystemStream}
+ * @param messageCollector the {@link MessageCollector} to use to send the {@link MessageEnvelope}
+ * @param taskCoordinator the {@link TaskCoordinator} to request commits or shutdown
+ */
+ void apply(M message, MessageCollector messageCollector, TaskCoordinator taskCoordinator);
+
+}
[3/6] samza git commit: SAMZA-1054: Refactor Operator APIs
Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
index efa6a96..c77914e 100644
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
@@ -18,88 +18,60 @@
*/
package org.apache.samza.operators.impl;
-import org.apache.samza.operators.data.Message;
import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.data.MessageEnvelope;
import org.apache.samza.task.MessageCollector;
import org.apache.samza.task.TaskContext;
import org.apache.samza.task.TaskCoordinator;
-import org.reactivestreams.Processor;
-import org.reactivestreams.Subscriber;
-import org.reactivestreams.Subscription;
import java.util.HashSet;
import java.util.Set;
/**
- * Abstract base class for all stream operator implementation classes.
+ * Abstract base class for all stream operator implementations.
*/
-public abstract class OperatorImpl<M extends Message, RM extends Message>
- implements Processor<ProcessorContext<M>, ProcessorContext<RM>> {
+public abstract class OperatorImpl<M extends MessageEnvelope, RM extends MessageEnvelope> {
- private final Set<Subscriber<? super ProcessorContext<RM>>> subscribers = new HashSet<>();
-
- @Override public void subscribe(Subscriber<? super ProcessorContext<RM>> s) {
- // Only add once
- subscribers.add(s);
- }
-
- @Override public void onSubscribe(Subscription s) {
-
- }
-
- @Override public void onNext(ProcessorContext<M> o) {
-
- onNext(o.getMessage(), o.getCollector(), o.getCoordinator());
- }
-
- @Override public void onError(Throwable t) {
-
- }
-
- @Override public void onComplete() {
+ private final Set<OperatorImpl<RM, ? extends MessageEnvelope>> nextOperators = new HashSet<>();
+ /**
+ * Register the next operator in the chain that this operator should propagate its output to.
+ * @param nextOperator the next operator in the chain.
+ */
+ void registerNextOperator(OperatorImpl<RM, ? extends MessageEnvelope> nextOperator) {
+ nextOperators.add(nextOperator);
}
/**
- * Default method for timer event
+ * Initialize the initial state for stateful operators.
*
- * @param nanoTime the system nano-second when the timer event is triggered
- * @param collector the {@link MessageCollector} in the context
- * @param coordinator the {@link TaskCoordinator} in the context
+ * @param source the source that this {@link OperatorImpl} operator is registered with
+ * @param context the task context to initialize the operator implementation
*/
- public void onTimer(long nanoTime, MessageCollector collector, TaskCoordinator coordinator) {
- this.subscribers.forEach(sub -> ((OperatorImpl) sub).onTimer(nanoTime, collector, coordinator));
- }
+ public void init(MessageStream<M> source, TaskContext context) {}
/**
- * Each sub-class will implement this method to actually perform the transformation and call the downstream subscribers.
+ * Perform the transformation required for this operator and call the downstream operators.
+ *
+ * Must call {@link #propagateResult} to propage the output to registered downstream operators correctly.
*
- * @param message the input {@link Message}
+ * @param message the input {@link MessageEnvelope}
* @param collector the {@link MessageCollector} in the context
* @param coordinator the {@link TaskCoordinator} in the context
*/
- protected abstract void onNext(M message, MessageCollector collector, TaskCoordinator coordinator);
+ public abstract void onNext(M message, MessageCollector collector, TaskCoordinator coordinator);
/**
- * Stateful operators will need to override this method to initialize the operators
+ * Helper method to propagate the output of this operator to all registered downstream operators.
*
- * @param source the source that this {@link OperatorImpl} object subscribe to
- * @param context the task context to initialize the operators within
- */
- protected void init(MessageStream<M> source, TaskContext context) {};
-
- /**
- * Method to trigger all downstream operators that consumes the output {@link MessageStream}
- * from this operator
+ * This method <b>must</b> be called from {@link #onNext} to propagate the operator output correctly.
*
- * @param omsg output {@link Message}
+ * @param outputMessage output {@link MessageEnvelope}
* @param collector the {@link MessageCollector} in the context
* @param coordinator the {@link TaskCoordinator} in the context
*/
- protected void nextProcessors(RM omsg, MessageCollector collector, TaskCoordinator coordinator) {
- subscribers.forEach(sub ->
- sub.onNext(new ProcessorContext<>(omsg, collector, coordinator))
- );
+ void propagateResult(RM outputMessage, MessageCollector collector, TaskCoordinator coordinator) {
+ nextOperators.forEach(sub -> sub.onNext(outputMessage, collector, coordinator));
}
}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpls.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..79446be
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpls.java
@@ -0,0 +1,125 @@
+/*
+ * 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.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.windows.WindowOutput;
+import org.apache.samza.operators.windows.WindowState;
+import org.apache.samza.task.TaskContext;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+
+/**
+ * Instantiates the DAG of {@link OperatorImpl}s corresponding to the {@link OperatorSpec}s for a
+ * {@link MessageStreamImpl}
+ */
+public class OperatorImpls {
+
+ /**
+ * Holds the mapping between the {@link OperatorSpec} and {@link OperatorImpl}s instances.
+ */
+ private static final Map<OperatorSpec, OperatorImpl> OPERATOR_IMPLS = new ConcurrentHashMap<>();
+
+ /**
+ * Traverses the DAG of {@link OperatorSpec}s starting from the provided {@link MessageStreamImpl},
+ * creates the corresponding DAG of {@link OperatorImpl}s, and returns its root {@link RootOperatorImpl} node.
+ *
+ * @param source the input {@link MessageStreamImpl} to instantiate {@link OperatorImpl}s for
+ * @param <M> the type of {@link MessageEnvelope}s in the {@code source} {@link MessageStream}
+ * @param context the {@link TaskContext} required to instantiate operators
+ * @return root node for the {@link OperatorImpl} DAG
+ */
+ public static <M extends MessageEnvelope> RootOperatorImpl createOperatorImpls(MessageStreamImpl<M> source, TaskContext context) {
+ // since the source message stream might have multiple operator specs registered on it,
+ // create a new root node as a single point of entry for the DAG.
+ RootOperatorImpl<M> 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
+ OperatorImpl<M, ? extends MessageEnvelope> operatorImpl =
+ createAndRegisterOperatorImpl(registeredOperator, source, context);
+ rootOperator.registerNextOperator(operatorImpl);
+ });
+ return rootOperator;
+ }
+
+ /**
+ * Helper method to recursively traverse the {@link OperatorSpec} DAG and instantiate and link the corresponding
+ * {@link OperatorImpl}s.
+ *
+ * @param operatorSpec the operatorSpec registered with the {@code source}
+ * @param source the source {@link MessageStreamImpl}
+ * @param context the context of the task
+ * @return the operator implementation for the operatorSpec
+ */
+ private static <M extends MessageEnvelope> OperatorImpl<M, ? extends MessageEnvelope> createAndRegisterOperatorImpl(OperatorSpec operatorSpec,
+ MessageStream source, TaskContext context) {
+ if (!OPERATOR_IMPLS.containsKey(operatorSpec)) {
+ OperatorImpl<M, ? extends MessageEnvelope> operatorImpl = createOperatorImpl(operatorSpec);
+ if (OPERATOR_IMPLS.putIfAbsent(operatorSpec, operatorImpl) == null) {
+ // this is the first time we've added the operatorImpl corresponding to the operatorSpec,
+ // so traverse and initialize and register the rest of the DAG.
+ MessageStream<? extends MessageEnvelope> outStream = operatorSpec.getOutputStream();
+ Collection<OperatorSpec> registeredSpecs = ((MessageStreamImpl) outStream).getRegisteredOperatorSpecs();
+ registeredSpecs.forEach(registeredSpec -> {
+ OperatorImpl subImpl = createAndRegisterOperatorImpl(registeredSpec, outStream, context);
+ operatorImpl.registerNextOperator(subImpl);
+ });
+ operatorImpl.init(source, context);
+ return operatorImpl;
+ }
+ }
+
+ // the implementation corresponding to operatorSpec has already been instantiated
+ // and registered, so we do not need to traverse the DAG further.
+ return OPERATOR_IMPLS.get(operatorSpec);
+ }
+
+ /**
+ * Creates a new {@link OperatorImpl} instance for the provided {@link OperatorSpec}.
+ *
+ * @param operatorSpec the immutable {@link OperatorSpec} definition.
+ * @param <M> type of input {@link MessageEnvelope}
+ * @return the {@link OperatorImpl} implementation instance
+ */
+ protected static <M extends MessageEnvelope> OperatorImpl<M, ? extends MessageEnvelope> createOperatorImpl(OperatorSpec operatorSpec) {
+ if (operatorSpec instanceof StreamOperatorSpec) {
+ return new StreamOperatorImpl<>((StreamOperatorSpec<M, ? extends MessageEnvelope>) operatorSpec);
+ } else if (operatorSpec instanceof SinkOperatorSpec) {
+ return new SinkOperatorImpl<>((SinkOperatorSpec<M>) operatorSpec);
+ } else if (operatorSpec instanceof WindowOperatorSpec) {
+ return new SessionWindowOperatorImpl<>((WindowOperatorSpec<M, ?, ? extends WindowState, ? extends WindowOutput>) operatorSpec);
+ } else if (operatorSpec instanceof PartialJoinOperatorSpec) {
+ return new PartialJoinOperatorImpl<>((PartialJoinOperatorSpec) operatorSpec);
+ }
+ throw new IllegalArgumentException(
+ String.format("Unsupported OperatorSpec: %s", operatorSpec.getClass().getName()));
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/impl/PartialJoinOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/PartialJoinOperatorImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/PartialJoinOperatorImpl.java
new file mode 100644
index 0000000..90569b4
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/PartialJoinOperatorImpl.java
@@ -0,0 +1,46 @@
+/*
+ * 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.PartialJoinOperatorSpec;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskCoordinator;
+
+
+/**
+ * Implementation of a {@link PartialJoinOperatorSpec}. This class implements function
+ * that only takes in one input stream among all inputs to the join and generate the join output.
+ *
+ * @param <M> type of {@link MessageEnvelope}s in the input stream
+ * @param <JM> type of {@link MessageEnvelope}s in the stream to join with
+ * @param <RM> type of {@link MessageEnvelope}s in the joined stream
+ */
+class PartialJoinOperatorImpl<M extends MessageEnvelope<K, ?>, K, JM extends MessageEnvelope<K, ?>, RM extends MessageEnvelope>
+ extends OperatorImpl<M, RM> {
+
+ PartialJoinOperatorImpl(PartialJoinOperatorSpec<M, K, JM, RM> joinOp) {
+ // TODO: implement PartialJoinOperatorImpl constructor
+ }
+
+ @Override
+ public void onNext(M message, MessageCollector collector, TaskCoordinator coordinator) {
+ // TODO: implement PartialJoinOperatorImpl processing logic
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/impl/ProcessorContext.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/ProcessorContext.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/ProcessorContext.java
deleted file mode 100644
index cc7ef2b..0000000
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/ProcessorContext.java
+++ /dev/null
@@ -1,53 +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.data.Message;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskCoordinator;
-
-
-/**
- * Wrapper class to be used by {@link OperatorImpl}
- *
- * @param <M> Type of input stream {@link Message}
- */
-public class ProcessorContext<M extends Message> {
- private final M message;
- private final MessageCollector collector;
- private final TaskCoordinator coordinator;
-
- ProcessorContext(M message, MessageCollector collector, TaskCoordinator coordinator) {
- this.message = message;
- this.collector = collector;
- this.coordinator = coordinator;
- }
-
- M getMessage() {
- return this.message;
- }
-
- MessageCollector getCollector() {
- return this.collector;
- }
-
- TaskCoordinator getCoordinator() {
- return this.coordinator;
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/impl/RootOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/RootOperatorImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/RootOperatorImpl.java
new file mode 100644
index 0000000..7132b86
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/RootOperatorImpl.java
@@ -0,0 +1,36 @@
+/*
+ * 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.task.MessageCollector;
+import org.apache.samza.task.TaskCoordinator;
+
+
+/**
+ * A no-op operator implementation that forwards incoming {@link MessageEnvelope}s to all of its subscribers.
+ * @param <M> type of incoming {@link MessageEnvelope}s
+ */
+final class RootOperatorImpl<M extends MessageEnvelope> extends OperatorImpl<M, M> {
+
+ @Override
+ public void onNext(M message, MessageCollector collector, TaskCoordinator coordinator) {
+ this.propagateResult(message, collector, coordinator);
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/impl/SessionWindowOperatorImpl.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..e8a635c
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/SessionWindowOperatorImpl.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.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.windows.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 <M> the type of input {@link MessageEnvelope}
+ * @param <RK> the type of window key
+ * @param <WS> the type of window state
+ * @param <RM> the type of aggregated value of the window
+ */
+class SessionWindowOperatorImpl<M extends MessageEnvelope, RK, WS extends WindowState, RM extends WindowOutput<RK, ?>>
+ extends OperatorImpl<M, RM> {
+
+ private final WindowOperatorSpec<M, RK, WS, RM> windowSpec;
+ private StateStoreImpl<M, RK, WS> stateStore = null;
+
+ SessionWindowOperatorImpl(WindowOperatorSpec<M, RK, WS, RM> windowSpec) {
+ this.windowSpec = windowSpec;
+ }
+
+ @Override
+ public void init(MessageStream<M> 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<RK, WS> 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
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.java
deleted file mode 100644
index b0f4f27..0000000
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/SimpleOperatorImpl.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.impl;
-
-import org.apache.samza.operators.data.Message;
-import org.apache.samza.operators.internal.Operators.StreamOperator;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskCoordinator;
-
-import java.util.Collection;
-import java.util.function.Function;
-
-
-/**
- * Base class for all implementation of operators
- *
- * @param <M> type of message in the input stream
- * @param <RM> type of message in the output stream
- */
-public class SimpleOperatorImpl<M extends Message, RM extends Message> extends OperatorImpl<M, RM> {
-
- private final Function<M, Collection<RM>> transformFn;
-
- SimpleOperatorImpl(StreamOperator<M, RM> op) {
- super();
- this.transformFn = op.getFunction();
- }
-
- @Override protected void onNext(M imsg, MessageCollector collector, TaskCoordinator coordinator) {
- // actually calling the transform function and then for each output, call nextProcessors()
- this.transformFn.apply(imsg).forEach(r -> this.nextProcessors(r, collector, coordinator));
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java
index a8a639e..abed03f 100644
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java
@@ -18,24 +18,26 @@
*/
package org.apache.samza.operators.impl;
-import org.apache.samza.operators.internal.Operators.SinkOperator;
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.data.Message;
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.functions.SinkFunction;
+import org.apache.samza.operators.spec.SinkOperatorSpec;
import org.apache.samza.task.MessageCollector;
import org.apache.samza.task.TaskCoordinator;
/**
- * Implementation for {@link SinkOperator}
+ * Implementation for {@link SinkOperatorSpec}
*/
-public class SinkOperatorImpl<M extends Message> extends OperatorImpl<M, Message> {
- private final MessageStream.VoidFunction3<M, MessageCollector, TaskCoordinator> sinkFunc;
+class SinkOperatorImpl<M extends MessageEnvelope> extends OperatorImpl<M, MessageEnvelope> {
- SinkOperatorImpl(SinkOperator<M> sinkOp) {
- this.sinkFunc = sinkOp.getFunction();
+ private final SinkFunction<M> sinkFn;
+
+ SinkOperatorImpl(SinkOperatorSpec<M> sinkOp) {
+ this.sinkFn = sinkOp.getSinkFn();
}
- @Override protected void onNext(M message, MessageCollector collector, TaskCoordinator coordinator) {
- this.sinkFunc.apply(message, collector, coordinator);
+ @Override
+ public void onNext(M message, MessageCollector collector, TaskCoordinator coordinator) {
+ this.sinkFn.apply(message, collector, coordinator);
}
}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/StateStoreImpl.java
deleted file mode 100644
index 7840b5b..0000000
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/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.impl;
-
-import org.apache.samza.operators.data.Message;
-import org.apache.samza.operators.internal.Operators.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<M extends Message, SK, SS> {
- private final String storeName;
- private final StoreFunctions<M, SK, SS> storeFunctions;
- private KeyValueStore<SK, SS> kvStore = null;
-
- public StateStoreImpl(StoreFunctions<M, SK, SS> store, String storeName) {
- this.storeFunctions = store;
- this.storeName = storeName;
- }
-
- public void init(TaskContext context) {
- this.kvStore = (KeyValueStore<SK, SS>) context.getStore(this.storeName);
- }
-
- public Entry<SK, SS> getState(M m) {
- SK key = this.storeFunctions.getStoreKeyFinder().apply(m);
- SS state = this.kvStore.get(key);
- return new Entry<>(key, state);
- }
-
- public Entry<SK, SS> updateState(M m, Entry<SK, SS> oldEntry) {
- SS newValue = this.storeFunctions.getStateUpdater().apply(m, oldEntry.getValue());
- this.kvStore.put(oldEntry.getKey(), newValue);
- return new Entry<>(oldEntry.getKey(), newValue);
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/impl/StreamOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/StreamOperatorImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/StreamOperatorImpl.java
new file mode 100644
index 0000000..3a5c56e
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/impl/StreamOperatorImpl.java
@@ -0,0 +1,47 @@
+/*
+ * 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.functions.FlatMapFunction;
+import org.apache.samza.operators.spec.StreamOperatorSpec;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskCoordinator;
+
+
+/**
+ * A StreamOperator that accepts a 1:n transform function and applies it to each incoming {@link MessageEnvelope}.
+ *
+ * @param <M> type of {@link MessageEnvelope} in the input stream
+ * @param <RM> type of {@link MessageEnvelope} in the output stream
+ */
+class StreamOperatorImpl<M extends MessageEnvelope, RM extends MessageEnvelope> extends OperatorImpl<M, RM> {
+
+ private final FlatMapFunction<M, RM> transformFn;
+
+ StreamOperatorImpl(StreamOperatorSpec<M, RM> streamOperatorSpec) {
+ this.transformFn = streamOperatorSpec.getTransformFn();
+ }
+
+ @Override
+ public void onNext(M message, MessageCollector collector, TaskCoordinator coordinator) {
+ // call the transform function and then for each output call propagateResult()
+ this.transformFn.apply(message).forEach(r -> this.propagateResult(r, collector, coordinator));
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/impl/join/PartialJoinOpImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/join/PartialJoinOpImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/join/PartialJoinOpImpl.java
deleted file mode 100644
index 4238d45..0000000
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/join/PartialJoinOpImpl.java
+++ /dev/null
@@ -1,44 +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.join;
-
-import org.apache.samza.operators.data.Message;
-import org.apache.samza.operators.internal.Operators.PartialJoinOperator;
-import org.apache.samza.operators.impl.OperatorImpl;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskCoordinator;
-
-
-/**
- * Implementation of a {@link PartialJoinOperator}. This class implements function
- * that only takes in one input stream among all inputs to the join and generate the join output.
- *
- * @param <M> Type of input stream {@link Message}
- * @param <RM> Type of join output stream {@link Message}
- */
-public class PartialJoinOpImpl<M extends Message<K, ?>, K, JM extends Message<K, ?>, RM extends Message> extends OperatorImpl<M, RM> {
-
- public PartialJoinOpImpl(PartialJoinOperator<M, K, JM, RM> joinOp) {
- // TODO: implement PartialJoinOpImpl constructor
- }
-
- @Override protected void onNext(M message, MessageCollector collector, TaskCoordinator coordinator) {
- // TODO: implement PartialJoinOpImpl processing logic
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
deleted file mode 100644
index 0d6141e..0000000
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/window/SessionWindowImpl.java
+++ /dev/null
@@ -1,65 +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.window;
-
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.WindowState;
-import org.apache.samza.operators.data.Message;
-import org.apache.samza.operators.internal.Operators.WindowOperator;
-import org.apache.samza.operators.internal.WindowOutput;
-import org.apache.samza.operators.impl.OperatorImpl;
-import org.apache.samza.operators.impl.StateStoreImpl;
-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 WindowOperator} for a session window.
- *
- * @param <M> the type of input {@link Message}
- * @param <RK> the type of window key
- * @param <WS> the type of window state
- * @param <RM> the type of aggregated value of the window
- */
-public class SessionWindowImpl<M extends Message, RK, WS extends WindowState, RM extends WindowOutput<RK, ?>> extends
- OperatorImpl<M, RM> {
- private final WindowOperator<M, RK, WS, RM> sessWnd;
- private StateStoreImpl<M, RK, WS> wndStore = null;
-
- public SessionWindowImpl(WindowOperator<M, RK, WS, RM> sessWnd) {
- this.sessWnd = sessWnd;
- }
-
- @Override protected void onNext(M message, MessageCollector collector, TaskCoordinator coordinator) {
- Entry<RK, WS> state = this.wndStore.getState(message);
- this.nextProcessors(this.sessWnd.getFunction().apply(message, state), collector, coordinator);
- this.wndStore.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
- }
-
- @Override protected void init(MessageStream<M> source, TaskContext context) {
- this.wndStore = new StateStoreImpl<>(this.sessWnd.getStoreFunctions(), sessWnd.getStoreName(source));
- this.wndStore.init(context);
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java
new file mode 100644
index 0000000..8b75cdc
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java
@@ -0,0 +1,37 @@
+/*
+ * 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;
+
+
+/**
+ * A stateless serializable stream operator specification that holds all the information required
+ * to transform the input {@link MessageStream} and produce the output {@link MessageStream}.
+ */
+public interface OperatorSpec<OM extends MessageEnvelope> {
+
+ /**
+ * Get the output stream containing transformed {@link MessageEnvelope} produced by this operator.
+ * @return the output stream containing transformed {@link MessageEnvelope} produced by this operator.
+ */
+ MessageStream<OM> getOutputStream();
+
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..f622b34
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java
@@ -0,0 +1,116 @@
+/*
+ * 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 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 java.util.ArrayList;
+import java.util.UUID;
+import java.util.function.BiFunction;
+
+
+/**
+ * Factory methods for creating {@link OperatorSpec} instances.
+ */
+public class OperatorSpecs {
+
+ private OperatorSpecs() {}
+
+ private static String getOperatorId() {
+ // TODO: need to change the IDs to be a consistent, durable IDs that can be recovered across container and job restarts
+ return UUID.randomUUID().toString();
+ }
+
+ /**
+ * Creates a {@link StreamOperatorSpec}.
+ *
+ * @param transformFn the transformation function
+ * @param <M> type of input {@link MessageEnvelope}
+ * @param <OM> type of output {@link MessageEnvelope}
+ * @return the {@link StreamOperatorSpec}
+ */
+ public static <M extends MessageEnvelope, OM extends MessageEnvelope> StreamOperatorSpec<M, OM> createStreamOperator(
+ FlatMapFunction<M, OM> transformFn) {
+ return new StreamOperatorSpec<>(transformFn);
+ }
+
+ /**
+ * Creates a {@link SinkOperatorSpec}.
+ *
+ * @param sinkFn the sink function
+ * @param <M> type of input {@link MessageEnvelope}
+ * @return the {@link SinkOperatorSpec}
+ */
+ public static <M extends MessageEnvelope> SinkOperatorSpec<M> createSinkOperator(SinkFunction<M> sinkFn) {
+ return new SinkOperatorSpec<>(sinkFn);
+ }
+
+ /**
+ * Creates a {@link WindowOperatorSpec}.
+ *
+ * @param windowFn the {@link WindowFn} function
+ * @param <M> type of input {@link MessageEnvelope}
+ * @param <WK> type of window key
+ * @param <WS> type of {@link WindowState}
+ * @param <WM> type of output {@link WindowOutput}
+ * @return the {@link WindowOperatorSpec}
+ */
+ public static <M extends MessageEnvelope, WK, WS extends WindowState, WM extends WindowOutput<WK, ?>> WindowOperatorSpec<M, WK, WS, WM> createWindowOperator(
+ WindowFn<M, WK, WS, WM> windowFn) {
+ return new WindowOperatorSpec<>(windowFn, OperatorSpecs.getOperatorId());
+ }
+
+ /**
+ * Creates a {@link PartialJoinOperatorSpec}.
+ *
+ * @param partialJoinFn the join function
+ * @param joinOutput the output {@link MessageStreamImpl}
+ * @param <M> type of input {@link MessageEnvelope}
+ * @param <K> type of join key
+ * @param <JM> the type of {@link MessageEnvelope} in the other join stream
+ * @param <OM> the type of {@link MessageEnvelope} in the join output
+ * @return the {@link PartialJoinOperatorSpec}
+ */
+ public static <M extends MessageEnvelope<K, ?>, K, JM extends MessageEnvelope<K, ?>, OM extends MessageEnvelope> PartialJoinOperatorSpec<M, K, JM, OM> createPartialJoinOperator(
+ BiFunction<M, JM, OM> partialJoinFn, MessageStreamImpl<OM> joinOutput) {
+ return new PartialJoinOperatorSpec<>(partialJoinFn, joinOutput, OperatorSpecs.getOperatorId());
+ }
+
+ /**
+ * Creates a {@link StreamOperatorSpec} with a merger function.
+ *
+ * @param mergeOutput the output {@link MessageStreamImpl} from the merger
+ * @param <M> the type of input {@link MessageEnvelope}
+ * @return the {@link StreamOperatorSpec} for the merge
+ */
+ public static <M extends MessageEnvelope> StreamOperatorSpec<M, M> createMergeOperator(MessageStreamImpl<M> mergeOutput) {
+ return new StreamOperatorSpec<M, M>(t ->
+ new ArrayList<M>() { {
+ this.add(t);
+ } },
+ mergeOutput);
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/spec/PartialJoinOperatorSpec.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..f74f35d
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/spec/PartialJoinOperatorSpec.java
@@ -0,0 +1,104 @@
+/*
+ * 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 org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.windows.StoreFunctions;
+
+import java.util.function.BiFunction;
+
+
+/**
+ * Spec for the partial join operator that takes {@link MessageEnvelope}s from one input stream, joins with buffered
+ * {@link MessageEnvelope}s from another stream, and produces join results to an output {@link MessageStreamImpl}.
+ *
+ * @param <M> the type of input {@link MessageEnvelope}
+ * @param <K> the type of join key
+ * @param <JM> the type of {@link MessageEnvelope} in the other join stream
+ * @param <RM> the type of {@link MessageEnvelope} in the join output stream
+ */
+public class PartialJoinOperatorSpec<M extends MessageEnvelope<K, ?>, K, JM extends MessageEnvelope<K, ?>, RM extends MessageEnvelope>
+ implements OperatorSpec<RM> {
+
+ private final MessageStreamImpl<RM> joinOutput;
+
+ /**
+ * The transformation function of {@link PartialJoinOperatorSpec} that takes an input {@link MessageEnvelope} of
+ * type {@code M}, joins with a stream of buffered {@link MessageEnvelope}s of type {@code JM} from another stream,
+ * and generates a joined result {@link MessageEnvelope} of type {@code RM}.
+ */
+ private final BiFunction<M, JM, RM> transformFn;
+
+ /**
+ * The {@link MessageEnvelope} store functions that read the buffered {@link MessageEnvelope}s from the other
+ * stream in the join.
+ */
+ private final StoreFunctions<JM, K, JM> joinStoreFns;
+
+ /**
+ * The {@link MessageEnvelope} store functions that save the buffered {@link MessageEnvelope} of this
+ * {@link MessageStreamImpl} in the join.
+ */
+ private final StoreFunctions<M, K, M> selfStoreFns;
+
+ /**
+ * The unique ID for this operator.
+ */
+ private final String operatorId;
+
+ /**
+ * Default constructor for a {@link PartialJoinOperatorSpec}.
+ *
+ * @param partialJoinFn partial join function that take type {@code M} of input {@link MessageEnvelope} and join
+ * w/ type {@code JM} of buffered {@link MessageEnvelope} from another stream
+ * @param joinOutput the output {@link MessageStreamImpl} of the join results
+ */
+ PartialJoinOperatorSpec(BiFunction<M, JM, RM> partialJoinFn, MessageStreamImpl<RM> 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;
+ }
+
+ @Override
+ public String toString() {
+ return this.operatorId;
+ }
+
+ @Override
+ public MessageStreamImpl<RM> getOutputStream() {
+ return this.joinOutput;
+ }
+
+ public StoreFunctions<JM, K, JM> getJoinStoreFns() {
+ return this.joinStoreFns;
+ }
+
+ public StoreFunctions<M, K, M> getSelfStoreFns() {
+ return this.selfStoreFns;
+ }
+
+ public BiFunction<M, JM, RM> getTransformFn() {
+ return this.transformFn;
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/spec/SinkOperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/SinkOperatorSpec.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/SinkOperatorSpec.java
new file mode 100644
index 0000000..4348bc0
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/spec/SinkOperatorSpec.java
@@ -0,0 +1,62 @@
+/*
+ * 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 org.apache.samza.operators.functions.SinkFunction;
+import org.apache.samza.operators.MessageStreamImpl;
+
+
+/**
+ * The spec for a sink operator that accepts user-defined logic to output a {@link MessageStreamImpl} to an external
+ * system. This is a terminal operator and does allows further operator chaining.
+ *
+ * @param <M> the type of input {@link MessageEnvelope}
+ */
+public class SinkOperatorSpec<M extends MessageEnvelope> implements OperatorSpec {
+
+ /**
+ * The user-defined sink function
+ */
+ private final SinkFunction<M> sinkFn;
+
+ /**
+ * Default constructor for a {@link SinkOperatorSpec}.
+ *
+ * @param sinkFn a user defined {@link SinkFunction} that will be called with the output {@link MessageEnvelope},
+ * the output {@link org.apache.samza.task.MessageCollector} and the
+ * {@link org.apache.samza.task.TaskCoordinator}.
+ */
+ SinkOperatorSpec(SinkFunction<M> sinkFn) {
+ this.sinkFn = sinkFn;
+ }
+
+ /**
+ * This is a terminal operator and doesn't allow further operator chaining.
+ * @return null
+ */
+ @Override
+ public MessageStreamImpl getOutputStream() {
+ return null;
+ }
+
+ public SinkFunction<M> getSinkFn() {
+ return this.sinkFn;
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/spec/StreamOperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/StreamOperatorSpec.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/StreamOperatorSpec.java
new file mode 100644
index 0000000..ed18da4
--- /dev/null
+++ b/samza-operator/src/main/java/org/apache/samza/operators/spec/StreamOperatorSpec.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 org.apache.samza.operators.functions.FlatMapFunction;
+import org.apache.samza.operators.MessageStreamImpl;
+
+
+/**
+ * The spec for a linear stream operator that outputs 0 or more {@link MessageEnvelope}s for each input {@link MessageEnvelope}.
+ *
+ * @param <M> the type of input {@link MessageEnvelope}
+ * @param <OM> the type of output {@link MessageEnvelope}
+ */
+public class StreamOperatorSpec<M extends MessageEnvelope, OM extends MessageEnvelope> implements OperatorSpec<OM> {
+
+ private final MessageStreamImpl<OM> outputStream;
+
+ private final FlatMapFunction<M, OM> transformFn;
+
+ /**
+ * Default constructor for a {@link StreamOperatorSpec}.
+ *
+ * @param transformFn the transformation function that transforms each input {@link MessageEnvelope} into a collection
+ * of output {@link MessageEnvelope}s
+ */
+ StreamOperatorSpec(FlatMapFunction<M, OM> transformFn) {
+ this(transformFn, new MessageStreamImpl<>());
+ }
+
+ /**
+ * Constructor for a {@link StreamOperatorSpec} that accepts an output {@link MessageStreamImpl}.
+ *
+ * @param transformFn the transformation function
+ * @param outputStream the output {@link MessageStreamImpl}
+ */
+ StreamOperatorSpec(FlatMapFunction<M, OM> transformFn, MessageStreamImpl<OM> outputStream) {
+ this.outputStream = outputStream;
+ this.transformFn = transformFn;
+ }
+
+ @Override
+ public MessageStreamImpl<OM> getOutputStream() {
+ return this.outputStream;
+ }
+
+ public FlatMapFunction<M, OM> getTransformFn() {
+ return this.transformFn;
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java
----------------------------------------------------------------------
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 0000000..2f5b1e7
--- /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 <M> the type of input {@link MessageEnvelope}
+ * @param <WK> the type of key in the output {@link MessageEnvelope} from the {@link WindowOperatorSpec} function
+ * @param <WS> the type of window state in the {@link WindowOperatorSpec} function
+ * @param <WM> the type of window output {@link MessageEnvelope}
+ */
+public class WindowOperatorSpec<M extends MessageEnvelope, WK, WS extends WindowState, WM extends WindowOutput<WK, ?>> implements
+ OperatorSpec<WM> {
+
+ /**
+ * The output {@link MessageStream}.
+ */
+ private final MessageStreamImpl<WM> 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<M, Entry<WK, WS>, WM> transformFn;
+
+ /**
+ * The state store functions for the {@link WindowOperatorSpec}.
+ */
+ private final StoreFunctions<M, WK, WS> storeFns;
+
+ /**
+ * The window trigger.
+ */
+ private final Trigger<M, WS> 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<M, WK, WS, WM> 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<WM> getOutputStream() {
+ return this.outputStream;
+ }
+
+ public StoreFunctions<M, WK, WS> getStoreFns() {
+ return this.storeFns;
+ }
+
+ public BiFunction<M, Entry<WK, WS>, WM> getTransformFn() {
+ return this.transformFn;
+ }
+
+ public Trigger<M, WS> 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<M> 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());
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/main/java/org/apache/samza/task/StreamOperatorAdaptorTask.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/task/StreamOperatorAdaptorTask.java b/samza-operator/src/main/java/org/apache/samza/task/StreamOperatorAdaptorTask.java
deleted file mode 100644
index c2f780d..0000000
--- a/samza-operator/src/main/java/org/apache/samza/task/StreamOperatorAdaptorTask.java
+++ /dev/null
@@ -1,85 +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.task;
-
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.MessageStreams;
-import org.apache.samza.operators.MessageStreams.SystemMessageStream;
-import org.apache.samza.operators.data.IncomingSystemMessage;
-import org.apache.samza.operators.impl.ChainedOperators;
-import org.apache.samza.operators.task.StreamOperatorTask;
-import org.apache.samza.system.IncomingMessageEnvelope;
-import org.apache.samza.system.SystemStreamPartition;
-
-import java.util.HashMap;
-import java.util.Map;
-
-
-/**
- * An adaptor task class that invoke the user-implemented (@link StreamOperatorTask}s via {@link org.apache.samza.operators.MessageStream} programming APIs
- *
- */
-public final class StreamOperatorAdaptorTask implements StreamTask, InitableTask, WindowableTask {
- /**
- * A map with entries mapping {@link SystemStreamPartition} to {@link org.apache.samza.operators.impl.ChainedOperators} that takes the {@link SystemStreamPartition}
- * as the input stream
- */
- private final Map<SystemStreamPartition, ChainedOperators> operatorChains = new HashMap<>();
-
- /**
- * Wrapped {@link StreamOperatorTask} class
- */
- private final StreamOperatorTask userTask;
-
- /**
- * Constructor that wraps the user-defined {@link StreamOperatorTask}
- *
- * @param userTask the user-defined {@link StreamOperatorTask}
- */
- public StreamOperatorAdaptorTask(StreamOperatorTask userTask) {
- this.userTask = userTask;
- }
-
- @Override
- public final void init(Config config, TaskContext context) throws Exception {
- if (this.userTask instanceof InitableTask) {
- ((InitableTask) this.userTask).init(config, context);
- }
- Map<SystemStreamPartition, SystemMessageStream> sources = new HashMap<>();
- context.getSystemStreamPartitions().forEach(ssp -> {
- SystemMessageStream ds = MessageStreams.input(ssp);
- sources.put(ssp, ds);
- });
- this.userTask.initOperators(sources.values());
- sources.forEach((ssp, ds) -> operatorChains.put(ssp, ChainedOperators.create(ds, context)));
- }
-
- @Override
- public final void process(IncomingMessageEnvelope ime, MessageCollector collector, TaskCoordinator coordinator) {
- this.operatorChains.get(ime.getSystemStreamPartition()).onNext(new IncomingSystemMessage(ime), collector, coordinator);
- }
-
- @Override
- public final void window(MessageCollector collector, TaskCoordinator coordinator) throws Exception {
- this.operatorChains.forEach((ssp, chain) -> chain.onTimer(collector, coordinator));
- if (this.userTask instanceof WindowableTask) {
- ((WindowableTask) this.userTask).window(collector, coordinator);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/BroadcastTask.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..e45d068
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/BroadcastTask.java
@@ -0,0 +1,101 @@
+/*
+ * 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.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.Windows;
+import org.apache.samza.system.SystemStreamPartition;
+
+import java.util.Collection;
+import java.util.Map;
+
+
+/**
+ * Example implementation of split stream tasks
+ *
+ */
+public class BroadcastTask implements StreamOperatorTask {
+ class MessageType {
+ String field1;
+ String field2;
+ String field3;
+ String field4;
+ String parKey;
+ private long timestamp;
+
+ public long getTimestamp() {
+ return this.timestamp;
+ }
+ }
+
+ class JsonMessageEnvelope extends JsonIncomingSystemMessageEnvelope<MessageType> {
+ JsonMessageEnvelope(String key, MessageType data, Offset offset, SystemStreamPartition partition) {
+ super(key, data, offset, partition);
+ }
+ }
+
+ @Override
+ public void transform(Map<SystemStreamPartition, MessageStream<IncomingSystemMessageEnvelope>> messageStreams) {
+ messageStreams.values().forEach(entry -> {
+ MessageStream<JsonMessageEnvelope> inputStream = entry.map(this::getInputMessage);
+
+ inputStream.filter(this::myFilter1).
+ window(Windows.<JsonMessageEnvelope, String>intoSessionCounter(
+ m -> String.format("%s-%s", m.getMessage().field1, m.getMessage().field2)).
+ setTriggers(TriggerBuilder.<JsonMessageEnvelope, Integer>earlyTriggerWhenExceedWndLen(100).
+ addLateTriggerOnSizeLimit(10).
+ addTimeoutSinceLastMessage(30000)));
+
+ inputStream.filter(this::myFilter2).
+ window(Windows.<JsonMessageEnvelope, String>intoSessions(
+ m -> String.format("%s-%s", m.getMessage().field3, m.getMessage().field4)).
+ setTriggers(TriggerBuilder.<JsonMessageEnvelope, Collection<JsonMessageEnvelope>>earlyTriggerWhenExceedWndLen(100).
+ addTimeoutSinceLastMessage(30000)));
+
+ inputStream.filter(this::myFilter3).
+ window(Windows.<JsonMessageEnvelope, String, MessageType>intoSessions(
+ m -> String.format("%s-%s", m.getMessage().field3, m.getMessage().field4), m -> m.getMessage()).
+ setTriggers(TriggerBuilder.<JsonMessageEnvelope, Collection<MessageType>>earlyTriggerOnEventTime(m -> m.getMessage().getTimestamp(), 30000).
+ addTimeoutSinceFirstMessage(60000)));
+ });
+ }
+
+ JsonMessageEnvelope getInputMessage(IncomingSystemMessageEnvelope m1) {
+ return (JsonMessageEnvelope) m1.getMessage();
+ }
+
+ boolean myFilter1(JsonMessageEnvelope m1) {
+ // Do user defined processing here
+ return m1.getMessage().parKey.equals("key1");
+ }
+
+ boolean myFilter2(JsonMessageEnvelope m1) {
+ // Do user defined processing here
+ return m1.getMessage().parKey.equals("key2");
+ }
+
+ boolean myFilter3(JsonMessageEnvelope m1) {
+ return m1.getMessage().parKey.equals("key3");
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/JoinTask.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/JoinTask.java b/samza-operator/src/test/java/org/apache/samza/operators/JoinTask.java
new file mode 100644
index 0000000..1b10609
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/JoinTask.java
@@ -0,0 +1,77 @@
+/*
+ * 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.IncomingSystemMessageEnvelope;
+import org.apache.samza.operators.data.Offset;
+import org.apache.samza.operators.data.JsonIncomingSystemMessageEnvelope;
+import org.apache.samza.system.SystemStreamPartition;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+
+/**
+ * Example implementation of unique key-based stream-stream join tasks
+ *
+ */
+public class JoinTask implements StreamOperatorTask {
+ class MessageType {
+ String joinKey;
+ List<String> joinFields = new ArrayList<>();
+ }
+
+ class JsonMessageEnvelope extends JsonIncomingSystemMessageEnvelope<MessageType> {
+ JsonMessageEnvelope(String key, MessageType data, Offset offset, SystemStreamPartition partition) {
+ super(key, data, offset, partition);
+ }
+ }
+
+ MessageStream<JsonMessageEnvelope> joinOutput = null;
+
+ @Override
+ public void transform(Map<SystemStreamPartition, MessageStream<IncomingSystemMessageEnvelope>> messageStreams) {
+ messageStreams.values().forEach(messageStream -> {
+ MessageStream<JsonMessageEnvelope> newSource = messageStream.map(this::getInputMessage);
+ if (joinOutput == null) {
+ joinOutput = newSource;
+ } else {
+ joinOutput = joinOutput.join(newSource, (m1, m2) -> this.myJoinResult(m1, m2));
+ }
+ });
+ }
+
+ private JsonMessageEnvelope getInputMessage(IncomingSystemMessageEnvelope ism) {
+ return new JsonMessageEnvelope(
+ ((MessageType) ism.getMessage()).joinKey,
+ (MessageType) ism.getMessage(),
+ ism.getOffset(),
+ ism.getSystemStreamPartition());
+ }
+
+ JsonMessageEnvelope myJoinResult(JsonMessageEnvelope m1, JsonMessageEnvelope m2) {
+ MessageType newJoinMsg = new MessageType();
+ newJoinMsg.joinKey = m1.getKey();
+ newJoinMsg.joinFields.addAll(m1.getMessage().joinFields);
+ newJoinMsg.joinFields.addAll(m2.getMessage().joinFields);
+ return new JsonMessageEnvelope(m1.getMessage().joinKey, newJoinMsg, null, null);
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/TestFluentStreamAdaptorTask.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/TestFluentStreamAdaptorTask.java b/samza-operator/src/test/java/org/apache/samza/operators/TestFluentStreamAdaptorTask.java
new file mode 100644
index 0000000..61bb32a
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/TestFluentStreamAdaptorTask.java
@@ -0,0 +1,85 @@
+/*
+ * 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.Partition;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.impl.OperatorImpl;
+import org.apache.samza.system.SystemStreamPartition;
+import org.apache.samza.task.TaskContext;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.lang.reflect.Field;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+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 TestFluentStreamAdaptorTask {
+ Field userTaskField = null;
+ Field operatorChainsField = null;
+
+ @Before
+ public void prep() throws NoSuchFieldException {
+ userTaskField = StreamOperatorAdaptorTask.class.getDeclaredField("userTask");
+ operatorChainsField = StreamOperatorAdaptorTask.class.getDeclaredField("operatorChains");
+ userTaskField.setAccessible(true);
+ operatorChainsField.setAccessible(true);
+ }
+
+ @Test
+ public void testConstructor() throws IllegalAccessException {
+ StreamOperatorTask userTask = mock(StreamOperatorTask.class);
+ StreamOperatorAdaptorTask adaptorTask = new StreamOperatorAdaptorTask(userTask);
+ StreamOperatorTask taskMemberVar = (StreamOperatorTask) userTaskField.get(adaptorTask);
+ Map<SystemStreamPartition, OperatorImpl> chainsMap = (Map<SystemStreamPartition, OperatorImpl>) operatorChainsField.get(adaptorTask);
+ assertEquals(taskMemberVar, userTask);
+ assertTrue(chainsMap.isEmpty());
+ }
+
+ @Test
+ public void testInit() throws Exception {
+ StreamOperatorTask userTask = mock(StreamOperatorTask.class);
+ StreamOperatorAdaptorTask adaptorTask = new StreamOperatorAdaptorTask(userTask);
+ Config mockConfig = mock(Config.class);
+ TaskContext mockContext = mock(TaskContext.class);
+ Set<SystemStreamPartition> testInputs = new HashSet() { {
+ this.add(new SystemStreamPartition("test-sys", "test-strm", new Partition(0)));
+ this.add(new SystemStreamPartition("test-sys", "test-strm", new Partition(1)));
+ } };
+ when(mockContext.getSystemStreamPartitions()).thenReturn(testInputs);
+ adaptorTask.init(mockConfig, mockContext);
+ verify(userTask, times(1)).transform(Mockito.anyMap());
+ Map<SystemStreamPartition, OperatorImpl> chainsMap = (Map<SystemStreamPartition, OperatorImpl>) operatorChainsField.get(adaptorTask);
+ assertTrue(chainsMap.size() == 2);
+ assertTrue(chainsMap.containsKey(testInputs.toArray()[0]));
+ assertTrue(chainsMap.containsKey(testInputs.toArray()[1]));
+ }
+
+ // TODO: window and process methods to be added after implementation of ChainedOperators.create()
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/00543804/samza-operator/src/test/java/org/apache/samza/operators/TestFluentStreamTasks.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/TestFluentStreamTasks.java b/samza-operator/src/test/java/org/apache/samza/operators/TestFluentStreamTasks.java
new file mode 100644
index 0000000..d804bf8
--- /dev/null
+++ b/samza-operator/src/test/java/org/apache/samza/operators/TestFluentStreamTasks.java
@@ -0,0 +1,112 @@
+/*
+ * 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.Partition;
+import org.apache.samza.config.Config;
+
+import org.apache.samza.operators.impl.OperatorImpl;
+import org.apache.samza.system.SystemStreamPartition;
+import org.apache.samza.task.TaskContext;
+import org.junit.Test;
+
+import java.lang.reflect.Field;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+
+/**
+ * Unit test for {@link StreamOperatorTask}
+ */
+public class TestFluentStreamTasks {
+
+ private final WindowTask userTask = new WindowTask();
+
+ private final BroadcastTask splitTask = new BroadcastTask();
+
+ private final JoinTask joinTask = new JoinTask();
+
+ private final Set<SystemStreamPartition> inputPartitions = new HashSet<SystemStreamPartition>() { {
+ for (int i = 0; i < 4; i++) {
+ this.add(new SystemStreamPartition("my-system", "my-topic1", new Partition(i)));
+ }
+ } };
+
+ @Test
+ public void testUserTask() throws Exception {
+ Config mockConfig = mock(Config.class);
+ TaskContext mockContext = mock(TaskContext.class);
+ when(mockContext.getSystemStreamPartitions()).thenReturn(this.inputPartitions);
+ StreamOperatorAdaptorTask adaptorTask = new StreamOperatorAdaptorTask(this.userTask);
+ Field pipelineMapFld = StreamOperatorAdaptorTask.class.getDeclaredField("operatorChains");
+ pipelineMapFld.setAccessible(true);
+ Map<SystemStreamPartition, OperatorImpl> pipelineMap =
+ (Map<SystemStreamPartition, OperatorImpl>) pipelineMapFld.get(adaptorTask);
+
+ adaptorTask.init(mockConfig, mockContext);
+ assertEquals(pipelineMap.size(), 4);
+ this.inputPartitions.forEach(partition -> {
+ assertNotNull(pipelineMap.get(partition));
+ });
+ }
+
+ @Test
+ public void testSplitTask() throws Exception {
+ Config mockConfig = mock(Config.class);
+ TaskContext mockContext = mock(TaskContext.class);
+ when(mockContext.getSystemStreamPartitions()).thenReturn(this.inputPartitions);
+ StreamOperatorAdaptorTask adaptorTask = new StreamOperatorAdaptorTask(this.splitTask);
+ Field pipelineMapFld = StreamOperatorAdaptorTask.class.getDeclaredField("operatorChains");
+ pipelineMapFld.setAccessible(true);
+ Map<SystemStreamPartition, OperatorImpl> pipelineMap =
+ (Map<SystemStreamPartition, OperatorImpl>) pipelineMapFld.get(adaptorTask);
+
+ adaptorTask.init(mockConfig, mockContext);
+ assertEquals(pipelineMap.size(), 4);
+ this.inputPartitions.forEach(partition -> {
+ assertNotNull(pipelineMap.get(partition));
+ });
+ }
+
+ @Test
+ public void testJoinTask() throws Exception {
+ Config mockConfig = mock(Config.class);
+ TaskContext mockContext = mock(TaskContext.class);
+ when(mockContext.getSystemStreamPartitions()).thenReturn(this.inputPartitions);
+ StreamOperatorAdaptorTask adaptorTask = new StreamOperatorAdaptorTask(this.joinTask);
+ Field pipelineMapFld = StreamOperatorAdaptorTask.class.getDeclaredField("operatorChains");
+ pipelineMapFld.setAccessible(true);
+ Map<SystemStreamPartition, OperatorImpl> pipelineMap =
+ (Map<SystemStreamPartition, OperatorImpl>) pipelineMapFld.get(adaptorTask);
+
+ adaptorTask.init(mockConfig, mockContext);
+ assertEquals(pipelineMap.size(), 4);
+ this.inputPartitions.forEach(partition -> {
+ assertNotNull(pipelineMap.get(partition));
+ });
+ }
+
+
+}