You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@samza.apache.org by jm...@apache.org on 2017/02/23 19:27:47 UTC
[01/13] samza git commit: SAMZA-1086;
New Grouper for ZK based standalone.
Repository: samza
Updated Branches:
refs/heads/samza-fluent-api-v1 ea37b7463 -> 03bacdc60
SAMZA-1086; New Grouper for ZK based standalone.
SAMZA-1086.
Create new grouper with support for arbitrary container ids.
Add support for this list of container IDs in the JobModelManager.
Author: Boris Shkolnik <bs...@bshkolni-ld1.linkedin.biz>
Reviewers: Xinyu Liu <xi...@linkedin.com>, Fred Ji <fr...@yahoo.com>
Closes #52 from sborya/JobModel
Project: http://git-wip-us.apache.org/repos/asf/samza/repo
Commit: http://git-wip-us.apache.org/repos/asf/samza/commit/daaad7b8
Tree: http://git-wip-us.apache.org/repos/asf/samza/tree/daaad7b8
Diff: http://git-wip-us.apache.org/repos/asf/samza/diff/daaad7b8
Branch: refs/heads/samza-fluent-api-v1
Commit: daaad7b84837a5fc7f995234248d3349a7f6f1d2
Parents: c249443
Author: Boris Shkolnik <bo...@apache.org>
Authored: Thu Feb 16 11:40:12 2017 -0800
Committer: navina <na...@apache.org>
Committed: Thu Feb 16 11:40:12 2017 -0800
----------------------------------------------------------------------
.../grouper/task/GroupByContainerIds.java | 97 +++++++++
.../task/GroupByContainerIdsFactory.java | 34 +++
.../container/grouper/task/TaskNameGrouper.java | 5 +
.../standalone/StandaloneJobCoordinator.java | 2 +-
.../samza/coordinator/JobModelManager.scala | 18 +-
.../grouper/task/TestGroupByContainerIds.java | 207 +++++++++++++++++++
6 files changed, 355 insertions(+), 8 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/samza/blob/daaad7b8/samza-core/src/main/java/org/apache/samza/container/grouper/task/GroupByContainerIds.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/container/grouper/task/GroupByContainerIds.java b/samza-core/src/main/java/org/apache/samza/container/grouper/task/GroupByContainerIds.java
new file mode 100644
index 0000000..6d3f673
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/container/grouper/task/GroupByContainerIds.java
@@ -0,0 +1,97 @@
+/*
+ * 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.container.grouper.task;
+
+import java.util.Arrays;
+import org.apache.samza.container.TaskName;
+import org.apache.samza.job.model.ContainerModel;
+import org.apache.samza.job.model.TaskModel;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+
+/**
+ * Simple grouper.
+ * It exposes two group methods - one that assumes sequential container numbers and one that gets a set of container
+ * IDs as an argument. Please note - this first implementation ignores locality information.
+ */
+public class GroupByContainerIds implements TaskNameGrouper {
+ private final int startContainerCount;
+ public GroupByContainerIds(int count) {
+ this.startContainerCount = count;
+ }
+
+ @Override
+ public Set<ContainerModel> group(Set<TaskModel> tasks) {
+ if (tasks.isEmpty())
+ throw new IllegalArgumentException("cannot group an empty set");
+
+ if (startContainerCount > tasks.size())
+ throw new IllegalArgumentException("number of containers=" + startContainerCount + " is bigger than number of tasks=" + tasks.size());
+
+ List<Integer> containerIds = new ArrayList<>(startContainerCount);
+ for (int i = 0; i < startContainerCount; i++) {
+ containerIds.add(i);
+ }
+ return group(tasks, containerIds);
+ }
+
+ public Set<ContainerModel> group(Set<TaskModel> tasks, List<Integer> containersIds) {
+ if (tasks.isEmpty())
+ throw new IllegalArgumentException("cannot group an empty set. containersIds=" + Arrays
+ .toString(containersIds.toArray()));
+
+ if (containersIds.size() > tasks.size())
+ throw new IllegalArgumentException("number of containers " + containersIds.size() + " is bigger than number of tasks " + tasks.size());
+
+ if (containersIds == null)
+ return this.group(tasks);
+
+ int containerCount = containersIds.size();
+
+ // Sort tasks by taskName.
+ List<TaskModel> sortedTasks = new ArrayList<>(tasks);
+ Collections.sort(sortedTasks);
+
+ // Map every task to a container in round-robin fashion.
+ Map<TaskName, TaskModel>[] taskGroups = new Map[containerCount];
+ for (int i = 0; i < containerCount; i++) {
+ taskGroups[i] = new HashMap<>();
+ }
+ for (int i = 0; i < sortedTasks.size(); i++) {
+ TaskModel tm = sortedTasks.get(i);
+ taskGroups[i % containerCount].put(tm.getTaskName(), tm);
+ }
+
+ // Convert to a Set of ContainerModel
+ Set<ContainerModel> containerModels = new HashSet<>();
+ for (int i = 0; i < containerCount; i++) {
+ containerModels.add(new ContainerModel(containersIds.get(i), taskGroups[i]));
+ }
+
+ return Collections.unmodifiableSet(containerModels);
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/daaad7b8/samza-core/src/main/java/org/apache/samza/container/grouper/task/GroupByContainerIdsFactory.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/container/grouper/task/GroupByContainerIdsFactory.java b/samza-core/src/main/java/org/apache/samza/container/grouper/task/GroupByContainerIdsFactory.java
new file mode 100644
index 0000000..0383d00
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/container/grouper/task/GroupByContainerIdsFactory.java
@@ -0,0 +1,34 @@
+/*
+ * 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.container.grouper.task;
+
+
+import org.apache.samza.config.Config;
+import org.apache.samza.config.JobConfig;
+
+
+/**
+ * Factory to build the GroupByContainerCount class.
+ */
+public class GroupByContainerIdsFactory implements TaskNameGrouperFactory {
+ @Override
+ public TaskNameGrouper build(Config config) {
+ return new GroupByContainerIds(new JobConfig(config).getContainerCount());
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/daaad7b8/samza-core/src/main/java/org/apache/samza/container/grouper/task/TaskNameGrouper.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/container/grouper/task/TaskNameGrouper.java b/samza-core/src/main/java/org/apache/samza/container/grouper/task/TaskNameGrouper.java
index 59a3237..d06bf62 100644
--- a/samza-core/src/main/java/org/apache/samza/container/grouper/task/TaskNameGrouper.java
+++ b/samza-core/src/main/java/org/apache/samza/container/grouper/task/TaskNameGrouper.java
@@ -18,6 +18,7 @@
*/
package org.apache.samza.container.grouper.task;
+import java.util.List;
import org.apache.samza.job.model.ContainerModel;
import org.apache.samza.job.model.TaskModel;
@@ -50,4 +51,8 @@ public interface TaskNameGrouper {
* @return Set of containers, which contain the tasks that were passed in.
*/
Set<ContainerModel> group(Set<TaskModel> tasks);
+
+ default Set<ContainerModel> group(Set<TaskModel> tasks, List<Integer> containersIds) {
+ return group(tasks);
+ }
}
http://git-wip-us.apache.org/repos/asf/samza/blob/daaad7b8/samza-core/src/main/java/org/apache/samza/standalone/StandaloneJobCoordinator.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/standalone/StandaloneJobCoordinator.java b/samza-core/src/main/java/org/apache/samza/standalone/StandaloneJobCoordinator.java
index 1401725..46dbf30 100644
--- a/samza-core/src/main/java/org/apache/samza/standalone/StandaloneJobCoordinator.java
+++ b/samza-core/src/main/java/org/apache/samza/standalone/StandaloneJobCoordinator.java
@@ -105,7 +105,7 @@ public class StandaloneJobCoordinator implements JobCoordinator {
* TaskNameGrouper with the LocalityManager! Hence, groupers should be a property of the jobcoordinator
* (job.coordinator.task.grouper, instead of task.systemstreampartition.grouper)
*/
- this.jobModelManager = JobModelManager$.MODULE$.getJobCoordinator(this.config, null, null, streamMetadataCache, null);
+ this.jobModelManager = JobModelManager$.MODULE$.getJobCoordinator(this.config, null, null, streamMetadataCache, null, null);
}
@Override
http://git-wip-us.apache.org/repos/asf/samza/blob/daaad7b8/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala b/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala
index 7f5d05d..14d5dff 100644
--- a/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala
+++ b/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala
@@ -106,7 +106,7 @@ object JobModelManager extends Logging {
}
}
- val jobCoordinator = getJobCoordinator(config, changelogManager, localityManager, streamMetadataCache, streamPartitionCountMonitor)
+ val jobCoordinator = getJobCoordinator(config, changelogManager, localityManager, streamMetadataCache, streamPartitionCountMonitor, null)
createChangeLogStreams(config, jobCoordinator.jobModel.maxChangeLogStreamPartitions)
jobCoordinator
@@ -121,8 +121,9 @@ object JobModelManager extends Logging {
changelogManager: ChangelogPartitionManager,
localityManager: LocalityManager,
streamMetadataCache: StreamMetadataCache,
- streamPartitionCountMonitor: StreamPartitionCountMonitor) = {
- val jobModel: JobModel = initializeJobModel(config, changelogManager, localityManager, streamMetadataCache)
+ streamPartitionCountMonitor: StreamPartitionCountMonitor,
+ containerIds: java.util.List[Integer]) = {
+ val jobModel: JobModel = initializeJobModel(config, changelogManager, localityManager, streamMetadataCache, containerIds)
jobModelRef.set(jobModel)
val server = new HttpServer
@@ -188,7 +189,8 @@ object JobModelManager extends Logging {
private def initializeJobModel(config: Config,
changelogManager: ChangelogPartitionManager,
localityManager: LocalityManager,
- streamMetadataCache: StreamMetadataCache): JobModel = {
+ streamMetadataCache: StreamMetadataCache,
+ containerIds: java.util.List[Integer]): JobModel = {
// Do grouping to fetch TaskName to SSP mapping
val allSystemStreamPartitions = getMatchedInputStreamPartitions(config, streamMetadataCache)
val grouper = getSystemStreamPartitionGrouper(config)
@@ -216,7 +218,8 @@ object JobModelManager extends Logging {
def jobModelGenerator(): JobModel = refreshJobModel(config,
groups,
previousChangelogMapping,
- localityManager)
+ localityManager,
+ containerIds)
val jobModel = jobModelGenerator()
@@ -248,7 +251,8 @@ object JobModelManager extends Logging {
private def refreshJobModel(config: Config,
groups: util.Map[TaskName, util.Set[SystemStreamPartition]],
previousChangelogMapping: util.Map[TaskName, Integer],
- localityManager: LocalityManager): JobModel = {
+ localityManager: LocalityManager,
+ containerIds: java.util.List[Integer]): JobModel = {
// If no mappings are present(first time the job is running) we return -1, this will allow 0 to be the first change
// mapping.
@@ -280,7 +284,7 @@ object JobModelManager extends Logging {
if (containerGrouper.isInstanceOf[BalancingTaskNameGrouper])
containerGrouper.asInstanceOf[BalancingTaskNameGrouper].balance(taskModels, localityManager)
else
- containerGrouper.group(taskModels)
+ containerGrouper.group(taskModels, containerIds)
}
val containerMap = asScalaSet(containerModels).map { case (containerModel) => Integer.valueOf(containerModel.getContainerId) -> containerModel }.toMap
http://git-wip-us.apache.org/repos/asf/samza/blob/daaad7b8/samza-core/src/test/java/org/apache/samza/container/grouper/task/TestGroupByContainerIds.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/container/grouper/task/TestGroupByContainerIds.java b/samza-core/src/test/java/org/apache/samza/container/grouper/task/TestGroupByContainerIds.java
new file mode 100644
index 0000000..82f2b7a
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/container/grouper/task/TestGroupByContainerIds.java
@@ -0,0 +1,207 @@
+/*
+ * 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.container.grouper.task;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.MapConfig;
+import org.apache.samza.container.LocalityManager;
+import org.apache.samza.job.model.ContainerModel;
+import org.apache.samza.job.model.TaskModel;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.apache.samza.container.mock.ContainerMocks.generateTaskModels;
+import static org.apache.samza.container.mock.ContainerMocks.getTaskModel;
+import static org.apache.samza.container.mock.ContainerMocks.getTaskName;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+
+public class TestGroupByContainerIds {
+ private TaskAssignmentManager taskAssignmentManager;
+ private LocalityManager localityManager;
+
+ @Before
+ public void setup() {
+ taskAssignmentManager = mock(TaskAssignmentManager.class);
+ localityManager = mock(LocalityManager.class);
+ when(localityManager.getTaskAssignmentManager()).thenReturn(taskAssignmentManager);
+
+
+ }
+
+ private Config buildConfigForContainerCount(int count) {
+ Map<String, String> map = new HashMap<>();
+ map.put("job.container.count", String.valueOf(count));
+ return new MapConfig(map);
+ }
+
+ private TaskNameGrouper buildSimpleGrouper() {
+ return buildSimpleGrouper(1);
+ }
+ private TaskNameGrouper buildSimpleGrouper(int containerCount) {
+ return new GroupByContainerIdsFactory().build(buildConfigForContainerCount(containerCount));
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testGroupEmptyTasks() {
+ buildSimpleGrouper(1).group(new HashSet());
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testGroupFewerTasksThanContainers() {
+ Set<TaskModel> taskModels = new HashSet<>();
+ taskModels.add(getTaskModel(1));
+ buildSimpleGrouper(2).group(taskModels);
+ }
+
+ @Test(expected = UnsupportedOperationException.class)
+ public void testGrouperResultImmutable() {
+ Set<TaskModel> taskModels = generateTaskModels(3);
+ Set<ContainerModel> containers = buildSimpleGrouper(2).group(taskModels);
+ containers.remove(containers.iterator().next());
+ }
+
+ @Test
+ public void testGroupHappyPath() {
+ Set<TaskModel> taskModels = generateTaskModels(5);
+
+ Set<ContainerModel> containers = buildSimpleGrouper(2).group(taskModels);
+
+ Map<Integer, ContainerModel> containersMap = new HashMap<>();
+ for (ContainerModel container : containers) {
+ containersMap.put(container.getContainerId(), container);
+ }
+
+ assertEquals(2, containers.size());
+ ContainerModel container0 = containersMap.get(0);
+ ContainerModel container1 = containersMap.get(1);
+ assertNotNull(container0);
+ assertNotNull(container1);
+ assertEquals(0, container0.getContainerId());
+ assertEquals(1, container1.getContainerId());
+ assertEquals(3, container0.getTasks().size());
+ assertEquals(2, container1.getTasks().size());
+ assertTrue(container0.getTasks().containsKey(getTaskName(0)));
+ assertTrue(container0.getTasks().containsKey(getTaskName(2)));
+ assertTrue(container0.getTasks().containsKey(getTaskName(4)));
+ assertTrue(container1.getTasks().containsKey(getTaskName(1)));
+ assertTrue(container1.getTasks().containsKey(getTaskName(3)));
+ }
+
+ @Test
+ public void testGroupHappyPathWithListOfContainers() {
+ Set<TaskModel> taskModels = generateTaskModels(5);
+
+ List<Integer> containerIds = new ArrayList<Integer>() {
+ {
+ add(4);
+ add(2);
+ }
+ };
+
+ Set<ContainerModel> containers = buildSimpleGrouper().group(taskModels, containerIds);
+
+ Map<Integer, ContainerModel> containersMap = new HashMap<>();
+ for (ContainerModel container : containers) {
+ containersMap.put(container.getContainerId(), container);
+ }
+
+ assertEquals(2, containers.size());
+ ContainerModel container0 = containersMap.get(4);
+ ContainerModel container1 = containersMap.get(2);
+ assertNotNull(container0);
+ assertNotNull(container1);
+ assertEquals(4, container0.getContainerId());
+ assertEquals(2, container1.getContainerId());
+ assertEquals(3, container0.getTasks().size());
+ assertEquals(2, container1.getTasks().size());
+ assertTrue(container0.getTasks().containsKey(getTaskName(0)));
+ assertTrue(container0.getTasks().containsKey(getTaskName(2)));
+ assertTrue(container0.getTasks().containsKey(getTaskName(4)));
+ assertTrue(container1.getTasks().containsKey(getTaskName(1)));
+ assertTrue(container1.getTasks().containsKey(getTaskName(3)));
+ }
+
+
+ @Test
+ public void testGroupManyTasks() {
+ Set<TaskModel> taskModels = generateTaskModels(21);
+
+ List<Integer> containerIds = new ArrayList<Integer>() {
+ {
+ add(4);
+ add(2);
+ }
+ };
+
+
+ Set<ContainerModel> containers = buildSimpleGrouper().group(taskModels, containerIds);
+
+ Map<Integer, ContainerModel> containersMap = new HashMap<>();
+ for (ContainerModel container : containers) {
+ containersMap.put(container.getContainerId(), container);
+ }
+
+ assertEquals(2, containers.size());
+ ContainerModel container0 = containersMap.get(4);
+ ContainerModel container1 = containersMap.get(2);
+ assertNotNull(container0);
+ assertNotNull(container1);
+ assertEquals(4, container0.getContainerId());
+ assertEquals(2, container1.getContainerId());
+ assertEquals(11, container0.getTasks().size());
+ assertEquals(10, container1.getTasks().size());
+
+ // NOTE: tasks are sorted lexicographically, so the container assignment
+ // can seem odd, but the consistency is the key focus
+ assertTrue(container0.getTasks().containsKey(getTaskName(0)));
+ assertTrue(container0.getTasks().containsKey(getTaskName(10)));
+ assertTrue(container0.getTasks().containsKey(getTaskName(12)));
+ assertTrue(container0.getTasks().containsKey(getTaskName(14)));
+ assertTrue(container0.getTasks().containsKey(getTaskName(16)));
+ assertTrue(container0.getTasks().containsKey(getTaskName(18)));
+ assertTrue(container0.getTasks().containsKey(getTaskName(2)));
+ assertTrue(container0.getTasks().containsKey(getTaskName(3)));
+ assertTrue(container0.getTasks().containsKey(getTaskName(5)));
+ assertTrue(container0.getTasks().containsKey(getTaskName(7)));
+ assertTrue(container0.getTasks().containsKey(getTaskName(9)));
+
+ assertTrue(container1.getTasks().containsKey(getTaskName(1)));
+ assertTrue(container1.getTasks().containsKey(getTaskName(11)));
+ assertTrue(container1.getTasks().containsKey(getTaskName(13)));
+ assertTrue(container1.getTasks().containsKey(getTaskName(15)));
+ assertTrue(container1.getTasks().containsKey(getTaskName(17)));
+ assertTrue(container1.getTasks().containsKey(getTaskName(19)));
+ assertTrue(container1.getTasks().containsKey(getTaskName(20)));
+ assertTrue(container1.getTasks().containsKey(getTaskName(4)));
+ assertTrue(container1.getTasks().containsKey(getTaskName(6)));
+ assertTrue(container1.getTasks().containsKey(getTaskName(8)));
+ }
+}
[02/13] samza git commit: SAMZA-1073: moving all operator classes
into samza-core
Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-operator/src/test/java/org/apache/samza/operators/TestMessageStreamImplUtil.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/operators/TestMessageStreamImplUtil.java b/samza-operator/src/test/java/org/apache/samza/operators/TestMessageStreamImplUtil.java
deleted file mode 100644
index c4e9f51..0000000
--- a/samza-operator/src/test/java/org/apache/samza/operators/TestMessageStreamImplUtil.java
+++ /dev/null
@@ -1,26 +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;
-
-
-public class TestMessageStreamImplUtil {
- public static <M> MessageStreamImpl<M> getMessageStreamImpl(StreamGraphImpl graph) {
- return new MessageStreamImpl<M>(graph);
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/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
deleted file mode 100644
index 9a425d1..0000000
--- a/samza-operator/src/test/java/org/apache/samza/operators/data/JsonIncomingSystemMessageEnvelope.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.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/8515448a/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
deleted file mode 100644
index 361972e..0000000
--- a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java
+++ /dev/null
@@ -1,73 +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.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 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;
-
-
-public class TestOperatorImpl {
-
- TestMessageEnvelope curInputMsg;
- MessageCollector curCollector;
- TaskCoordinator curCoordinator;
-
- @Test
- public void testSubscribers() {
- this.curInputMsg = null;
- this.curCollector = null;
- this.curCoordinator = null;
- 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 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.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
- 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/8515448a/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
deleted file mode 100644
index 02637a3..0000000
--- a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestOperatorImpls.java
+++ /dev/null
@@ -1,235 +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 java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.MessageStreamImpl;
-import org.apache.samza.operators.StreamGraphImpl;
-import org.apache.samza.operators.TestMessageEnvelope;
-import org.apache.samza.operators.TestMessageStreamImplUtil;
-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.JoinFunction;
-import org.apache.samza.operators.functions.PartialJoinFunction;
-import org.apache.samza.operators.functions.SinkFunction;
-import org.apache.samza.operators.spec.OperatorSpec;
-import org.apache.samza.operators.spec.WindowOperatorSpec;
-import org.apache.samza.operators.spec.PartialJoinOperatorSpec;
-import org.apache.samza.operators.spec.SinkOperatorSpec;
-import org.apache.samza.operators.spec.StreamOperatorSpec;
-import org.apache.samza.operators.windows.Windows;
-import org.apache.samza.operators.windows.internal.WindowInternal;
-import org.apache.samza.task.TaskContext;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.lang.reflect.Field;
-import java.time.Duration;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.Set;
-
-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;
- Method createOpMethod = null;
- Method createOpsMethod = null;
-
- @Before
- public void prep() throws NoSuchFieldException, NoSuchMethodException {
- nextOperatorsField = OperatorImpl.class.getDeclaredField("nextOperators");
- nextOperatorsField.setAccessible(true);
-
- createOpMethod = OperatorGraph.class.getDeclaredMethod("createOperatorImpl", MessageStreamImpl.class,
- OperatorSpec.class, Config.class, TaskContext.class);
- createOpMethod.setAccessible(true);
-
- createOpsMethod = OperatorGraph.class.getDeclaredMethod("createOperatorImpls", MessageStreamImpl.class, Config.class, TaskContext.class);
- createOpsMethod.setAccessible(true);
- }
-
- @Test
- public void testCreateOperator() throws NoSuchFieldException, IllegalAccessException, InvocationTargetException {
- // get window operator
- WindowOperatorSpec mockWnd = mock(WindowOperatorSpec.class);
- WindowInternal<TestMessageEnvelope, String, Integer> windowInternal = new WindowInternal<>(null, null, null, null);
- when(mockWnd.getWindow()).thenReturn(windowInternal);
- MessageStreamImpl<TestMessageEnvelope> mockStream = mock(MessageStreamImpl.class);
- Config mockConfig = mock(Config.class);
- TaskContext mockContext = mock(TaskContext.class);
-
- OperatorGraph opGraph = new OperatorGraph();
- OperatorImpl<TestMessageEnvelope, ? extends MessageEnvelope> opImpl = (OperatorImpl<TestMessageEnvelope, ? extends MessageEnvelope>)
- createOpMethod.invoke(opGraph, mockStream, mockWnd, mockConfig, mockContext);
- assertTrue(opImpl instanceof WindowOperatorImpl);
- Field wndInternalField = WindowOperatorImpl.class.getDeclaredField("window");
- wndInternalField.setAccessible(true);
- WindowInternal wndInternal = (WindowInternal) wndInternalField.get(opImpl);
- assertEquals(wndInternal, windowInternal);
-
- // get simple operator
- StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope> mockSimpleOp = mock(StreamOperatorSpec.class);
- FlatMapFunction<TestMessageEnvelope, TestOutputMessageEnvelope> mockTxfmFn = mock(FlatMapFunction.class);
- when(mockSimpleOp.getTransformFn()).thenReturn(mockTxfmFn);
- opImpl = (OperatorImpl<TestMessageEnvelope, ? extends MessageEnvelope>) createOpMethod.invoke(opGraph, mockStream, mockSimpleOp, mockConfig, mockContext);
- 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 = (OperatorImpl<TestMessageEnvelope, ? extends MessageEnvelope>) createOpMethod.invoke(opGraph, mockStream, sinkOp, mockConfig, mockContext);
- 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);
- PartialJoinFunction<String, TestMessageEnvelope, TestMessageEnvelope, TestOutputMessageEnvelope> joinFn = mock(PartialJoinFunction.class);
- when(joinOp.getTransformFn()).thenReturn(joinFn);
- opImpl = (OperatorImpl<TestMessageEnvelope, ? extends MessageEnvelope>) createOpMethod.invoke(opGraph, mockStream, joinOp, mockConfig, mockContext);
- assertTrue(opImpl instanceof PartialJoinOperatorImpl);
- }
-
- @Test
- public void testEmptyChain() throws InvocationTargetException, IllegalAccessException {
- // test creation of empty chain
- MessageStreamImpl<TestMessageEnvelope> testStream = mock(MessageStreamImpl.class);
- TaskContext mockContext = mock(TaskContext.class);
- Config mockConfig = mock(Config.class);
- OperatorGraph opGraph = new OperatorGraph();
- RootOperatorImpl operatorChain = (RootOperatorImpl) createOpsMethod.invoke(opGraph, testStream, mockConfig, mockContext);
- assertTrue(operatorChain != null);
- }
-
- @Test
- public void testLinearChain() throws IllegalAccessException, InvocationTargetException {
- // test creation of linear chain
- StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
- MessageStreamImpl<TestMessageEnvelope> testInput = TestMessageStreamImplUtil.<TestMessageEnvelope>getMessageStreamImpl(mockGraph);
- TaskContext mockContext = mock(TaskContext.class);
- Config mockConfig = mock(Config.class);
- testInput.map(m -> m).window(Windows.keyedSessionWindow(TestMessageEnvelope::getKey, Duration.ofMinutes(10)));
- OperatorGraph opGraph = new OperatorGraph();
- RootOperatorImpl operatorChain = (RootOperatorImpl) createOpsMethod.invoke(opGraph, testInput, mockConfig, 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, InvocationTargetException {
- // test creation of broadcast chain
- StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
- MessageStreamImpl<TestMessageEnvelope> testInput = TestMessageStreamImplUtil.<TestMessageEnvelope>getMessageStreamImpl(mockGraph);
- TaskContext mockContext = mock(TaskContext.class);
- Config mockConfig = mock(Config.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);
- OperatorGraph opGraph = new OperatorGraph();
- RootOperatorImpl operatorChain = (RootOperatorImpl) createOpsMethod.invoke(opGraph, testInput, mockConfig, 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, InvocationTargetException {
- // test creation of join chain
- StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
- MessageStreamImpl<TestMessageEnvelope> input1 = TestMessageStreamImplUtil.getMessageStreamImpl(mockGraph);
- MessageStreamImpl<TestMessageEnvelope> input2 = TestMessageStreamImplUtil.getMessageStreamImpl(mockGraph);
- TaskContext mockContext = mock(TaskContext.class);
- Config mockConfig = mock(Config.class);
- input1
- .join(input2,
- new JoinFunction<String, TestMessageEnvelope, TestMessageEnvelope, TestOutputMessageEnvelope>() {
- @Override
- public TestOutputMessageEnvelope apply(TestMessageEnvelope m1, TestMessageEnvelope m2) {
- return new TestOutputMessageEnvelope(m1.getKey(), m1.getMessage().getValue().length() + m2.getMessage().getValue().length());
- }
-
- @Override
- public String getFirstKey(TestMessageEnvelope message) {
- return message.getKey();
- }
-
- @Override
- public String getSecondKey(TestMessageEnvelope message) {
- return message.getKey();
- }
- })
- .map(m -> m);
- OperatorGraph opGraph = new OperatorGraph();
- // now, we create chained operators from each input sources
- RootOperatorImpl chain1 = (RootOperatorImpl) createOpsMethod.invoke(opGraph, input1, mockConfig, mockContext);
- RootOperatorImpl chain2 = (RootOperatorImpl) createOpsMethod.invoke(opGraph, input2, mockConfig, 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/8515448a/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
deleted file mode 100644
index ce9fdd2..0000000
--- a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestSinkOperatorImpl.java
+++ /dev/null
@@ -1,50 +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.config.Config;
-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.TaskContext;
-import org.apache.samza.task.TaskCoordinator;
-import org.junit.Test;
-
-import static org.mockito.Mockito.*;
-
-
-public class TestSinkOperatorImpl {
-
- @Test
- public void testSinkOperator() {
- SinkOperatorSpec<TestOutputMessageEnvelope> sinkOp = mock(SinkOperatorSpec.class);
- SinkFunction<TestOutputMessageEnvelope> sinkFn = mock(SinkFunction.class);
- when(sinkOp.getSinkFn()).thenReturn(sinkFn);
- Config mockConfig = mock(Config.class);
- TaskContext mockContext = mock(TaskContext.class);
- SinkOperatorImpl<TestOutputMessageEnvelope> sinkImpl = new SinkOperatorImpl<>(sinkOp, mockConfig, mockContext);
- TestOutputMessageEnvelope mockMsg = mock(TestOutputMessageEnvelope.class);
- MessageCollector mockCollector = mock(MessageCollector.class);
- TaskCoordinator mockCoordinator = mock(TaskCoordinator.class);
-
- sinkImpl.onNext(mockMsg, mockCollector, mockCoordinator);
- verify(sinkFn, times(1)).apply(mockMsg, mockCollector, mockCoordinator);
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/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
deleted file mode 100644
index 010a210..0000000
--- a/samza-operator/src/test/java/org/apache/samza/operators/impl/TestStreamOperatorImpl.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.impl;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.MessageStreamImpl;
-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.TaskContext;
-import org.apache.samza.task.TaskCoordinator;
-import org.junit.Test;
-
-import static org.mockito.Mockito.*;
-
-
-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);
- MessageStreamImpl<TestMessageEnvelope> mockInput = mock(MessageStreamImpl.class);
- Config mockConfig = mock(Config.class);
- TaskContext mockContext = mock(TaskContext.class);
- StreamOperatorImpl<TestMessageEnvelope, TestOutputMessageEnvelope> opImpl = spy(new StreamOperatorImpl<>(mockOp, mockInput, mockConfig, mockContext));
- 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/8515448a/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
deleted file mode 100644
index 31257a4..0000000
--- a/samza-operator/src/test/java/org/apache/samza/operators/spec/TestOperatorSpecs.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.samza.operators.spec;
-
-import org.apache.samza.operators.MessageStreamImpl;
-import org.apache.samza.operators.StreamGraphImpl;
-import org.apache.samza.operators.TestMessageEnvelope;
-import org.apache.samza.operators.TestMessageStreamImplUtil;
-import org.apache.samza.operators.data.MessageEnvelope;
-import org.apache.samza.operators.functions.FlatMapFunction;
-import org.apache.samza.operators.functions.PartialJoinFunction;
-import org.apache.samza.operators.functions.SinkFunction;
-import org.apache.samza.operators.windows.internal.WindowInternal;
-import org.apache.samza.operators.windows.WindowPane;
-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.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-
-
-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));
- } };
- MessageStreamImpl<TestMessageEnvelope> mockOutput = mock(MessageStreamImpl.class);
- StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
- StreamOperatorSpec<MessageEnvelope, TestMessageEnvelope> strmOp = OperatorSpecs.createStreamOperatorSpec(transformFn, mockGraph, mockOutput);
- assertEquals(strmOp.getTransformFn(), transformFn);
- assertEquals(strmOp.getNextStream(), mockOutput);
- }
-
- @Test
- public void testGetSinkOperator() {
- SinkFunction<TestMessageEnvelope> sinkFn = (TestMessageEnvelope message, MessageCollector messageCollector,
- TaskCoordinator taskCoordinator) -> { };
- StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
- SinkOperatorSpec<TestMessageEnvelope> sinkOp = OperatorSpecs.createSinkOperatorSpec(sinkFn, mockGraph);
- assertEquals(sinkOp.getSinkFn(), sinkFn);
- assertTrue(sinkOp.getNextStream() == null);
- }
-
- @Test
- public void testGetWindowOperator() throws Exception {
- Function<TestMessageEnvelope, String> keyExtractor = m -> "globalkey";
- BiFunction<TestMessageEnvelope, Integer, Integer> aggregator = (m, c) -> c + 1;
-
- //instantiate a window using reflection
- WindowInternal window = new WindowInternal(null, aggregator, keyExtractor, null);
-
- StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
- MessageStreamImpl<WindowPane<String, Integer>> mockWndOut = mock(MessageStreamImpl.class);
- WindowOperatorSpec spec = OperatorSpecs.<TestMessageEnvelope, String, Integer>createWindowOperatorSpec(window, mockGraph, mockWndOut);
- assertEquals(spec.getWindow(), window);
- assertEquals(spec.getWindow().getKeyExtractor(), keyExtractor);
- assertEquals(spec.getWindow().getFoldFunction(), aggregator);
- }
-
- @Test
- public void testGetPartialJoinOperator() {
- PartialJoinFunction<Object, MessageEnvelope<Object, ?>, MessageEnvelope<Object, ?>, TestMessageEnvelope> merger =
- new PartialJoinFunction<Object, MessageEnvelope<Object, ?>, MessageEnvelope<Object, ?>, TestMessageEnvelope>() {
- @Override
- public TestMessageEnvelope apply(MessageEnvelope<Object, ?> m1, MessageEnvelope<Object, ?> m2) {
- return new TestMessageEnvelope(m1.getKey().toString(), m2.getMessage().toString(), System.nanoTime());
- }
-
- @Override
- public Object getKey(MessageEnvelope<Object, ?> message) {
- return message.getKey();
- }
-
- @Override
- public Object getOtherKey(MessageEnvelope<Object, ?> message) {
- return message.getKey();
- }
- };
-
- StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
- MessageStreamImpl<TestMessageEnvelope> joinOutput = TestMessageStreamImplUtil.<TestMessageEnvelope>getMessageStreamImpl(mockGraph);
- PartialJoinOperatorSpec<MessageEnvelope<Object, ?>, Object, MessageEnvelope<Object, ?>, TestMessageEnvelope> partialJoin =
- OperatorSpecs.createPartialJoinOperatorSpec(merger, mockGraph, joinOutput);
-
- assertEquals(partialJoin.getNextStream(), joinOutput);
- MessageEnvelope<Object, Object> m = mock(MessageEnvelope.class);
- MessageEnvelope<Object, Object> s = mock(MessageEnvelope.class);
- assertEquals(partialJoin.getTransformFn(), merger);
- }
-
- @Test
- public void testGetMergeOperator() {
- StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
- MessageStreamImpl<TestMessageEnvelope> output = TestMessageStreamImplUtil.<TestMessageEnvelope>getMessageStreamImpl(mockGraph);
- StreamOperatorSpec<TestMessageEnvelope, TestMessageEnvelope> mergeOp = OperatorSpecs.createMergeOperatorSpec(mockGraph, 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.getNextStream(), output);
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/settings.gradle
----------------------------------------------------------------------
diff --git a/settings.gradle b/settings.gradle
index 813882c..5de30d8 100644
--- a/settings.gradle
+++ b/settings.gradle
@@ -20,7 +20,6 @@ include \
'samza-api',
'samza-elasticsearch',
'samza-log4j',
- 'samza-operator',
'samza-rest',
'samza-shell'
[08/13] samza git commit: Fix hyphens in url for committer
instructions
Posted by jm...@apache.org.
Fix hyphens in url for committer instructions
Project: http://git-wip-us.apache.org/repos/asf/samza/repo
Commit: http://git-wip-us.apache.org/repos/asf/samza/commit/6939b8cf
Tree: http://git-wip-us.apache.org/repos/asf/samza/tree/6939b8cf
Diff: http://git-wip-us.apache.org/repos/asf/samza/diff/6939b8cf
Branch: refs/heads/samza-fluent-api-v1
Commit: 6939b8cf9c4a7b34d2426fb45c0c55d10ebfeafe
Parents: 7e7747f
Author: Jacob Maes <jm...@linkedin.com>
Authored: Fri Feb 17 15:37:22 2017 -0800
Committer: Jacob Maes <jm...@linkedin.com>
Committed: Fri Feb 17 15:37:22 2017 -0800
----------------------------------------------------------------------
docs/contribute/contributors-corner.md | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/samza/blob/6939b8cf/docs/contribute/contributors-corner.md
----------------------------------------------------------------------
diff --git a/docs/contribute/contributors-corner.md b/docs/contribute/contributors-corner.md
index b41bca7..454ce59 100644
--- a/docs/contribute/contributors-corner.md
+++ b/docs/contribute/contributors-corner.md
@@ -97,7 +97,7 @@ If you are a committer you need to use https instead of http to check in, otherw
* Set the `JIRA_USERNAME` and `JIRA_PASSWORD` environment variables with the appropriate credentials for interacting with Jira. This is required to correctly close the JIRA associated with the PR
2. Setup aliases for the remote repositories:\u200b(Samza Github repo and Apache Samza Repo)
* Add ASF git repo for committing the PR
- ```git remote add samza-apache https://git�wip�us.apache.org/repos/asf/samza.git```
+ ```git remote add samza-apache https://git-wip-us.apache.org/repos/asf/samza.git```
* Add Github repo for fetching the patch from the PR
```git remote add samza-github https://github.com/apache/samza.git```
3. Set up API tokens for Git
[05/13] samza git commit: SAMZA-1073: moving all operator classes
into samza-core
Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-core/src/main/java/org/apache/samza/system/StandaloneExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/system/StandaloneExecutionEnvironment.java b/samza-core/src/main/java/org/apache/samza/system/StandaloneExecutionEnvironment.java
new file mode 100644
index 0000000..f0f6ef2
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/system/StandaloneExecutionEnvironment.java
@@ -0,0 +1,50 @@
+/*
+ * 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.system;
+
+import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.operators.StreamGraphBuilder;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.StreamGraphImpl;
+
+
+/**
+ * This class implements the {@link ExecutionEnvironment} that runs the applications in standalone environment
+ */
+public class StandaloneExecutionEnvironment implements ExecutionEnvironment {
+
+ // TODO: may want to move this to a common base class for all {@link ExecutionEnvironment}
+ StreamGraph createGraph(StreamGraphBuilder app, Config config) {
+ StreamGraphImpl graph = new StreamGraphImpl();
+ app.init(graph, config);
+ return graph;
+ }
+
+ @Override public void run(StreamGraphBuilder app, Config config) {
+ // 1. get logic graph for optimization
+ // StreamGraph logicGraph = this.createGraph(app, config);
+ // 2. potential optimization....
+ // 3. create new instance of StreamGraphBuilder that would generate the optimized graph
+ // 4. create all input/output/intermediate topics
+ // 5. create the configuration for StreamProcessor
+ // 6. start the StreamProcessor w/ optimized instance of StreamGraphBuilder
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java b/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.java
new file mode 100644
index 0000000..b007e3c
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/task/StreamOperatorTask.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.task;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.ContextManager;
+import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.StreamGraphBuilder;
+import org.apache.samza.operators.StreamGraphImpl;
+import org.apache.samza.operators.data.InputMessageEnvelope;
+import org.apache.samza.operators.impl.OperatorGraph;
+import org.apache.samza.system.IncomingMessageEnvelope;
+import org.apache.samza.system.SystemStream;
+
+import java.util.HashMap;
+import java.util.Map;
+
+
+/**
+ * Execution of the logic sub-DAG
+ *
+ *
+ * An {@link StreamTask} implementation that receives {@link InputMessageEnvelope}s and propagates them
+ * through the user's stream transformations defined in {@link StreamGraphImpl} using the
+ * {@link org.apache.samza.operators.MessageStream} APIs.
+ * <p>
+ * This class brings all the operator API implementation components together and feeds the
+ * {@link InputMessageEnvelope}s into the transformation chains.
+ * <p>
+ * It accepts an instance of the user implemented factory {@link StreamGraphBuilder} as input parameter of the constructor.
+ * When its own {@link #init(Config, TaskContext)} method is called during startup, it instantiate a user-defined {@link StreamGraphImpl}
+ * from the {@link StreamGraphBuilder}, calls {@link StreamGraphImpl#getContextManager()} to initialize the task-wide context
+ * for the graph, and creates a {@link MessageStreamImpl} corresponding to each of its input
+ * {@link org.apache.samza.system.SystemStreamPartition}s. Each input {@link MessageStreamImpl}
+ * will be corresponding to either an input stream or intermediate stream in {@link StreamGraphImpl}.
+ * <p>
+ * Then, this task calls {@link org.apache.samza.operators.impl.OperatorGraph#init(Map, Config, 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 org.apache.samza.system.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 StreamOperatorTask implements StreamTask, InitableTask, WindowableTask, ClosableTask {
+
+ /**
+ * A mapping from each {@link SystemStream} to the root node of its operator chain DAG.
+ */
+ private final OperatorGraph operatorGraph = new OperatorGraph();
+
+ private final StreamGraphBuilder graphBuilder;
+
+ private ContextManager contextManager;
+
+ public StreamOperatorTask(StreamGraphBuilder graphBuilder) {
+ this.graphBuilder = graphBuilder;
+ }
+
+ @Override
+ public final void init(Config config, TaskContext context) throws Exception {
+ // create the MessageStreamsImpl object and initialize app-specific logic DAG within the task
+ StreamGraphImpl streams = new StreamGraphImpl();
+ this.graphBuilder.init(streams, config);
+ // get the context manager of the {@link StreamGraph} and initialize the task-specific context
+ this.contextManager = streams.getContextManager();
+
+ Map<SystemStream, MessageStreamImpl> inputBySystemStream = new HashMap<>();
+ context.getSystemStreamPartitions().forEach(ssp -> {
+ if (!inputBySystemStream.containsKey(ssp.getSystemStream())) {
+ // create mapping from the physical input {@link SystemStream} to the logic {@link MessageStream}
+ inputBySystemStream.putIfAbsent(ssp.getSystemStream(), streams.getInputStream(ssp.getSystemStream()));
+ }
+ });
+ operatorGraph.init(inputBySystemStream, config, this.contextManager.initTaskContext(config, context));
+ }
+
+ @Override
+ public final void process(IncomingMessageEnvelope ime, MessageCollector collector, TaskCoordinator coordinator) {
+ this.operatorGraph.get(ime.getSystemStreamPartition().getSystemStream())
+ .onNext(new InputMessageEnvelope(ime), collector, coordinator);
+ }
+
+ @Override
+ public final void window(MessageCollector collector, TaskCoordinator coordinator) throws Exception {
+ // TODO: invoke timer based triggers
+ }
+
+ @Override
+ public void close() throws Exception {
+ this.contextManager.finalizeTaskContext();
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-core/src/test/java/org/apache/samza/example/KeyValueStoreExample.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/example/KeyValueStoreExample.java b/samza-core/src/test/java/org/apache/samza/example/KeyValueStoreExample.java
new file mode 100644
index 0000000..85ebc6c
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/example/KeyValueStoreExample.java
@@ -0,0 +1,180 @@
+/*
+ * 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.example;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.samza.operators.*;
+import org.apache.samza.operators.StreamGraphBuilder;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.functions.FlatMapFunction;
+import org.apache.samza.serializers.JsonSerde;
+import org.apache.samza.serializers.StringSerde;
+import org.apache.samza.storage.kv.KeyValueStore;
+import org.apache.samza.system.ExecutionEnvironment;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.task.TaskContext;
+import org.apache.samza.util.CommandLine;
+
+import java.util.Properties;
+
+
+/**
+ * Example code using {@link KeyValueStore} to implement event-time window
+ */
+public class KeyValueStoreExample implements StreamGraphBuilder {
+
+ /**
+ * used by remote execution environment to launch the job in remote program. The remote program should follow the similar
+ * invoking context as in standalone:
+ *
+ * public static void main(String args[]) throws Exception {
+ * CommandLine cmdLine = new CommandLine();
+ * Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+ * ExecutionEnvironment remoteEnv = ExecutionEnvironment.getRemoteEnvironment(config);
+ * UserMainExample runnableApp = new UserMainExample();
+ * runnableApp.run(remoteEnv, config);
+ * }
+ *
+ */
+ @Override public void init(StreamGraph graph, Config config) {
+
+ MessageStream<PageViewEvent> pageViewEvents = graph.createInStream(input1, new StringSerde("UTF-8"), new JsonSerde<>());
+ OutputStream<StatsOutput> pageViewPerMemberCounters = graph.createOutStream(output, new StringSerde("UTF-8"), new JsonSerde<StatsOutput>());
+
+ pageViewEvents.
+ partitionBy(m -> m.getMessage().memberId).
+ flatMap(new MyStatsCounter()).
+ sendTo(pageViewPerMemberCounters);
+
+ }
+
+ // standalone local program model
+ public static void main(String[] args) throws Exception {
+ CommandLine cmdLine = new CommandLine();
+ Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+ ExecutionEnvironment standaloneEnv = ExecutionEnvironment.getLocalEnvironment(config);
+ standaloneEnv.run(new KeyValueStoreExample(), config);
+ }
+
+ class MyStatsCounter implements FlatMapFunction<PageViewEvent, StatsOutput> {
+ private final int timeoutMs = 10 * 60 * 1000;
+
+ KeyValueStore<String, StatsWindowState> statsStore;
+
+ class StatsWindowState {
+ int lastCount = 0;
+ long timeAtLastOutput = 0;
+ int newCount = 0;
+ }
+
+ @Override
+ public Collection<StatsOutput> apply(PageViewEvent message) {
+ List<StatsOutput> outputStats = new ArrayList<>();
+ long wndTimestamp = (long) Math.floor(TimeUnit.MILLISECONDS.toMinutes(message.getMessage().timestamp) / 5) * 5;
+ String wndKey = String.format("%s-%d", message.getMessage().memberId, wndTimestamp);
+ StatsWindowState curState = this.statsStore.get(wndKey);
+ curState.newCount++;
+ long curTimeMs = System.currentTimeMillis();
+ if (curState.newCount > 0 && curState.timeAtLastOutput + timeoutMs < curTimeMs) {
+ curState.timeAtLastOutput = curTimeMs;
+ curState.lastCount += curState.newCount;
+ curState.newCount = 0;
+ outputStats.add(new StatsOutput(message.getMessage().memberId, wndTimestamp, curState.lastCount));
+ }
+ // update counter w/o generating output
+ this.statsStore.put(wndKey, curState);
+ return outputStats;
+ }
+
+ @Override
+ public void init(Config config, TaskContext context) {
+ this.statsStore = (KeyValueStore<String, StatsWindowState>) context.getStore("my-stats-wnd-store");
+ }
+ }
+
+ StreamSpec input1 = new StreamSpec() {
+ @Override public SystemStream getSystemStream() {
+ return new SystemStream("kafka", "PageViewEvent");
+ }
+
+ @Override public Properties getProperties() {
+ return null;
+ }
+ };
+
+ StreamSpec output = new StreamSpec() {
+ @Override public SystemStream getSystemStream() {
+ return new SystemStream("kafka", "PageViewPerMember5min");
+ }
+
+ @Override public Properties getProperties() {
+ return null;
+ }
+ };
+
+ class PageViewEvent implements MessageEnvelope<String, PageViewEvent> {
+ String pageId;
+ String memberId;
+ long timestamp;
+
+ PageViewEvent(String pageId, String memberId, long timestamp) {
+ this.pageId = pageId;
+ this.memberId = memberId;
+ this.timestamp = timestamp;
+ }
+
+ @Override
+ public String getKey() {
+ return this.pageId;
+ }
+
+ @Override
+ public PageViewEvent getMessage() {
+ return this;
+ }
+ }
+
+ class StatsOutput implements MessageEnvelope<String, StatsOutput> {
+ private String memberId;
+ private long timestamp;
+ private Integer count;
+
+ StatsOutput(String key, long timestamp, Integer count) {
+ this.memberId = key;
+ this.timestamp = timestamp;
+ this.count = count;
+ }
+
+ @Override
+ public String getKey() {
+ return this.memberId;
+ }
+
+ @Override
+ public StatsOutput getMessage() {
+ return this;
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-core/src/test/java/org/apache/samza/example/NoContextStreamExample.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/example/NoContextStreamExample.java b/samza-core/src/test/java/org/apache/samza/example/NoContextStreamExample.java
new file mode 100644
index 0000000..c6d2e6e
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/example/NoContextStreamExample.java
@@ -0,0 +1,151 @@
+/*
+ * 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.example;
+
+import org.apache.samza.operators.*;
+import org.apache.samza.operators.StreamGraphBuilder;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.data.InputMessageEnvelope;
+import org.apache.samza.operators.data.JsonIncomingSystemMessageEnvelope;
+import org.apache.samza.operators.data.Offset;
+import org.apache.samza.operators.functions.JoinFunction;
+import org.apache.samza.serializers.JsonSerde;
+import org.apache.samza.serializers.StringSerde;
+import org.apache.samza.system.ExecutionEnvironment;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.SystemStreamPartition;
+import org.apache.samza.util.CommandLine;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+
+
+/**
+ * Example {@link StreamGraphBuilder} code to test the API methods
+ */
+public class NoContextStreamExample implements StreamGraphBuilder {
+
+ StreamSpec input1 = new StreamSpec() {
+ @Override public SystemStream getSystemStream() {
+ return new SystemStream("kafka", "input1");
+ }
+
+ @Override public Properties getProperties() {
+ return null;
+ }
+ };
+
+ StreamSpec input2 = new StreamSpec() {
+ @Override public SystemStream getSystemStream() {
+ return new SystemStream("kafka", "input2");
+ }
+
+ @Override public Properties getProperties() {
+ return null;
+ }
+ };
+
+ StreamSpec output = new StreamSpec() {
+ @Override public SystemStream getSystemStream() {
+ return new SystemStream("kafka", "output");
+ }
+
+ @Override public Properties getProperties() {
+ return null;
+ }
+ };
+
+ 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);
+ }
+ }
+
+ private JsonMessageEnvelope getInputMessage(InputMessageEnvelope ism) {
+ return new JsonMessageEnvelope(
+ ((MessageType) ism.getMessage()).joinKey,
+ (MessageType) ism.getMessage(),
+ ism.getOffset(),
+ ism.getSystemStreamPartition());
+ }
+
+ class MyJoinFunction implements JoinFunction<String, JsonMessageEnvelope, JsonMessageEnvelope, JsonIncomingSystemMessageEnvelope<MessageType>> {
+
+ @Override
+ public JsonIncomingSystemMessageEnvelope<MessageType> apply(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);
+ }
+
+ @Override
+ public String getFirstKey(JsonMessageEnvelope message) {
+ return message.getKey();
+ }
+
+ @Override
+ public String getSecondKey(JsonMessageEnvelope message) {
+ return message.getKey();
+ }
+ }
+
+ /**
+ * used by remote execution environment to launch the job in remote program. The remote program should follow the similar
+ * invoking context as in standalone:
+ *
+ * public static void main(String args[]) throws Exception {
+ * CommandLine cmdLine = new CommandLine();
+ * Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+ * ExecutionEnvironment remoteEnv = ExecutionEnvironment.fromConfig(config);
+ * remoteEnv.run(new NoContextStreamExample(), config);
+ * }
+ *
+ */
+ @Override public void init(StreamGraph graph, Config config) {
+ MessageStream<InputMessageEnvelope> inputSource1 = graph.<Object, Object, InputMessageEnvelope>createInStream(
+ input1, null, null);
+ MessageStream<InputMessageEnvelope> inputSource2 = graph.<Object, Object, InputMessageEnvelope>createInStream(
+ input2, null, null);
+ OutputStream<JsonIncomingSystemMessageEnvelope<MessageType>> outStream = graph.createOutStream(output,
+ new StringSerde("UTF-8"), new JsonSerde<>());
+
+ inputSource1.map(this::getInputMessage).
+ join(inputSource2.map(this::getInputMessage), new MyJoinFunction()).
+ sendTo(outStream);
+
+ }
+
+ // standalone local program model
+ public static void main(String[] args) throws Exception {
+ CommandLine cmdLine = new CommandLine();
+ Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+ ExecutionEnvironment standaloneEnv = ExecutionEnvironment.getLocalEnvironment(config);
+ standaloneEnv.run(new NoContextStreamExample(), config);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-core/src/test/java/org/apache/samza/example/OrderShipmentJoinExample.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/example/OrderShipmentJoinExample.java b/samza-core/src/test/java/org/apache/samza/example/OrderShipmentJoinExample.java
new file mode 100644
index 0000000..0477066
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/example/OrderShipmentJoinExample.java
@@ -0,0 +1,188 @@
+/*
+ * 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.example;
+
+import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.OutputStream;
+import org.apache.samza.operators.StreamGraphBuilder;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.operators.StreamSpec;
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.functions.JoinFunction;
+import org.apache.samza.serializers.JsonSerde;
+import org.apache.samza.serializers.StringSerde;
+import org.apache.samza.system.ExecutionEnvironment;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.util.CommandLine;
+
+import java.util.Properties;
+
+
+/**
+ * Simple 2-way stream-to-stream join example
+ */
+public class OrderShipmentJoinExample implements StreamGraphBuilder {
+
+ /**
+ * used by remote execution environment to launch the job in remote program. The remote program should follow the similar
+ * invoking context as in standalone:
+ *
+ * public static void main(String args[]) throws Exception {
+ * CommandLine cmdLine = new CommandLine();
+ * Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+ * ExecutionEnvironment remoteEnv = ExecutionEnvironment.getRemoteEnvironment(config);
+ * UserMainExample runnableApp = new UserMainExample();
+ * runnableApp.run(remoteEnv, config);
+ * }
+ *
+ */
+ @Override public void init(StreamGraph graph, Config config) {
+
+ MessageStream<OrderRecord> orders = graph.createInStream(input1, new StringSerde("UTF-8"), new JsonSerde<>());
+ MessageStream<ShipmentRecord> shipments = graph.createInStream(input2, new StringSerde("UTF-8"), new JsonSerde<>());
+ OutputStream<FulFilledOrderRecord> fulfilledOrders = graph.createOutStream(output, new StringSerde("UTF-8"), new JsonSerde<>());
+
+ orders.join(shipments, new MyJoinFunction()).sendTo(fulfilledOrders);
+
+ }
+
+ // standalone local program model
+ public static void main(String[] args) throws Exception {
+ CommandLine cmdLine = new CommandLine();
+ Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+ ExecutionEnvironment standaloneEnv = ExecutionEnvironment.getLocalEnvironment(config);
+ standaloneEnv.run(new OrderShipmentJoinExample(), config);
+ }
+
+ StreamSpec input1 = new StreamSpec() {
+ @Override public SystemStream getSystemStream() {
+ return new SystemStream("kafka", "Orders");
+ }
+
+ @Override public Properties getProperties() {
+ return null;
+ }
+ };
+
+ StreamSpec input2 = new StreamSpec() {
+ @Override public SystemStream getSystemStream() {
+ return new SystemStream("kafka", "Shipment");
+ }
+
+ @Override public Properties getProperties() {
+ return null;
+ }
+ };
+
+ StreamSpec output = new StreamSpec() {
+ @Override public SystemStream getSystemStream() {
+ return new SystemStream("kafka", "FulfilledOrders");
+ }
+
+ @Override public Properties getProperties() {
+ return null;
+ }
+ };
+
+ class OrderRecord implements MessageEnvelope<String, OrderRecord> {
+ String orderId;
+ long orderTimeMs;
+
+ OrderRecord(String orderId, long timeMs) {
+ this.orderId = orderId;
+ this.orderTimeMs = timeMs;
+ }
+
+ @Override
+ public String getKey() {
+ return this.orderId;
+ }
+
+ @Override
+ public OrderRecord getMessage() {
+ return this;
+ }
+ }
+
+ class ShipmentRecord implements MessageEnvelope<String, ShipmentRecord> {
+ String orderId;
+ long shipTimeMs;
+
+ ShipmentRecord(String orderId, long timeMs) {
+ this.orderId = orderId;
+ this.shipTimeMs = timeMs;
+ }
+
+ @Override
+ public String getKey() {
+ return this.orderId;
+ }
+
+ @Override
+ public ShipmentRecord getMessage() {
+ return this;
+ }
+ }
+
+ class FulFilledOrderRecord implements MessageEnvelope<String, FulFilledOrderRecord> {
+ String orderId;
+ long orderTimeMs;
+ long shipTimeMs;
+
+ FulFilledOrderRecord(String orderId, long orderTimeMs, long shipTimeMs) {
+ this.orderId = orderId;
+ this.orderTimeMs = orderTimeMs;
+ this.shipTimeMs = shipTimeMs;
+ }
+
+
+ @Override
+ public String getKey() {
+ return this.orderId;
+ }
+
+ @Override
+ public FulFilledOrderRecord getMessage() {
+ return this;
+ }
+ }
+
+ FulFilledOrderRecord myJoinResult(OrderRecord m1, ShipmentRecord m2) {
+ return new FulFilledOrderRecord(m1.getKey(), m1.orderTimeMs, m2.shipTimeMs);
+ }
+
+ class MyJoinFunction implements JoinFunction<String, OrderRecord, ShipmentRecord, FulFilledOrderRecord> {
+
+ @Override
+ public FulFilledOrderRecord apply(OrderRecord message, ShipmentRecord otherMessage) {
+ return OrderShipmentJoinExample.this.myJoinResult(message, otherMessage);
+ }
+
+ @Override
+ public String getFirstKey(OrderRecord message) {
+ return message.getKey();
+ }
+
+ @Override
+ public String getSecondKey(ShipmentRecord message) {
+ return message.getKey();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-core/src/test/java/org/apache/samza/example/PageViewCounterExample.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/example/PageViewCounterExample.java b/samza-core/src/test/java/org/apache/samza/example/PageViewCounterExample.java
new file mode 100644
index 0000000..f7d8bda
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/example/PageViewCounterExample.java
@@ -0,0 +1,129 @@
+/*
+ * 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.example;
+
+import org.apache.samza.operators.*;
+import org.apache.samza.operators.StreamGraphBuilder;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.triggers.Triggers;
+import org.apache.samza.operators.windows.AccumulationMode;
+import org.apache.samza.operators.windows.WindowPane;
+import org.apache.samza.operators.windows.Windows;
+import org.apache.samza.serializers.JsonSerde;
+import org.apache.samza.serializers.StringSerde;
+import org.apache.samza.system.ExecutionEnvironment;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.util.CommandLine;
+
+import java.time.Duration;
+import java.util.Properties;
+
+
+/**
+ * Example code to implement window-based counter
+ */
+public class PageViewCounterExample implements StreamGraphBuilder {
+
+ @Override public void init(StreamGraph graph, Config config) {
+
+ MessageStream<PageViewEvent> pageViewEvents = graph.createInStream(input1, new StringSerde("UTF-8"), new JsonSerde<>());
+ OutputStream<MyStreamOutput> pageViewPerMemberCounters = graph.createOutStream(output, new StringSerde("UTF-8"), new JsonSerde<>());
+
+ pageViewEvents.
+ window(Windows.<PageViewEvent, String, Integer>keyedTumblingWindow(m -> m.getMessage().memberId, Duration.ofSeconds(10), (m, c) -> c + 1).
+ setEarlyTrigger(Triggers.repeat(Triggers.count(5))).
+ setAccumulationMode(AccumulationMode.DISCARDING)).
+ map(MyStreamOutput::new).
+ sendTo(pageViewPerMemberCounters);
+
+ }
+
+ public static void main(String[] args) {
+ CommandLine cmdLine = new CommandLine();
+ Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+ ExecutionEnvironment standaloneEnv = ExecutionEnvironment.getLocalEnvironment(config);
+ standaloneEnv.run(new PageViewCounterExample(), config);
+ }
+
+ StreamSpec input1 = new StreamSpec() {
+ @Override public SystemStream getSystemStream() {
+ return new SystemStream("kafka", "PageViewEvent");
+ }
+
+ @Override public Properties getProperties() {
+ return null;
+ }
+ };
+
+ StreamSpec output = new StreamSpec() {
+ @Override public SystemStream getSystemStream() {
+ return new SystemStream("kafka", "PageViewPerMember5min");
+ }
+
+ @Override public Properties getProperties() {
+ return null;
+ }
+ };
+
+ class PageViewEvent implements MessageEnvelope<String, PageViewEvent> {
+ String pageId;
+ String memberId;
+ long timestamp;
+
+ PageViewEvent(String pageId, String memberId, long timestamp) {
+ this.pageId = pageId;
+ this.memberId = memberId;
+ this.timestamp = timestamp;
+ }
+
+ @Override
+ public String getKey() {
+ return this.pageId;
+ }
+
+ @Override
+ public PageViewEvent getMessage() {
+ return this;
+ }
+ }
+
+ class MyStreamOutput implements MessageEnvelope<String, MyStreamOutput> {
+ String memberId;
+ long timestamp;
+ int count;
+
+ MyStreamOutput(WindowPane<String, Integer> m) {
+ this.memberId = m.getKey().getKey();
+ this.timestamp = Long.valueOf(m.getKey().getPaneId());
+ this.count = m.getMessage();
+ }
+
+ @Override
+ public String getKey() {
+ return this.memberId;
+ }
+
+ @Override
+ public MyStreamOutput getMessage() {
+ return this;
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-core/src/test/java/org/apache/samza/example/RepartitionExample.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/example/RepartitionExample.java b/samza-core/src/test/java/org/apache/samza/example/RepartitionExample.java
new file mode 100644
index 0000000..6994ac4
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/example/RepartitionExample.java
@@ -0,0 +1,140 @@
+/*
+ * 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.example;
+
+import org.apache.samza.operators.*;
+import org.apache.samza.operators.StreamGraphBuilder;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.windows.WindowPane;
+import org.apache.samza.operators.windows.Windows;
+import org.apache.samza.serializers.JsonSerde;
+import org.apache.samza.serializers.StringSerde;
+import org.apache.samza.system.ExecutionEnvironment;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.util.CommandLine;
+
+import java.time.Duration;
+import java.util.*;
+
+
+/**
+ * Example {@link StreamGraphBuilder} code to test the API methods with re-partition operator
+ */
+public class RepartitionExample implements StreamGraphBuilder {
+
+ /**
+ * used by remote execution environment to launch the job in remote program. The remote program should follow the similar
+ * invoking context as in standalone:
+ *
+ * public static void main(String args[]) throws Exception {
+ * CommandLine cmdLine = new CommandLine();
+ * Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+ * ExecutionEnvironment remoteEnv = ExecutionEnvironment.getRemoteEnvironment(config);
+ * remoteEnv.run(new UserMainExample(), config);
+ * }
+ *
+ */
+ @Override public void init(StreamGraph graph, Config config) {
+
+ MessageStream<PageViewEvent> pageViewEvents = graph.createInStream(input1, new StringSerde("UTF-8"), new JsonSerde<>());
+ OutputStream<MyStreamOutput> pageViewPerMemberCounters = graph.createOutStream(output, new StringSerde("UTF-8"), new JsonSerde<>());
+
+ pageViewEvents.
+ partitionBy(m -> m.getMessage().memberId).
+ window(Windows.<PageViewEvent, String, Integer>keyedTumblingWindow(
+ msg -> msg.getMessage().memberId, Duration.ofMinutes(5), (m, c) -> c + 1)).
+ map(MyStreamOutput::new).
+ sendTo(pageViewPerMemberCounters);
+
+ }
+
+ // standalone local program model
+ public static void main(String[] args) throws Exception {
+ CommandLine cmdLine = new CommandLine();
+ Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
+ ExecutionEnvironment standaloneEnv = ExecutionEnvironment.getLocalEnvironment(config);
+ standaloneEnv.run(new RepartitionExample(), config);
+ }
+
+ StreamSpec input1 = new StreamSpec() {
+ @Override public SystemStream getSystemStream() {
+ return new SystemStream("kafka", "PageViewEvent");
+ }
+
+ @Override public Properties getProperties() {
+ return null;
+ }
+ };
+
+ StreamSpec output = new StreamSpec() {
+ @Override public SystemStream getSystemStream() {
+ return new SystemStream("kafka", "PageViewPerMember5min");
+ }
+
+ @Override public Properties getProperties() {
+ return null;
+ }
+ };
+
+ class PageViewEvent implements MessageEnvelope<String, PageViewEvent> {
+ String pageId;
+ String memberId;
+ long timestamp;
+
+ PageViewEvent(String pageId, String memberId, long timestamp) {
+ this.pageId = pageId;
+ this.memberId = memberId;
+ this.timestamp = timestamp;
+ }
+
+ @Override
+ public String getKey() {
+ return this.pageId;
+ }
+
+ @Override
+ public PageViewEvent getMessage() {
+ return this;
+ }
+ }
+
+ class MyStreamOutput implements MessageEnvelope<String, MyStreamOutput> {
+ String memberId;
+ long timestamp;
+ int count;
+
+ MyStreamOutput(WindowPane<String, Integer> m) {
+ this.memberId = m.getKey().getKey();
+ this.timestamp = Long.valueOf(m.getKey().getPaneId());
+ this.count = m.getMessage();
+ }
+
+ @Override
+ public String getKey() {
+ return this.memberId;
+ }
+
+ @Override
+ public MyStreamOutput getMessage() {
+ return this;
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-core/src/test/java/org/apache/samza/example/TestBasicStreamGraphs.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/example/TestBasicStreamGraphs.java b/samza-core/src/test/java/org/apache/samza/example/TestBasicStreamGraphs.java
new file mode 100644
index 0000000..8ecd44f
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/example/TestBasicStreamGraphs.java
@@ -0,0 +1,99 @@
+/*
+ * 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.example;
+
+import java.lang.reflect.Field;
+import org.apache.samza.Partition;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.impl.OperatorGraph;
+import org.apache.samza.system.SystemStreamPartition;
+import org.apache.samza.task.StreamOperatorTask;
+import org.apache.samza.task.TaskContext;
+import org.junit.Test;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+
+/**
+ * Unit test for {@link StreamOperatorTask}
+ */
+public class TestBasicStreamGraphs {
+
+ private final Set<SystemStreamPartition> inputPartitions = new HashSet<SystemStreamPartition>() { {
+ for (int i = 0; i < 4; i++) {
+ this.add(new SystemStreamPartition("my-system", String.format("my-topic%d", i), 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);
+ TestWindowExample userTask = new TestWindowExample(this.inputPartitions);
+ StreamOperatorTask adaptorTask = new StreamOperatorTask(userTask);
+ Field pipelineMapFld = StreamOperatorTask.class.getDeclaredField("operatorGraph");
+ pipelineMapFld.setAccessible(true);
+ OperatorGraph opGraph = (OperatorGraph) pipelineMapFld.get(adaptorTask);
+
+ adaptorTask.init(mockConfig, mockContext);
+ this.inputPartitions.forEach(partition -> {
+ assertNotNull(opGraph.get(partition.getSystemStream()));
+ });
+ }
+
+ @Test
+ public void testSplitTask() throws Exception {
+ Config mockConfig = mock(Config.class);
+ TaskContext mockContext = mock(TaskContext.class);
+ when(mockContext.getSystemStreamPartitions()).thenReturn(this.inputPartitions);
+ TestBroadcastExample splitTask = new TestBroadcastExample(this.inputPartitions);
+ StreamOperatorTask adaptorTask = new StreamOperatorTask(splitTask);
+ Field pipelineMapFld = StreamOperatorTask.class.getDeclaredField("operatorGraph");
+ pipelineMapFld.setAccessible(true);
+ OperatorGraph opGraph = (OperatorGraph) pipelineMapFld.get(adaptorTask);
+
+ adaptorTask.init(mockConfig, mockContext);
+ this.inputPartitions.forEach(partition -> {
+ assertNotNull(opGraph.get(partition.getSystemStream()));
+ });
+ }
+
+ @Test
+ public void testJoinTask() throws Exception {
+ Config mockConfig = mock(Config.class);
+ TaskContext mockContext = mock(TaskContext.class);
+ when(mockContext.getSystemStreamPartitions()).thenReturn(this.inputPartitions);
+ TestJoinExample joinTask = new TestJoinExample(this.inputPartitions);
+ StreamOperatorTask adaptorTask = new StreamOperatorTask(joinTask);
+ Field pipelineMapFld = StreamOperatorTask.class.getDeclaredField("operatorGraph");
+ pipelineMapFld.setAccessible(true);
+ OperatorGraph opGraph = (OperatorGraph) pipelineMapFld.get(adaptorTask);
+
+ adaptorTask.init(mockConfig, mockContext);
+ this.inputPartitions.forEach(partition -> {
+ assertNotNull(opGraph.get(partition.getSystemStream()));
+ });
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-core/src/test/java/org/apache/samza/example/TestBroadcastExample.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/example/TestBroadcastExample.java b/samza-core/src/test/java/org/apache/samza/example/TestBroadcastExample.java
new file mode 100644
index 0000000..d22324b
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/example/TestBroadcastExample.java
@@ -0,0 +1,113 @@
+/*
+ * 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.example;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.operators.StreamSpec;
+import org.apache.samza.operators.data.InputMessageEnvelope;
+import org.apache.samza.operators.data.JsonIncomingSystemMessageEnvelope;
+import org.apache.samza.operators.data.Offset;
+import org.apache.samza.operators.triggers.Triggers;
+import org.apache.samza.operators.windows.Windows;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.SystemStreamPartition;
+
+import java.time.Duration;
+import java.util.function.BiFunction;
+import java.util.Properties;
+import java.util.Set;
+
+
+/**
+ * Example implementation of split stream tasks
+ *
+ */
+public class TestBroadcastExample extends TestExampleBase {
+
+ TestBroadcastExample(Set<SystemStreamPartition> inputs) {
+ super(inputs);
+ }
+
+ 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 init(StreamGraph graph, Config config) {
+ BiFunction<JsonMessageEnvelope, Integer, Integer> sumAggregator = (m, c) -> c + 1;
+ inputs.keySet().forEach(entry -> {
+ MessageStream<JsonMessageEnvelope> inputStream = graph.<Object, Object, InputMessageEnvelope>createInStream(new StreamSpec() {
+ @Override public SystemStream getSystemStream() {
+ return entry;
+ }
+
+ @Override public Properties getProperties() {
+ return null;
+ }
+ }, null, null).map(this::getInputMessage);
+
+ inputStream.filter(this::myFilter1).window(Windows.tumblingWindow(Duration.ofMillis(100), sumAggregator)
+ .setLateTrigger(Triggers.any(Triggers.count(30000), Triggers.timeSinceFirstMessage(Duration.ofMillis(10)))));
+
+ inputStream.filter(this::myFilter2).window(Windows.tumblingWindow(Duration.ofMillis(100), sumAggregator)
+ .setLateTrigger(Triggers.any(Triggers.count(30000), Triggers.timeSinceFirstMessage(Duration.ofMillis(10)))));
+
+ inputStream.filter(this::myFilter3).window(Windows.tumblingWindow(Duration.ofMillis(100), sumAggregator)
+ .setLateTrigger(Triggers.any(Triggers.count(30000), Triggers.timeSinceFirstMessage(Duration.ofMillis(10)))));
+
+ });
+ }
+
+ JsonMessageEnvelope getInputMessage(InputMessageEnvelope 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/8515448a/samza-core/src/test/java/org/apache/samza/example/TestExampleBase.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/example/TestExampleBase.java b/samza-core/src/test/java/org/apache/samza/example/TestExampleBase.java
new file mode 100644
index 0000000..c4df9d4
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/example/TestExampleBase.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.example;
+
+import org.apache.samza.operators.StreamGraphBuilder;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.SystemStreamPartition;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Base class for test examples
+ *
+ */
+public abstract class TestExampleBase implements StreamGraphBuilder {
+
+ protected final Map<SystemStream, Set<SystemStreamPartition>> inputs;
+
+ TestExampleBase(Set<SystemStreamPartition> inputs) {
+ this.inputs = new HashMap<>();
+ for (SystemStreamPartition input : inputs) {
+ this.inputs.putIfAbsent(input.getSystemStream(), new HashSet<>());
+ this.inputs.get(input.getSystemStream()).add(input);
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-core/src/test/java/org/apache/samza/example/TestJoinExample.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/example/TestJoinExample.java b/samza-core/src/test/java/org/apache/samza/example/TestJoinExample.java
new file mode 100644
index 0000000..fe6e7e7
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/example/TestJoinExample.java
@@ -0,0 +1,129 @@
+/*
+ * 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.example;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.operators.StreamSpec;
+import org.apache.samza.operators.data.InputMessageEnvelope;
+import org.apache.samza.operators.data.JsonIncomingSystemMessageEnvelope;
+import org.apache.samza.operators.data.Offset;
+import org.apache.samza.operators.functions.JoinFunction;
+import org.apache.samza.serializers.JsonSerde;
+import org.apache.samza.serializers.StringSerde;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.SystemStreamPartition;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+
+
+/**
+ * Example implementation of unique key-based stream-stream join tasks
+ *
+ */
+public class TestJoinExample extends TestExampleBase {
+
+ TestJoinExample(Set<SystemStreamPartition> inputs) {
+ super(inputs);
+ }
+
+ 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 init(StreamGraph graph, Config config) {
+
+ for (SystemStream input : inputs.keySet()) {
+ MessageStream<JsonMessageEnvelope> newSource = graph.<Object, Object, InputMessageEnvelope>createInStream(
+ new StreamSpec() {
+ @Override public SystemStream getSystemStream() {
+ return input;
+ }
+
+ @Override public Properties getProperties() {
+ return null;
+ }
+ }, null, null).map(this::getInputMessage);
+ if (joinOutput == null) {
+ joinOutput = newSource;
+ } else {
+ joinOutput = joinOutput.join(newSource, new MyJoinFunction());
+ }
+ }
+
+ joinOutput.sendTo(graph.createOutStream(new StreamSpec() {
+ @Override public SystemStream getSystemStream() {
+ return null;
+ }
+
+ @Override public Properties getProperties() {
+ return null;
+ }
+ }, new StringSerde("UTF-8"), new JsonSerde<>()));
+
+ }
+
+ private JsonMessageEnvelope getInputMessage(InputMessageEnvelope ism) {
+ return new JsonMessageEnvelope(
+ ((MessageType) ism.getMessage()).joinKey,
+ (MessageType) ism.getMessage(),
+ ism.getOffset(),
+ ism.getSystemStreamPartition());
+ }
+
+ class MyJoinFunction implements JoinFunction<String, JsonMessageEnvelope, JsonMessageEnvelope, JsonMessageEnvelope> {
+ 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);
+ }
+
+ @Override
+ public JsonMessageEnvelope apply(JsonMessageEnvelope message, JsonMessageEnvelope otherMessage) {
+ return this.myJoinResult(message, otherMessage);
+ }
+
+ @Override
+ public String getFirstKey(JsonMessageEnvelope message) {
+ return message.getKey();
+ }
+
+ @Override
+ public String getSecondKey(JsonMessageEnvelope message) {
+ return message.getKey();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-core/src/test/java/org/apache/samza/example/TestWindowExample.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/example/TestWindowExample.java b/samza-core/src/test/java/org/apache/samza/example/TestWindowExample.java
new file mode 100644
index 0000000..e08ca20
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/example/TestWindowExample.java
@@ -0,0 +1,81 @@
+/*
+ * 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.example;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.operators.StreamSpec;
+import org.apache.samza.operators.data.InputMessageEnvelope;
+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.Windows;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.SystemStreamPartition;
+
+import java.time.Duration;
+import java.util.function.BiFunction;
+import java.util.Properties;
+import java.util.Set;
+
+
+/**
+ * Example implementation of a simple user-defined tasks w/ window operators
+ *
+ */
+public class TestWindowExample extends TestExampleBase {
+ class MessageType {
+ String field1;
+ String field2;
+ }
+
+ TestWindowExample(Set<SystemStreamPartition> inputs) {
+ super(inputs);
+ }
+
+ class JsonMessageEnvelope extends JsonIncomingSystemMessageEnvelope<MessageType> {
+
+ JsonMessageEnvelope(String key, MessageType data, Offset offset, SystemStreamPartition partition) {
+ super(key, data, offset, partition);
+ }
+ }
+
+ @Override
+ public void init(StreamGraph graph, Config config) {
+ BiFunction<JsonMessageEnvelope, Integer, Integer> maxAggregator = (m, c) -> c + 1;
+ inputs.keySet().forEach(source -> graph.<Object, Object, InputMessageEnvelope>createInStream(new StreamSpec() {
+ @Override public SystemStream getSystemStream() {
+ return source;
+ }
+
+ @Override public Properties getProperties() {
+ return null;
+ }
+ }, null, null).
+ map(m1 -> new JsonMessageEnvelope(this.myMessageKeyFunction(m1), (MessageType) m1.getMessage(), m1.getOffset(),
+ m1.getSystemStreamPartition())).window(Windows.tumblingWindow(Duration.ofMillis(200), maxAggregator)));
+
+ }
+
+ String myMessageKeyFunction(MessageEnvelope<Object, Object> m) {
+ return m.getKey().toString();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-core/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java b/samza-core/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java
new file mode 100644
index 0000000..160a47a
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java
@@ -0,0 +1,204 @@
+/*
+ * 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.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 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 TestMessageStreamImpl {
+
+ private StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+
+ @Test
+ public void testMap() {
+ MessageStreamImpl<TestMessageEnvelope> inputStream = new MessageStreamImpl<>(mockGraph);
+ MapFunction<TestMessageEnvelope, TestOutputMessageEnvelope> xMap = (TestMessageEnvelope 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.getNextStream(), 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<>(mockGraph);
+ 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 = (TestMessageEnvelope message) -> 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.getNextStream(), 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<>(mockGraph);
+ FilterFunction<TestMessageEnvelope> xFilter = (TestMessageEnvelope 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.getNextStream(), 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<>(mockGraph);
+ SinkFunction<TestMessageEnvelope> xSink = (TestMessageEnvelope m, MessageCollector mc, TaskCoordinator 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).getNextStream());
+ }
+
+ @Test
+ public void testJoin() {
+ MessageStreamImpl<TestMessageEnvelope> source1 = new MessageStreamImpl<>(mockGraph);
+ MessageStreamImpl<TestMessageEnvelope> source2 = new MessageStreamImpl<>(mockGraph);
+ JoinFunction<String, TestMessageEnvelope, TestMessageEnvelope, TestOutputMessageEnvelope> joiner =
+ new JoinFunction<String, TestMessageEnvelope, TestMessageEnvelope, TestOutputMessageEnvelope>() {
+ @Override
+ public TestOutputMessageEnvelope apply(TestMessageEnvelope m1, TestMessageEnvelope m2) {
+ return new TestOutputMessageEnvelope(m1.getKey(), m1.getMessage().getValue().length() + m2.getMessage().getValue().length());
+ }
+
+ @Override
+ public String getFirstKey(TestMessageEnvelope message) {
+ return message.getKey();
+ }
+
+ @Override
+ public String getSecondKey(TestMessageEnvelope message) {
+ return message.getKey();
+ }
+ };
+
+ 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).getNextStream(), joinOutput);
+ subs = source2.getRegisteredOperatorSpecs();
+ assertEquals(subs.size(), 1);
+ OperatorSpec<TestMessageEnvelope> joinOp2 = subs.iterator().next();
+ assertTrue(joinOp2 instanceof PartialJoinOperatorSpec);
+ assertEquals(((PartialJoinOperatorSpec) joinOp2).getNextStream(), 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<>(mockGraph);
+ Collection<MessageStream<TestMessageEnvelope>> others = new ArrayList<MessageStream<TestMessageEnvelope>>() { {
+ this.add(new MessageStreamImpl<>(mockGraph));
+ this.add(new MessageStreamImpl<>(mockGraph));
+ } };
+ 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).getNextStream(), 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/8515448a/samza-core/src/test/java/org/apache/samza/operators/TestMessageStreamImplUtil.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/operators/TestMessageStreamImplUtil.java b/samza-core/src/test/java/org/apache/samza/operators/TestMessageStreamImplUtil.java
new file mode 100644
index 0000000..c4e9f51
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/operators/TestMessageStreamImplUtil.java
@@ -0,0 +1,26 @@
+/*
+ * 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;
+
+
+public class TestMessageStreamImplUtil {
+ public static <M> MessageStreamImpl<M> getMessageStreamImpl(StreamGraphImpl graph) {
+ return new MessageStreamImpl<M>(graph);
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-core/src/test/java/org/apache/samza/operators/data/JsonIncomingSystemMessageEnvelope.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/operators/data/JsonIncomingSystemMessageEnvelope.java b/samza-core/src/test/java/org/apache/samza/operators/data/JsonIncomingSystemMessageEnvelope.java
new file mode 100644
index 0000000..9a425d1
--- /dev/null
+++ b/samza-core/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/8515448a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java
new file mode 100644
index 0000000..361972e
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java
@@ -0,0 +1,73 @@
+/*
+ * 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.task.MessageCollector;
+import org.apache.samza.task.TaskCoordinator;
+import org.hamcrest.core.IsEqual;
+import org.junit.Test;
+
+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;
+
+
+public class TestOperatorImpl {
+
+ TestMessageEnvelope curInputMsg;
+ MessageCollector curCollector;
+ TaskCoordinator curCoordinator;
+
+ @Test
+ public void testSubscribers() {
+ this.curInputMsg = null;
+ this.curCollector = null;
+ this.curCoordinator = null;
+ 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 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.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
+ TestMessageEnvelope mockInput = mock(TestMessageEnvelope.class);
+ opImpl.onNext(mockInput, mockCollector, mockCoordinator);
+ assertEquals(mockInput, this.curInputMsg);
+ assertEquals(mockCollector, this.curCollector);
+ assertEquals(mockCoordinator, this.curCoordinator);
+ }
+}
[11/13] samza git commit: SAMZA-1099: Documentation updates for Samza
0.12 release (for master branch)
Posted by jm...@apache.org.
SAMZA-1099: Documentation updates for Samza 0.12 release (for master branch)
Project: http://git-wip-us.apache.org/repos/asf/samza/repo
Commit: http://git-wip-us.apache.org/repos/asf/samza/commit/b5ea8776
Tree: http://git-wip-us.apache.org/repos/asf/samza/tree/b5ea8776
Diff: http://git-wip-us.apache.org/repos/asf/samza/diff/b5ea8776
Branch: refs/heads/samza-fluent-api-v1
Commit: b5ea8776ee5565cf12ef4bd4a0b1a0fde11154a4
Parents: eab1800
Author: vjagadish1989 <jv...@linkedin.com>
Authored: Wed Feb 22 10:53:12 2017 -0800
Committer: vjagadish1989 <jv...@linkedin.com>
Committed: Wed Feb 22 12:45:13 2017 -0800
----------------------------------------------------------------------
docs/_config.yml | 2 +-
docs/_layouts/default.html | 1 +
docs/archive/index.html | 8 ++++
.../versioned/jobs/split-deployment.md | 4 +-
.../versioned/deploy-samza-job-from-hdfs.md | 2 +-
.../tutorials/versioned/deploy-samza-to-CDH.md | 4 +-
.../versioned/remote-debugging-samza.md | 2 +-
.../versioned/run-in-multi-node-yarn.md | 16 ++++----
.../versioned/samza-rest-getting-started.md | 4 +-
docs/startup/download/index.md | 39 +++++++++++++-------
docs/startup/hello-samza/versioned/index.md | 2 +-
11 files changed, 52 insertions(+), 32 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/samza/blob/b5ea8776/docs/_config.yml
----------------------------------------------------------------------
diff --git a/docs/_config.yml b/docs/_config.yml
index 2b5bf05..29e3d42 100644
--- a/docs/_config.yml
+++ b/docs/_config.yml
@@ -25,4 +25,4 @@ exclude: [_docs]
baseurl: http://samza.apache.org
version: latest
# this is the version you will go if you click 'switch version' in "latest" pages.
-latest-release: '0.11'
+latest-release: '0.12'
http://git-wip-us.apache.org/repos/asf/samza/blob/b5ea8776/docs/_layouts/default.html
----------------------------------------------------------------------
diff --git a/docs/_layouts/default.html b/docs/_layouts/default.html
index dfcb6d0..b9a05fa 100644
--- a/docs/_layouts/default.html
+++ b/docs/_layouts/default.html
@@ -102,6 +102,7 @@
<h1><i class="fa fa-history"></i> Archive</h1>
<ul>
<li><a href="/archive/index.html#latest">latest</a></li>
+ <li><a href="/archive/index.html#12">0.12</a></li>
<li><a href="/archive/index.html#11">0.11</a></li>
<li><a href="/archive/index.html#10">0.10</a></li>
<li><a href="/archive/index.html#09">0.9</a></li>
http://git-wip-us.apache.org/repos/asf/samza/blob/b5ea8776/docs/archive/index.html
----------------------------------------------------------------------
diff --git a/docs/archive/index.html b/docs/archive/index.html
index e92f50e..549495b 100644
--- a/docs/archive/index.html
+++ b/docs/archive/index.html
@@ -27,6 +27,14 @@ title: Documentation
<li><a href="../startup/hello-samza/latest">Hello Samza</a></li>
</ul>
+<h4 id="12">0.12 Release</h4>
+
+<ul class="documentation-list">
+ <li><a href="../learn/documentation/0.12">Documentation</a></li>
+ <li><a href="../learn/tutorials/0.12">Tutorials</a></li>
+ <li><a href="../startup/hello-samza/0.12">Hello Samza</a></li>
+</ul>
+
<h4 id="11">0.11 Release</h4>
<ul class="documentation-list">
http://git-wip-us.apache.org/repos/asf/samza/blob/b5ea8776/docs/learn/documentation/versioned/jobs/split-deployment.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/versioned/jobs/split-deployment.md b/docs/learn/documentation/versioned/jobs/split-deployment.md
index ebab670..fa3e7ae 100644
--- a/docs/learn/documentation/versioned/jobs/split-deployment.md
+++ b/docs/learn/documentation/versioned/jobs/split-deployment.md
@@ -50,10 +50,10 @@ Each deployment will now consist of two separate packages:<p>
To run a job in split deployment mode:
1. **Deploy the framework**:
-The Samza framework package should be deployed to ALL the machines of a cluster into a predefined, fixed location. This could be done by merely copying the jars, or creating a meta package that would deploy all of them. Let's assume that 'samza-framework' package is installed into the '/.../samza-fwk/0.11.0' directory.
+The Samza framework package should be deployed to ALL the machines of a cluster into a predefined, fixed location. This could be done by merely copying the jars, or creating a meta package that would deploy all of them. Let's assume that 'samza-framework' package is installed into the '/.../samza-fwk/0.12.0' directory.
2. **Create symbolic link**:
-A symbolic link needs to be created for the **stable** version of the framework to point to the framework location, e.g.: {% highlight bash %} ln -s /.../samza-fwk/0.11.0 /.../samza-fwk/STABLE' {% endhighlight %}
+A symbolic link needs to be created for the **stable** version of the framework to point to the framework location, e.g.: {% highlight bash %} ln -s /.../samza-fwk/0.12.0 /.../samza-fwk/STABLE' {% endhighlight %}
3. **Deploy user job**:
In the job's config, the following property is required to enable split deployment, e.g. for Samza framework path at '/.../samza-fwk': {% highlight jproperties %} samza.fwk.path=/.../samza-fwk {% endhighlight %} By default Samza will look for the **stable** link inside the folder to find the framework. You can also override the version by configuring: {% highlight jproperties %} samza.fwk.version=0.11.1 {% endhighlight %} In this case Samza will pick '/.../samza-fwk/0.11.1' as the framework location. This way users can perform canary, upgrade and rollback their jobs easily by changing version in the config.
http://git-wip-us.apache.org/repos/asf/samza/blob/b5ea8776/docs/learn/tutorials/versioned/deploy-samza-job-from-hdfs.md
----------------------------------------------------------------------
diff --git a/docs/learn/tutorials/versioned/deploy-samza-job-from-hdfs.md b/docs/learn/tutorials/versioned/deploy-samza-job-from-hdfs.md
index 6f7dcc1..68b7658 100644
--- a/docs/learn/tutorials/versioned/deploy-samza-job-from-hdfs.md
+++ b/docs/learn/tutorials/versioned/deploy-samza-job-from-hdfs.md
@@ -24,7 +24,7 @@ This tutorial uses [hello-samza](../../../startup/hello-samza/{{site.version}}/)
### Upload the package
{% highlight bash %}
-hadoop fs -put ./target/hello-samza-0.11.0-dist.tar.gz /path/for/tgz
+hadoop fs -put ./target/hello-samza-0.12.0-dist.tar.gz /path/for/tgz
{% endhighlight %}
### Add HDFS configuration
http://git-wip-us.apache.org/repos/asf/samza/blob/b5ea8776/docs/learn/tutorials/versioned/deploy-samza-to-CDH.md
----------------------------------------------------------------------
diff --git a/docs/learn/tutorials/versioned/deploy-samza-to-CDH.md b/docs/learn/tutorials/versioned/deploy-samza-to-CDH.md
index fff209f..b764a57 100644
--- a/docs/learn/tutorials/versioned/deploy-samza-to-CDH.md
+++ b/docs/learn/tutorials/versioned/deploy-samza-to-CDH.md
@@ -34,7 +34,7 @@ mvn clean package -Dhadoop.version=cdh5.4.0
There are a few ways of uploading the package to the cluster's HDFS. If you do not have the job package in your cluster, **scp** from you local machine to the cluster. Then run
{% highlight bash %}
-hadoop fs -put path/to/hello-samza-0.11.0-dist.tar.gz /path/for/tgz
+hadoop fs -put path/to/hello-samza-0.12.0-dist.tar.gz /path/for/tgz
{% endhighlight %}
### Get Deploying Scripts
@@ -42,7 +42,7 @@ hadoop fs -put path/to/hello-samza-0.11.0-dist.tar.gz /path/for/tgz
Untar the job package (assume you will run from the current directory)
{% highlight bash %}
-tar -xvf path/to/samza-job-package-0.11.0-dist.tar.gz -C ./
+tar -xvf path/to/samza-job-package-0.12.0-dist.tar.gz -C ./
{% endhighlight %}
### Add Package Path to Properties File
http://git-wip-us.apache.org/repos/asf/samza/blob/b5ea8776/docs/learn/tutorials/versioned/remote-debugging-samza.md
----------------------------------------------------------------------
diff --git a/docs/learn/tutorials/versioned/remote-debugging-samza.md b/docs/learn/tutorials/versioned/remote-debugging-samza.md
index 7cc3a0e..3d8f069 100644
--- a/docs/learn/tutorials/versioned/remote-debugging-samza.md
+++ b/docs/learn/tutorials/versioned/remote-debugging-samza.md
@@ -83,7 +83,7 @@ Once the grid starts, you can start the wikipedia-feed Samza job.
{% highlight bash %}
mvn clean package
mkdir -p deploy/samza
-tar -xvf ./target/hello-samza-0.11.0-dist.tar.gz -C deploy/samza
+tar -xvf ./target/hello-samza-0.12.0-dist.tar.gz -C deploy/samza
deploy/samza/bin/run-job.sh --config-factory=org.apache.samza.config.factories.PropertiesConfigFactory --config-path=file://$PWD/deploy/samza/config/wikipedia-feed.properties
{% endhighlight %}
http://git-wip-us.apache.org/repos/asf/samza/blob/b5ea8776/docs/learn/tutorials/versioned/run-in-multi-node-yarn.md
----------------------------------------------------------------------
diff --git a/docs/learn/tutorials/versioned/run-in-multi-node-yarn.md b/docs/learn/tutorials/versioned/run-in-multi-node-yarn.md
index 7e7ba8d..e36f438 100644
--- a/docs/learn/tutorials/versioned/run-in-multi-node-yarn.md
+++ b/docs/learn/tutorials/versioned/run-in-multi-node-yarn.md
@@ -74,18 +74,18 @@ The goal of these steps is to configure YARN to read http filesystem because we
{% highlight bash %}
cd /tmp
-curl http://www.scala-lang.org/files/archive/scala-2.10.4.tgz > scala-2.10.4.tgz
-tar -xvf scala-2.10.4.tgz
+curl http://www.scala-lang.org/files/archive/scala-2.11.8.tgz > scala-2.11.8.tgz
+tar -xvf scala-2.11.8.tgz
{% endhighlight %}
5\. Add Scala, its log jars, and Samza's HttpFileSystem implementation.
{% highlight bash %}
-cp /tmp/scala-2.10.4/lib/scala-compiler.jar $HADOOP_YARN_HOME/share/hadoop/hdfs/lib
-cp /tmp/scala-2.10.4/lib/scala-library.jar $HADOOP_YARN_HOME/share/hadoop/hdfs/lib
+cp /tmp/scala-2.11.8/lib/scala-compiler.jar $HADOOP_YARN_HOME/share/hadoop/hdfs/lib
+cp /tmp/scala-2.11.8/lib/scala-library.jar $HADOOP_YARN_HOME/share/hadoop/hdfs/lib
curl -L http://search.maven.org/remotecontent?filepath=org/clapper/grizzled-slf4j_2.10/1.0.1/grizzled-slf4j_2.10-1.0.1.jar > $HADOOP_YARN_HOME/share/hadoop/hdfs/lib/grizzled-slf4j_2.10-1.0.1.jar
-curl -L http://search.maven.org/remotecontent?filepath=org/apache/samza/samza-yarn_2.10/0.11.0/samza-yarn_2.10-0.11.0.jar > $HADOOP_YARN_HOME/share/hadoop/hdfs/lib/samza-yarn_2.10-0.11.0.jar
-curl -L http://search.maven.org/remotecontent?filepath=org/apache/samza/samza-core_2.10/0.11.0/samza-core_2.10-0.11.0.jar > $HADOOP_YARN_HOME/share/hadoop/hdfs/lib/samza-core_2.10-0.11.0.jar
+curl -L http://search.maven.org/remotecontent?filepath=org/apache/samza/samza-yarn_2.11/0.12.0/samza-yarn_2.11-0.12.0.jar > $HADOOP_YARN_HOME/share/hadoop/hdfs/lib/samza-yarn_2.11-0.12.0.jar
+curl -L http://search.maven.org/remotecontent?filepath=org/apache/samza/samza-core_2.11/0.12.0/samza-core_2.11-0.12.0.jar > $HADOOP_YARN_HOME/share/hadoop/hdfs/lib/samza-core_2.11-0.12.0.jar
{% endhighlight %}
6\. Add http configuration in core-site.xml (create the core-site.xml file and add content).
@@ -145,7 +145,7 @@ vi src/main/config/wikipedia-feed.properties
Change the yarn.package.path property to be:
{% highlight jproperties %}
-yarn.package.path=http://yourHostname:8000/target/hello-samza-0.11.0-dist.tar.gz
+yarn.package.path=http://yourHostname:8000/target/hello-samza-0.12.0-dist.tar.gz
{% endhighlight %}
3\. Compile hello-samza.
@@ -153,7 +153,7 @@ yarn.package.path=http://yourHostname:8000/target/hello-samza-0.11.0-dist.tar.gz
{% highlight bash %}
mvn clean package
mkdir -p deploy/samza
-tar -xvf ./target/hello-samza-0.11.0-dist.tar.gz -C deploy/samza
+tar -xvf ./target/hello-samza-0.12.0-dist.tar.gz -C deploy/samza
{% endhighlight %}
4\. Deploy Samza job package to Http server..
http://git-wip-us.apache.org/repos/asf/samza/blob/b5ea8776/docs/learn/tutorials/versioned/samza-rest-getting-started.md
----------------------------------------------------------------------
diff --git a/docs/learn/tutorials/versioned/samza-rest-getting-started.md b/docs/learn/tutorials/versioned/samza-rest-getting-started.md
index c0e1cf5..91f3dd6 100644
--- a/docs/learn/tutorials/versioned/samza-rest-getting-started.md
+++ b/docs/learn/tutorials/versioned/samza-rest-getting-started.md
@@ -31,7 +31,7 @@ Take note of the path where you cloned hello-samza. You will need this to config
#### Build the Samza REST Service package
-The source code for Samza REST is in the samza-rest module of the Samza repository. It To build it, execute the following gradle task from the root of the project.
+The source code for Samza REST is in the samza-rest module of the Samza repository. To build it, execute the following gradle task from the root of the project.
{% highlight bash %}
./gradlew samza-rest:clean releaseRestServiceTar
{% endhighlight %}
@@ -48,7 +48,7 @@ Run the following commands:
{% highlight bash %}
cd samza-rest/build/distributions/
mkdir -p deploy/samza-rest
-tar -xvf ./samza-rest-0.11.1-SNAPSHOT.tgz -C deploy/samza-rest
+tar -xvf ./samza-rest-0.13.0-SNAPSHOT.tgz -C deploy/samza-rest
{% endhighlight %}
#### Configure the Installations Path
http://git-wip-us.apache.org/repos/asf/samza/blob/b5ea8776/docs/startup/download/index.md
----------------------------------------------------------------------
diff --git a/docs/startup/download/index.md b/docs/startup/download/index.md
index 7dfdfda..53fa299 100644
--- a/docs/startup/download/index.md
+++ b/docs/startup/download/index.md
@@ -29,6 +29,7 @@ Starting from 2016, Samza will begin requiring JDK8 or higher. Please see [this
### Source Releases
+ * [samza-sources-0.12.0.tgz](http://www.apache.org/dyn/closer.lua/samza/0.12.0)
* [samza-sources-0.11.0.tgz](http://www.apache.org/dyn/closer.lua/samza/0.11.0)
* [samza-sources-0.10.1.tgz](http://www.apache.org/dyn/closer.lua/samza/0.10.1)
* [samza-sources-0.10.0.tgz](http://www.apache.org/dyn/closer.lua/samza/0.10.0)
@@ -49,12 +50,12 @@ A Maven-based Samza project can pull in all required dependencies Samza dependen
<dependency>
<groupId>org.apache.samza</groupId>
<artifactId>samza-api</artifactId>
- <version>0.11.0</version>
+ <version>0.12.0</version>
</dependency>
<dependency>
<groupId>org.apache.samza</groupId>
- <artifactId>samza-core_2.10</artifactId>
- <version>0.11.0</version>
+ <artifactId>samza-core_2.11</artifactId>
+ <version>0.12.0</version>
<scope>runtime</scope>
</dependency>
<dependency>
@@ -62,41 +63,51 @@ A Maven-based Samza project can pull in all required dependencies Samza dependen
<artifactId>samza-shell</artifactId>
<classifier>dist</classifier>
<type>tgz</type>
- <version>0.11.0</version>
+ <version>0.12.0</version>
<scope>runtime</scope>
</dependency>
<dependency>
<groupId>org.apache.samza</groupId>
- <artifactId>samza-yarn_2.10</artifactId>
- <version>0.11.0</version>
+ <artifactId>samza-yarn_2.11</artifactId>
+ <version>0.12.0</version>
<scope>runtime</scope>
</dependency>
<dependency>
<groupId>org.apache.samza</groupId>
- <artifactId>samza-kv_2.10</artifactId>
- <version>0.11.0</version>
+ <artifactId>samza-kv_2.11</artifactId>
+ <version>0.12.0</version>
<scope>runtime</scope>
</dependency>
<dependency>
<groupId>org.apache.samza</groupId>
- <artifactId>samza-kv-rocksdb_2.10</artifactId>
- <version>0.11.0</version>
+ <artifactId>samza-kv-rocksdb_2.11</artifactId>
+ <version>0.12.0</version>
<scope>runtime</scope>
</dependency>
<dependency>
<groupId>org.apache.samza</groupId>
- <artifactId>samza-kv-inmemory_2.10</artifactId>
- <version>0.11.0</version>
+ <artifactId>samza-kv-inmemory_2.11</artifactId>
+ <version>0.12.0</version>
<scope>runtime</scope>
</dependency>
<dependency>
<groupId>org.apache.samza</groupId>
- <artifactId>samza-kafka_2.10</artifactId>
- <version>0.11.0</version>
+ <artifactId>samza-kafka_2.11</artifactId>
+ <version>0.12.0</version>
<scope>runtime</scope>
</dependency>
{% endhighlight %}
+Samza versions less than 0.12 should use artifacts with scala version 2.10 as suffix. For example,
+
+{% highlight xml %}
+<dependency>
+ <groupId>org.apache.samza</groupId>
+ <artifactId>samza-yarn_2.10</artifactId>
+ <version>0.11.0</version>
+</dependency>
+{% endhighlight %}
+
Samza versions less than 0.9 should include this additional dependency.
{% highlight xml %}
http://git-wip-us.apache.org/repos/asf/samza/blob/b5ea8776/docs/startup/hello-samza/versioned/index.md
----------------------------------------------------------------------
diff --git a/docs/startup/hello-samza/versioned/index.md b/docs/startup/hello-samza/versioned/index.md
index d0aca54..89b7ab9 100644
--- a/docs/startup/hello-samza/versioned/index.md
+++ b/docs/startup/hello-samza/versioned/index.md
@@ -61,7 +61,7 @@ Then, you can continue w/ the following command in hello-samza project:
{% highlight bash %}
mvn clean package
mkdir -p deploy/samza
-tar -xvf ./target/hello-samza-0.11.1-SNAPSHOT-dist.tar.gz -C deploy/samza
+tar -xvf ./target/hello-samza-0.13.0-SNAPSHOT-dist.tar.gz -C deploy/samza
{% endhighlight %}
### Run a Samza Job
[13/13] samza git commit: Merge master
Posted by jm...@apache.org.
Merge master
Project: http://git-wip-us.apache.org/repos/asf/samza/repo
Commit: http://git-wip-us.apache.org/repos/asf/samza/commit/03bacdc6
Tree: http://git-wip-us.apache.org/repos/asf/samza/tree/03bacdc6
Diff: http://git-wip-us.apache.org/repos/asf/samza/diff/03bacdc6
Branch: refs/heads/samza-fluent-api-v1
Commit: 03bacdc60e938179d59c2787a18eb144c5327909
Parents: 674e523 b5ea877
Author: Jacob Maes <jm...@linkedin.com>
Authored: Thu Feb 23 11:26:23 2017 -0800
Committer: Jacob Maes <jm...@linkedin.com>
Committed: Thu Feb 23 11:26:23 2017 -0800
----------------------------------------------------------------------
docs/_config.yml | 2 +-
docs/_layouts/default.html | 1 +
docs/archive/index.html | 8 +
docs/contribute/contributors-corner.md | 2 +-
.../versioned/jobs/split-deployment.md | 4 +-
.../versioned/deploy-samza-job-from-hdfs.md | 2 +-
.../tutorials/versioned/deploy-samza-to-CDH.md | 4 +-
.../versioned/remote-debugging-samza.md | 2 +-
.../versioned/run-in-multi-node-yarn.md | 16 +-
.../versioned/samza-rest-getting-started.md | 4 +-
docs/startup/download/index.md | 39 ++--
docs/startup/hello-samza/versioned/index.md | 2 +-
gradle.properties | 2 +-
.../grouper/task/GroupByContainerIds.java | 97 +++++++++
.../task/GroupByContainerIdsFactory.java | 34 +++
.../container/grouper/task/TaskNameGrouper.java | 5 +
.../standalone/StandaloneJobCoordinator.java | 2 +-
.../system/RemoteExecutionEnvironment.java | 3 +-
.../samza/coordinator/JobModelManager.scala | 18 +-
.../grouper/task/TestGroupByContainerIds.java | 207 +++++++++++++++++++
samza-test/src/main/python/configs/tests.json | 2 +-
21 files changed, 412 insertions(+), 44 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/samza/blob/03bacdc6/samza-core/src/main/java/org/apache/samza/system/RemoteExecutionEnvironment.java
----------------------------------------------------------------------
diff --cc samza-core/src/main/java/org/apache/samza/system/RemoteExecutionEnvironment.java
index 851c7f3,fafa2cb..ce129aa
--- a/samza-core/src/main/java/org/apache/samza/system/RemoteExecutionEnvironment.java
+++ b/samza-core/src/main/java/org/apache/samza/system/RemoteExecutionEnvironment.java
@@@ -33,12 -24,7 +33,13 @@@ import org.slf4j.LoggerFactory
/**
* This class implements the {@link ExecutionEnvironment} that runs the applications in YARN environment
*/
-public class RemoteExecutionEnvironment implements ExecutionEnvironment {
+public class RemoteExecutionEnvironment extends AbstractExecutionEnvironment {
++
++ private static final Logger log = LoggerFactory.getLogger(RemoteExecutionEnvironment.class);
+
+ public RemoteExecutionEnvironment(Config config) {
+ super(config);
+ }
- private static final Logger log = LoggerFactory.getLogger(RemoteExecutionEnvironment.class);
@Override public void run(StreamGraphBuilder app, Config config) {
// TODO: add description of ProcessContext that is going to create a sub-DAG of the {@code graph}
[06/13] samza git commit: SAMZA-1073: moving all operator classes
into samza-core
Posted by jm...@apache.org.
SAMZA-1073: moving all operator classes into samza-core
Project: http://git-wip-us.apache.org/repos/asf/samza/repo
Commit: http://git-wip-us.apache.org/repos/asf/samza/commit/8515448a
Tree: http://git-wip-us.apache.org/repos/asf/samza/tree/8515448a
Diff: http://git-wip-us.apache.org/repos/asf/samza/diff/8515448a
Branch: refs/heads/samza-fluent-api-v1
Commit: 8515448a2023ae6c78b9b0bb8e297cf346775e13
Parents: daaad7b
Author: Yi Pan (Data Infrastructure) <ni...@gmail.com>
Authored: Thu Feb 16 15:04:01 2017 -0800
Committer: Yi Pan (Data Infrastructure) <ni...@gmail.com>
Committed: Thu Feb 16 15:04:01 2017 -0800
----------------------------------------------------------------------
build.gradle | 24 +-
gradle/dependency-versions.gradle | 1 -
.../samza/operators/MessageStreamImpl.java | 182 +++++++++++++
.../apache/samza/operators/StreamGraphImpl.java | 260 +++++++++++++++++++
.../functions/PartialJoinFunction.java | 56 ++++
.../samza/operators/impl/OperatorGraph.java | 164 ++++++++++++
.../samza/operators/impl/OperatorImpl.java | 67 +++++
.../operators/impl/PartialJoinOperatorImpl.java | 47 ++++
.../samza/operators/impl/RootOperatorImpl.java | 35 +++
.../impl/SessionWindowOperatorImpl.java | 52 ++++
.../samza/operators/impl/SinkOperatorImpl.java | 44 ++++
.../operators/impl/StreamOperatorImpl.java | 49 ++++
.../operators/impl/WindowOperatorImpl.java | 43 +++
.../samza/operators/spec/OperatorSpec.java | 62 +++++
.../samza/operators/spec/OperatorSpecs.java | 210 +++++++++++++++
.../operators/spec/PartialJoinOperatorSpec.java | 86 ++++++
.../samza/operators/spec/SinkOperatorSpec.java | 116 +++++++++
.../operators/spec/StreamOperatorSpec.java | 91 +++++++
.../operators/spec/WindowOperatorSpec.java | 72 +++++
.../samza/operators/spec/WindowState.java | 81 ++++++
.../system/RemoteExecutionEnvironment.java | 37 +++
.../system/StandaloneExecutionEnvironment.java | 50 ++++
.../apache/samza/task/StreamOperatorTask.java | 111 ++++++++
.../samza/example/KeyValueStoreExample.java | 180 +++++++++++++
.../samza/example/NoContextStreamExample.java | 151 +++++++++++
.../samza/example/OrderShipmentJoinExample.java | 188 ++++++++++++++
.../samza/example/PageViewCounterExample.java | 129 +++++++++
.../samza/example/RepartitionExample.java | 140 ++++++++++
.../samza/example/TestBasicStreamGraphs.java | 99 +++++++
.../samza/example/TestBroadcastExample.java | 113 ++++++++
.../apache/samza/example/TestExampleBase.java | 46 ++++
.../apache/samza/example/TestJoinExample.java | 129 +++++++++
.../apache/samza/example/TestWindowExample.java | 81 ++++++
.../samza/operators/TestMessageStreamImpl.java | 204 +++++++++++++++
.../operators/TestMessageStreamImplUtil.java | 26 ++
.../data/JsonIncomingSystemMessageEnvelope.java | 60 +++++
.../samza/operators/impl/TestOperatorImpl.java | 73 ++++++
.../samza/operators/impl/TestOperatorImpls.java | 235 +++++++++++++++++
.../operators/impl/TestSinkOperatorImpl.java | 50 ++++
.../operators/impl/TestStreamOperatorImpl.java | 60 +++++
.../samza/operators/spec/TestOperatorSpecs.java | 127 +++++++++
samza-operator/README.md | 17 --
.../samza/operators/MessageStreamImpl.java | 182 -------------
.../apache/samza/operators/StreamGraphImpl.java | 260 -------------------
.../functions/PartialJoinFunction.java | 56 ----
.../samza/operators/impl/OperatorGraph.java | 164 ------------
.../samza/operators/impl/OperatorImpl.java | 67 -----
.../operators/impl/PartialJoinOperatorImpl.java | 47 ----
.../samza/operators/impl/RootOperatorImpl.java | 35 ---
.../impl/SessionWindowOperatorImpl.java | 52 ----
.../samza/operators/impl/SinkOperatorImpl.java | 44 ----
.../operators/impl/StreamOperatorImpl.java | 49 ----
.../operators/impl/WindowOperatorImpl.java | 43 ---
.../samza/operators/spec/OperatorSpec.java | 62 -----
.../samza/operators/spec/OperatorSpecs.java | 210 ---------------
.../operators/spec/PartialJoinOperatorSpec.java | 86 ------
.../samza/operators/spec/SinkOperatorSpec.java | 116 ---------
.../operators/spec/StreamOperatorSpec.java | 91 -------
.../operators/spec/WindowOperatorSpec.java | 72 -----
.../samza/operators/spec/WindowState.java | 81 ------
.../system/RemoteExecutionEnvironment.java | 37 ---
.../system/StandaloneExecutionEnvironment.java | 50 ----
.../apache/samza/task/StreamOperatorTask.java | 111 --------
.../samza/example/KeyValueStoreExample.java | 180 -------------
.../samza/example/NoContextStreamExample.java | 151 -----------
.../samza/example/OrderShipmentJoinExample.java | 188 --------------
.../samza/example/PageViewCounterExample.java | 129 ---------
.../samza/example/RepartitionExample.java | 140 ----------
.../samza/example/TestBasicStreamGraphs.java | 99 -------
.../samza/example/TestBroadcastExample.java | 113 --------
.../apache/samza/example/TestExampleBase.java | 46 ----
.../apache/samza/example/TestJoinExample.java | 129 ---------
.../apache/samza/example/TestWindowExample.java | 81 ------
.../samza/operators/TestMessageStreamImpl.java | 204 ---------------
.../operators/TestMessageStreamImplUtil.java | 26 --
.../data/JsonIncomingSystemMessageEnvelope.java | 60 -----
.../samza/operators/impl/TestOperatorImpl.java | 73 ------
.../samza/operators/impl/TestOperatorImpls.java | 235 -----------------
.../operators/impl/TestSinkOperatorImpl.java | 50 ----
.../operators/impl/TestStreamOperatorImpl.java | 60 -----
.../samza/operators/spec/TestOperatorSpecs.java | 127 ---------
settings.gradle | 1 -
82 files changed, 4007 insertions(+), 4048 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index 0d60970..400a913 100644
--- a/build.gradle
+++ b/build.gradle
@@ -160,6 +160,7 @@ project(":samza-core_$scalaVersion") {
compile "org.codehaus.jackson:jackson-mapper-asl:$jacksonVersion"
compile "org.eclipse.jetty:jetty-webapp:$jettyVersion"
compile "com.101tec:zkclient:$zkClientVersion"
+ testCompile project(":samza-api").sourceSets.test.output
testCompile "junit:junit:$junitVersion"
testCompile "org.mockito:mockito-all:$mockitoVersion"
testCompile "org.scalatest:scalatest_$scalaVersion:$scalaTestVersion"
@@ -372,29 +373,6 @@ project(":samza-yarn_$scalaVersion") {
jar.dependsOn("lesscss")
}
-project(":samza-operator") {
- apply plugin: 'java'
- apply plugin: 'checkstyle'
-
- sourceCompatibility = 1.8
-
- dependencies {
- compile project(':samza-api')
- compile project(":samza-core_$scalaVersion")
- // TODO: remove this dependency after refactoring operator implementation classes
- compile "org.reactivestreams:reactive-streams:$reactiveStreamVersion"
-
- testCompile project(":samza-api").sourceSets.test.output
- testCompile "junit:junit:$junitVersion"
- testCompile "org.mockito:mockito-all:$mockitoVersion"
- }
-
- checkstyle {
- configFile = new File(rootDir, "checkstyle/checkstyle.xml")
- toolVersion = "$checkstyleVersion"
- }
-}
-
project(":samza-shell") {
apply plugin: 'java'
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/gradle/dependency-versions.gradle
----------------------------------------------------------------------
diff --git a/gradle/dependency-versions.gradle b/gradle/dependency-versions.gradle
index 0193b64..0a8542b 100644
--- a/gradle/dependency-versions.gradle
+++ b/gradle/dependency-versions.gradle
@@ -38,6 +38,5 @@
commonsCodecVersion = "1.9"
commonsCollectionVersion = "3.2.1"
httpClientVersion="4.4.1"
- reactiveStreamVersion="1.0.0"
commonsLang3Version="3.4"
}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-core/src/main/java/org/apache/samza/operators/MessageStreamImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/MessageStreamImpl.java b/samza-core/src/main/java/org/apache/samza/operators/MessageStreamImpl.java
new file mode 100644
index 0000000..830e4a5
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/operators/MessageStreamImpl.java
@@ -0,0 +1,182 @@
+/*
+ * 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 java.util.Collection;
+import java.util.Collections;
+import java.util.function.Function;
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.samza.config.Config;
+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.PartialJoinFunction;
+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.WindowPane;
+import org.apache.samza.operators.windows.internal.WindowInternal;
+import org.apache.samza.task.TaskContext;
+
+
+/**
+ * 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 messages in this {@link MessageStream}
+ */
+public class MessageStreamImpl<M> implements MessageStream<M> {
+ /**
+ * The {@link StreamGraphImpl} object that contains this {@link MessageStreamImpl}
+ */
+ private final StreamGraphImpl graph;
+
+ /**
+ * The set of operators that consume the messages in this {@link MessageStream}
+ */
+ private final Set<OperatorSpec> registeredOperatorSpecs = new HashSet<>();
+
+ /**
+ * Default constructor
+ *
+ * @param graph the {@link StreamGraphImpl} object that this stream belongs to
+ */
+ MessageStreamImpl(StreamGraphImpl graph) {
+ this.graph = graph;
+ }
+
+ @Override public <TM> MessageStream<TM> map(MapFunction<M, TM> mapFn) {
+ OperatorSpec<TM> op = OperatorSpecs.<M, TM>createMapOperatorSpec(mapFn, this.graph, new MessageStreamImpl<>(this.graph));
+ this.registeredOperatorSpecs.add(op);
+ return op.getNextStream();
+ }
+
+ @Override public MessageStream<M> filter(FilterFunction<M> filterFn) {
+ OperatorSpec<M> op = OperatorSpecs.<M>createFilterOperatorSpec(filterFn, this.graph, new MessageStreamImpl<>(this.graph));
+ this.registeredOperatorSpecs.add(op);
+ return op.getNextStream();
+ }
+
+ @Override
+ public <TM> MessageStream<TM> flatMap(FlatMapFunction<M, TM> flatMapFn) {
+ OperatorSpec<TM> op = OperatorSpecs.createStreamOperatorSpec(flatMapFn, this.graph, new MessageStreamImpl<>(this.graph));
+ this.registeredOperatorSpecs.add(op);
+ return op.getNextStream();
+ }
+
+ @Override
+ public void sink(SinkFunction<M> sinkFn) {
+ this.registeredOperatorSpecs.add(OperatorSpecs.createSinkOperatorSpec(sinkFn, this.graph));
+ }
+
+ @Override public void sendTo(OutputStream<M> stream) {
+ this.registeredOperatorSpecs.add(OperatorSpecs.createSendToOperatorSpec(stream.getSinkFunction(), this.graph, stream));
+ }
+
+ @Override
+ public <K, WV> MessageStream<WindowPane<K, WV>> window(Window<M, K, WV> window) {
+ OperatorSpec<WindowPane<K, WV>> wndOp = OperatorSpecs.createWindowOperatorSpec((WindowInternal<M, K, WV>) window,
+ this.graph, new MessageStreamImpl<>(this.graph));
+ this.registeredOperatorSpecs.add(wndOp);
+ return wndOp.getNextStream();
+ }
+
+ @Override public <K, OM, RM> MessageStream<RM> join(MessageStream<OM> otherStream, JoinFunction<K, M, OM, RM> joinFn) {
+ MessageStreamImpl<RM> outputStream = new MessageStreamImpl<>(this.graph);
+
+ PartialJoinFunction<K, M, OM, RM> parJoin1 = new PartialJoinFunction<K, M, OM, RM>() {
+ @Override
+ public RM apply(M m1, OM om) {
+ return joinFn.apply(m1, om);
+ }
+
+ @Override
+ public K getKey(M message) {
+ return joinFn.getFirstKey(message);
+ }
+
+ @Override
+ public K getOtherKey(OM message) {
+ return joinFn.getSecondKey(message);
+ }
+
+ @Override
+ public void init(Config config, TaskContext context) {
+ joinFn.init(config, context);
+ }
+ };
+
+ PartialJoinFunction<K, OM, M, RM> parJoin2 = new PartialJoinFunction<K, OM, M, RM>() {
+ @Override
+ public RM apply(OM m1, M m) {
+ return joinFn.apply(m, m1);
+ }
+
+ @Override
+ public K getKey(OM message) {
+ return joinFn.getSecondKey(message);
+ }
+
+ @Override
+ public K getOtherKey(M message) {
+ return joinFn.getFirstKey(message);
+ }
+ };
+
+ // TODO: need to add default store functions for the two partial join functions
+
+ ((MessageStreamImpl<OM>) otherStream).registeredOperatorSpecs.add(
+ OperatorSpecs.<OM, K, M, RM>createPartialJoinOperatorSpec(parJoin2, this.graph, outputStream));
+ this.registeredOperatorSpecs.add(OperatorSpecs.<M, K, OM, RM>createPartialJoinOperatorSpec(parJoin1, this.graph, outputStream));
+ return outputStream;
+ }
+
+ @Override
+ public MessageStream<M> merge(Collection<MessageStream<M>> otherStreams) {
+ MessageStreamImpl<M> outputStream = new MessageStreamImpl<>(this.graph);
+
+ otherStreams.add(this);
+ otherStreams.forEach(other -> ((MessageStreamImpl<M>) other).registeredOperatorSpecs.
+ add(OperatorSpecs.createMergeOperatorSpec(this.graph, outputStream)));
+ return outputStream;
+ }
+
+ @Override
+ public <K> MessageStream<M> partitionBy(Function<M, K> parKeyExtractor) {
+ MessageStreamImpl<M> intStream = this.graph.createIntStream(parKeyExtractor);
+ OutputStream<M> outputStream = this.graph.getOutputStream(intStream);
+ this.registeredOperatorSpecs.add(OperatorSpecs.createPartitionOperatorSpec(outputStream.getSinkFunction(),
+ this.graph, outputStream));
+ return intStream;
+ }
+ /**
+ * 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/8515448a/samza-core/src/main/java/org/apache/samza/operators/StreamGraphImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/StreamGraphImpl.java b/samza-core/src/main/java/org/apache/samza/operators/StreamGraphImpl.java
new file mode 100644
index 0000000..dca3469
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/operators/StreamGraphImpl.java
@@ -0,0 +1,260 @@
+/*
+ * 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 java.util.Properties;
+import java.util.function.Function;
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.functions.SinkFunction;
+import org.apache.samza.serializers.Serde;
+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 java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * The implementation of {@link StreamGraph} interface. This class provides implementation of methods to allow users to
+ * create system input/output/intermediate streams.
+ */
+public class StreamGraphImpl implements StreamGraph {
+
+ /**
+ * Unique identifier for each {@link org.apache.samza.operators.spec.OperatorSpec} added to transform the {@link MessageEnvelope}
+ * in the input {@link MessageStream}s.
+ */
+ private int opId = 0;
+
+ private class InputStreamImpl<K, V, M extends MessageEnvelope<K, V>> extends MessageStreamImpl<M> {
+ final StreamSpec spec;
+ final Serde<K> keySerde;
+ final Serde<V> msgSerde;
+
+ InputStreamImpl(StreamGraphImpl graph, StreamSpec streamSpec, Serde<K> keySerde, Serde<V> msgSerde) {
+ super(graph);
+ this.spec = streamSpec;
+ this.keySerde = keySerde;
+ this.msgSerde = msgSerde;
+ }
+
+ StreamSpec getSpec() {
+ return this.spec;
+ }
+
+ }
+
+ private class OutputStreamImpl<K, V, M extends MessageEnvelope<K, V>> implements OutputStream<M> {
+ final StreamSpec spec;
+ final Serde<K> keySerde;
+ final Serde<V> msgSerde;
+
+ OutputStreamImpl(StreamGraphImpl graph, StreamSpec streamSpec, Serde<K> keySerde, Serde<V> msgSerde) {
+ this.spec = streamSpec;
+ this.keySerde = keySerde;
+ this.msgSerde = msgSerde;
+ }
+
+ StreamSpec getSpec() {
+ return this.spec;
+ }
+
+ @Override
+ public SinkFunction<M> getSinkFunction() {
+ return (M message, MessageCollector mc, TaskCoordinator tc) -> {
+ // TODO: need to find a way to directly pass in the serde class names
+ // mc.send(new OutgoingMessageEnvelope(this.spec.getSystemStream(), this.keySerde.getClass().getName(), this.msgSerde.getClass().getName(),
+ // message.getKey(), message.getKey(), message.getMessage()));
+ mc.send(new OutgoingMessageEnvelope(this.spec.getSystemStream(), message.getKey(), message.getMessage()));
+ };
+ }
+ }
+
+ private class IntermediateStreamImpl<PK, K, V, M extends MessageEnvelope<K, V>> extends InputStreamImpl<K, V, M> implements OutputStream<M> {
+ final Function<M, PK> parKeyFn;
+
+ /**
+ * Default constructor
+ *
+ * @param graph the {@link StreamGraphImpl} object that this stream belongs to
+ */
+ IntermediateStreamImpl(StreamGraphImpl graph, StreamSpec streamSpec, Serde<K> keySerde, Serde<V> msgSerde) {
+ this(graph, streamSpec, keySerde, msgSerde, null);
+ }
+
+ IntermediateStreamImpl(StreamGraphImpl graph, StreamSpec streamSpec, Serde<K> keySerde, Serde<V> msgSerde, Function<M, PK> parKeyFn) {
+ super(graph, streamSpec, keySerde, msgSerde);
+ this.parKeyFn = parKeyFn;
+ }
+
+ @Override
+ public SinkFunction<M> getSinkFunction() {
+ return (M message, MessageCollector mc, TaskCoordinator tc) -> {
+ // TODO: need to find a way to directly pass in the serde class names
+ // mc.send(new OutgoingMessageEnvelope(this.spec.getSystemStream(), this.keySerde.getClass().getName(), this.msgSerde.getClass().getName(),
+ // message.getKey(), message.getKey(), message.getMessage()));
+ if (this.parKeyFn == null) {
+ mc.send(new OutgoingMessageEnvelope(this.spec.getSystemStream(), message.getKey(), message.getMessage()));
+ } else {
+ // apply partition key function
+ mc.send(new OutgoingMessageEnvelope(this.spec.getSystemStream(), this.parKeyFn.apply(message), message.getKey(), message.getMessage()));
+ }
+ };
+ }
+ }
+
+ /**
+ * Maps keeping all {@link SystemStream}s that are input and output of operators in {@link StreamGraphImpl}
+ */
+ private final Map<SystemStream, MessageStream> inStreams = new HashMap<>();
+ private final Map<SystemStream, OutputStream> outStreams = new HashMap<>();
+
+ private ContextManager contextManager = new ContextManager() { };
+
+ @Override
+ public <K, V, M extends MessageEnvelope<K, V>> MessageStream<M> createInStream(StreamSpec streamSpec, Serde<K> keySerde, Serde<V> msgSerde) {
+ if (!this.inStreams.containsKey(streamSpec.getSystemStream())) {
+ this.inStreams.putIfAbsent(streamSpec.getSystemStream(), new InputStreamImpl<K, V, M>(this, streamSpec, keySerde, msgSerde));
+ }
+ return this.inStreams.get(streamSpec.getSystemStream());
+ }
+
+ /**
+ * Helper method to be used by {@link MessageStreamImpl} class
+ *
+ * @param streamSpec the {@link StreamSpec} object defining the {@link SystemStream} as the output
+ * @param <M> the type of {@link MessageEnvelope}s in the output {@link SystemStream}
+ * @return the {@link MessageStreamImpl} object
+ */
+ @Override
+ public <K, V, M extends MessageEnvelope<K, V>> OutputStream<M> createOutStream(StreamSpec streamSpec, Serde<K> keySerde, Serde<V> msgSerde) {
+ if (!this.outStreams.containsKey(streamSpec.getSystemStream())) {
+ this.outStreams.putIfAbsent(streamSpec.getSystemStream(), new OutputStreamImpl<K, V, M>(this, streamSpec, keySerde, msgSerde));
+ }
+ return this.outStreams.get(streamSpec.getSystemStream());
+ }
+
+ /**
+ * Helper method to be used by {@link MessageStreamImpl} class
+ *
+ * @param streamSpec the {@link StreamSpec} object defining the {@link SystemStream} as an intermediate {@link SystemStream}
+ * @param <M> the type of {@link MessageEnvelope}s in the output {@link SystemStream}
+ * @return the {@link MessageStreamImpl} object
+ */
+ @Override
+ public <K, V, M extends MessageEnvelope<K, V>> OutputStream<M> createIntStream(StreamSpec streamSpec, Serde<K> keySerde, Serde<V> msgSerde) {
+ if (!this.inStreams.containsKey(streamSpec.getSystemStream())) {
+ this.inStreams.putIfAbsent(streamSpec.getSystemStream(), new IntermediateStreamImpl<K, K, V, M>(this, streamSpec, keySerde, msgSerde));
+ }
+ IntermediateStreamImpl<K, K, V, M> intStream = (IntermediateStreamImpl<K, K, V, M>) this.inStreams.get(streamSpec.getSystemStream());
+ if (!this.outStreams.containsKey(streamSpec.getSystemStream())) {
+ this.outStreams.putIfAbsent(streamSpec.getSystemStream(), intStream);
+ }
+ return intStream;
+ }
+
+ @Override public Map<StreamSpec, MessageStream> getInStreams() {
+ Map<StreamSpec, MessageStream> inStreamMap = new HashMap<>();
+ this.inStreams.forEach((ss, entry) -> inStreamMap.put(((InputStreamImpl) entry).getSpec(), entry));
+ return Collections.unmodifiableMap(inStreamMap);
+ }
+
+ @Override public Map<StreamSpec, OutputStream> getOutStreams() {
+ Map<StreamSpec, OutputStream> outStreamMap = new HashMap<>();
+ this.outStreams.forEach((ss, entry) -> outStreamMap.put(((OutputStreamImpl) entry).getSpec(), entry));
+ return Collections.unmodifiableMap(outStreamMap);
+ }
+
+ @Override
+ public StreamGraph withContextManager(ContextManager manager) {
+ this.contextManager = manager;
+ return this;
+ }
+
+ public int getNextOpId() {
+ return this.opId++;
+ }
+
+ public ContextManager getContextManager() {
+ return this.contextManager;
+ }
+
+ /**
+ * Helper method to be get the input stream via {@link SystemStream}
+ *
+ * @param systemStream the {@link SystemStream}
+ * @return a {@link MessageStreamImpl} object corresponding to the {@code systemStream}
+ */
+ public MessageStreamImpl getInputStream(SystemStream systemStream) {
+ if (this.inStreams.containsKey(systemStream)) {
+ return (MessageStreamImpl) this.inStreams.get(systemStream);
+ }
+ return null;
+ }
+
+ <M> OutputStream<M> getOutputStream(MessageStreamImpl<M> intStream) {
+ if (this.outStreams.containsValue(intStream)) {
+ return (OutputStream<M>) intStream;
+ }
+ return null;
+ }
+
+ <M> MessageStream<M> getIntStream(OutputStream<M> outStream) {
+ if (this.inStreams.containsValue(outStream)) {
+ return (MessageStream<M>) outStream;
+ }
+ return null;
+ }
+
+ /**
+ * Method to create intermediate topics for {@link MessageStreamImpl#partitionBy(Function)} method.
+ *
+ * @param parKeyFn the function to extract the partition key from the input message
+ * @param <PK> the type of partition key
+ * @param <M> the type of input message
+ * @return the {@link OutputStream} object for the re-partitioned stream
+ */
+ <PK, M> MessageStreamImpl<M> createIntStream(Function<M, PK> parKeyFn) {
+ // TODO: placeholder to auto-generate intermediate streams via {@link StreamSpec}
+ StreamSpec streamSpec = new StreamSpec() {
+ @Override
+ public SystemStream getSystemStream() {
+ // TODO: should auto-generate intermedaite stream name here
+ return new SystemStream("intermediate", String.format("par-%d", StreamGraphImpl.this.opId));
+ }
+
+ @Override
+ public Properties getProperties() {
+ return null;
+ }
+ };
+
+ if (!this.inStreams.containsKey(streamSpec.getSystemStream())) {
+ this.inStreams.putIfAbsent(streamSpec.getSystemStream(), new IntermediateStreamImpl(this, streamSpec, null, null, parKeyFn));
+ }
+ IntermediateStreamImpl intStream = (IntermediateStreamImpl) this.inStreams.get(streamSpec.getSystemStream());
+ if (!this.outStreams.containsKey(streamSpec.getSystemStream())) {
+ this.outStreams.putIfAbsent(streamSpec.getSystemStream(), intStream);
+ }
+ return intStream;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-core/src/main/java/org/apache/samza/operators/functions/PartialJoinFunction.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/functions/PartialJoinFunction.java b/samza-core/src/main/java/org/apache/samza/operators/functions/PartialJoinFunction.java
new file mode 100644
index 0000000..809a70a
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/operators/functions/PartialJoinFunction.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.functions;
+
+import org.apache.samza.annotation.InterfaceStability;
+
+
+/**
+ * This defines the interface function a two-way join functions that takes input messages from two input
+ * {@link org.apache.samza.operators.MessageStream}s and merge them into a single output joined message in the join output
+ */
+@InterfaceStability.Unstable
+public interface PartialJoinFunction<K, M, OM, RM> extends InitableFunction {
+
+ /**
+ * Method to perform join method on the two input messages
+ *
+ * @param m1 message from the first input stream
+ * @param om message from the second input stream
+ * @return the joined message in the output stream
+ */
+ RM apply(M m1, OM om);
+
+ /**
+ * Method to get the key from the input message
+ *
+ * @param message the input message from the first strean
+ * @return the join key in the {@code message}
+ */
+ K getKey(M message);
+
+ /**
+ * Method to get the key from the input message in the other stream
+ *
+ * @param message the input message from the other stream
+ * @return the join key in the {@code message}
+ */
+ K getOtherKey(OM message);
+
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorGraph.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorGraph.java b/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorGraph.java
new file mode 100644
index 0000000..66336f8
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorGraph.java
@@ -0,0 +1,164 @@
+/*
+ * 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.config.Config;
+import org.apache.samza.operators.MessageStreamImpl;
+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.system.SystemStream;
+import org.apache.samza.task.TaskContext;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+
+/**
+ * Instantiates the DAG of {@link OperatorImpl}s corresponding to the {@link OperatorSpec}s for a
+ * {@link MessageStreamImpl}
+ */
+public class OperatorGraph {
+
+ /**
+ * A {@link Map} from {@link OperatorSpec} to {@link OperatorImpl}. This map registers all {@link OperatorImpl} in the DAG
+ * of {@link OperatorImpl} in a {@link org.apache.samza.container.TaskInstance}. Each {@link OperatorImpl} is created
+ * according to a single instance of {@link OperatorSpec}.
+ */
+ private final Map<OperatorSpec, OperatorImpl> operators = new HashMap<>();
+
+ /**
+ * This {@link Map} describes the DAG of {@link OperatorImpl} that are chained together to process the input messages.
+ */
+ private final Map<SystemStream, RootOperatorImpl> operatorGraph = new HashMap<>();
+
+ /**
+ * Initialize the whole DAG of {@link OperatorImpl}s, based on the input {@link MessageStreamImpl} from the {@link org.apache.samza.operators.StreamGraph}.
+ * This method will traverse each input {@link org.apache.samza.operators.MessageStream} in the {@code inputStreams} and
+ * instantiate the corresponding {@link OperatorImpl} chains that take the {@link org.apache.samza.operators.MessageStream} as input.
+ *
+ * @param inputStreams the map of input {@link org.apache.samza.operators.MessageStream}s
+ * @param config the {@link Config} required to instantiate operators
+ * @param context the {@link TaskContext} required to instantiate operators
+ */
+ public void init(Map<SystemStream, MessageStreamImpl> inputStreams, Config config, TaskContext context) {
+ inputStreams.forEach((ss, mstream) -> this.operatorGraph.put(ss, this.createOperatorImpls(mstream, config, context)));
+ }
+
+ /**
+ * Method to get the corresponding {@link RootOperatorImpl}
+ *
+ * @param ss input {@link SystemStream}
+ * @param <M> the type of input message
+ * @return the {@link OperatorImpl} that starts processing the input message
+ */
+ public <M> OperatorImpl<M, M> get(SystemStream ss) {
+ return this.operatorGraph.get(ss);
+ }
+
+ /**
+ * 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 messagess in the {@code source} {@link MessageStreamImpl}
+ * @param config the {@link Config} required to instantiate operators
+ * @param context the {@link TaskContext} required to instantiate operators
+ * @return root node for the {@link OperatorImpl} DAG
+ */
+ private <M> RootOperatorImpl<M> createOperatorImpls(MessageStreamImpl<M> source, Config config,
+ 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, ?> operatorImpl =
+ this.createAndRegisterOperatorImpl(registeredOperator, source, config, 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 <M> type of input message
+ * @param config the {@link Config} required to instantiate operators
+ * @param context the {@link TaskContext} required to instantiate operators
+ * @return the operator implementation for the operatorSpec
+ */
+ private <M> OperatorImpl<M, ?> createAndRegisterOperatorImpl(OperatorSpec operatorSpec,
+ MessageStreamImpl<M> source, Config config, TaskContext context) {
+ if (!operators.containsKey(operatorSpec)) {
+ OperatorImpl<M, ?> operatorImpl = createOperatorImpl(source, operatorSpec, config, context);
+ if (operators.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.
+ // initialize the corresponding operator function
+ operatorSpec.init(config, context);
+ MessageStreamImpl nextStream = operatorSpec.getNextStream();
+ if (nextStream != null) {
+ Collection<OperatorSpec> registeredSpecs = nextStream.getRegisteredOperatorSpecs();
+ registeredSpecs.forEach(registeredSpec -> {
+ OperatorImpl subImpl = this.createAndRegisterOperatorImpl(registeredSpec, nextStream, config, context);
+ operatorImpl.registerNextOperator(subImpl);
+ });
+ }
+ return operatorImpl;
+ }
+ }
+
+ // the implementation corresponding to operatorSpec has already been instantiated
+ // and registered, so we do not need to traverse the DAG further.
+ return operators.get(operatorSpec);
+ }
+
+ /**
+ * Creates a new {@link OperatorImpl} instance for the provided {@link OperatorSpec}.
+ *
+ * @param source the source {@link MessageStreamImpl}
+ * @param <M> type of input message
+ * @param operatorSpec the immutable {@link OperatorSpec} definition.
+ * @param config the {@link Config} required to instantiate operators
+ * @param context the {@link TaskContext} required to instantiate operators
+ * @return the {@link OperatorImpl} implementation instance
+ */
+ private static <M> OperatorImpl<M, ?> createOperatorImpl(MessageStreamImpl<M> source, OperatorSpec operatorSpec, Config config, TaskContext context) {
+ if (operatorSpec instanceof StreamOperatorSpec) {
+ StreamOperatorSpec<M, ?> streamOpSpec = (StreamOperatorSpec<M, ?>) operatorSpec;
+ return new StreamOperatorImpl<>(streamOpSpec, source, config, context);
+ } else if (operatorSpec instanceof SinkOperatorSpec) {
+ return new SinkOperatorImpl<>((SinkOperatorSpec<M>) operatorSpec, config, context);
+ } else if (operatorSpec instanceof WindowOperatorSpec) {
+ return new WindowOperatorImpl<>((WindowOperatorSpec<M, ?, ?>) operatorSpec, source, config, context);
+ } else if (operatorSpec instanceof PartialJoinOperatorSpec) {
+ return new PartialJoinOperatorImpl<>((PartialJoinOperatorSpec) operatorSpec, source, config, context);
+ }
+ throw new IllegalArgumentException(
+ String.format("Unsupported OperatorSpec: %s", operatorSpec.getClass().getName()));
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java b/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
new file mode 100644
index 0000000..abb1fa9
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImpl.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.task.MessageCollector;
+import org.apache.samza.task.TaskCoordinator;
+
+import java.util.HashSet;
+import java.util.Set;
+
+
+/**
+ * Abstract base class for all stream operator implementations.
+ */
+public abstract class OperatorImpl<M, RM> {
+
+ private final Set<OperatorImpl<RM, ?>> 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, ?> nextOperator) {
+ nextOperators.add(nextOperator);
+ }
+
+ /**
+ * 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 message
+ * @param collector the {@link MessageCollector} in the context
+ * @param coordinator the {@link TaskCoordinator} in the context
+ */
+ public abstract void onNext(M message, MessageCollector collector, TaskCoordinator coordinator);
+
+ /**
+ * Helper method to propagate the output of this operator to all registered downstream operators.
+ *
+ * This method <b>must</b> be called from {@link #onNext} to propagate the operator output correctly.
+ *
+ * @param outputMessage output message
+ * @param collector the {@link MessageCollector} in the context
+ * @param coordinator the {@link TaskCoordinator} in the context
+ */
+ 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/8515448a/samza-core/src/main/java/org/apache/samza/operators/impl/PartialJoinOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/PartialJoinOperatorImpl.java b/samza-core/src/main/java/org/apache/samza/operators/impl/PartialJoinOperatorImpl.java
new file mode 100644
index 0000000..c8515e1
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/operators/impl/PartialJoinOperatorImpl.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.config.Config;
+import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.spec.PartialJoinOperatorSpec;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskContext;
+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 messages in the input stream
+ * @param <JM> type of messages in the stream to join with
+ * @param <RM> type of messages in the joined stream
+ */
+class PartialJoinOperatorImpl<M, K, JM, RM> extends OperatorImpl<M, RM> {
+
+ PartialJoinOperatorImpl(PartialJoinOperatorSpec<M, K, JM, RM> joinOp, MessageStreamImpl<M> source, Config config, TaskContext context) {
+ // 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/8515448a/samza-core/src/main/java/org/apache/samza/operators/impl/RootOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/RootOperatorImpl.java b/samza-core/src/main/java/org/apache/samza/operators/impl/RootOperatorImpl.java
new file mode 100644
index 0000000..4b30a5d
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/operators/impl/RootOperatorImpl.java
@@ -0,0 +1,35 @@
+/*
+ * 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.task.MessageCollector;
+import org.apache.samza.task.TaskCoordinator;
+
+
+/**
+ * A no-op operator implementation that forwards incoming messages to all of its subscribers.
+ * @param <M> type of incoming messages
+ */
+final class RootOperatorImpl<M> 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/8515448a/samza-core/src/main/java/org/apache/samza/operators/impl/SessionWindowOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/SessionWindowOperatorImpl.java b/samza-core/src/main/java/org/apache/samza/operators/impl/SessionWindowOperatorImpl.java
new file mode 100644
index 0000000..2bb362c
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/operators/impl/SessionWindowOperatorImpl.java
@@ -0,0 +1,52 @@
+/*
+ * 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.config.Config;
+import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.spec.WindowOperatorSpec;
+import org.apache.samza.operators.windows.WindowPane;
+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 message
+ * @param <RK> the type of window key
+ * @param <WV> the type of window state
+ */
+class SessionWindowOperatorImpl<M, RK, WV> extends OperatorImpl<M, WindowPane<RK, WV>> {
+
+ private final WindowOperatorSpec<M, RK, WV> windowSpec;
+
+ SessionWindowOperatorImpl(WindowOperatorSpec<M, RK, WV> windowSpec, MessageStreamImpl<M> source, Config config, TaskContext context) {
+ this.windowSpec = windowSpec;
+ }
+
+ @Override
+ public void onNext(M message, MessageCollector collector, TaskCoordinator coordinator) {
+ }
+
+ 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/8515448a/samza-core/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java b/samza-core/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java
new file mode 100644
index 0000000..41d1778
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.operators.impl;
+
+import org.apache.samza.config.Config;
+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.TaskContext;
+import org.apache.samza.task.TaskCoordinator;
+
+
+/**
+ * Implementation for {@link SinkOperatorSpec}
+ */
+class SinkOperatorImpl<M> extends OperatorImpl<M, M> {
+
+ private final SinkFunction<M> sinkFn;
+
+ SinkOperatorImpl(SinkOperatorSpec<M> sinkOp, Config config, TaskContext context) {
+ this.sinkFn = sinkOp.getSinkFn();
+ }
+
+ @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/8515448a/samza-core/src/main/java/org/apache/samza/operators/impl/StreamOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/StreamOperatorImpl.java b/samza-core/src/main/java/org/apache/samza/operators/impl/StreamOperatorImpl.java
new file mode 100644
index 0000000..644de20
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/operators/impl/StreamOperatorImpl.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.impl;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.MessageStreamImpl;
+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.TaskContext;
+import org.apache.samza.task.TaskCoordinator;
+
+
+/**
+ * A StreamOperator that accepts a 1:n transform function and applies it to each incoming message.
+ *
+ * @param <M> type of message in the input stream
+ * @param <RM> type of message in the output stream
+ */
+class StreamOperatorImpl<M, RM> extends OperatorImpl<M, RM> {
+
+ private final FlatMapFunction<M, RM> transformFn;
+
+ StreamOperatorImpl(StreamOperatorSpec<M, RM> streamOperatorSpec, MessageStreamImpl<M> source, Config config, TaskContext context) {
+ 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/8515448a/samza-core/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java b/samza-core/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java
new file mode 100644
index 0000000..af00553
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.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.impl;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.spec.WindowOperatorSpec;
+import org.apache.samza.operators.windows.WindowPane;
+import org.apache.samza.operators.windows.internal.WindowInternal;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskContext;
+import org.apache.samza.task.TaskCoordinator;
+
+public class WindowOperatorImpl<M, WK, WV> extends OperatorImpl<M, WindowPane<WK, WV>> {
+
+ private final WindowInternal<M, WK, WV> window;
+
+ public WindowOperatorImpl(WindowOperatorSpec spec, MessageStreamImpl<M> source, Config config, TaskContext context) {
+ // source, config, and context are used to initialize the window kv-store
+ window = spec.getWindow();
+ }
+
+ @Override
+ public void onNext(M message, MessageCollector collector, TaskCoordinator coordinator) {
+
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java
new file mode 100644
index 0000000..1444662
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.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.annotation.InterfaceStability;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.task.TaskContext;
+
+
+/**
+ * A stateless serializable stream operator specification that holds all the information required
+ * to transform the input {@link MessageStreamImpl} and produce the output {@link MessageStreamImpl}.
+ *
+ * @param <OM> the type of output message from the operator
+ */
+@InterfaceStability.Unstable
+public interface OperatorSpec<OM> {
+
+ enum OpCode {
+ MAP,
+ FLAT_MAP,
+ FILTER,
+ SINK,
+ SEND_TO,
+ JOIN,
+ WINDOW,
+ MERGE,
+ PARTITION_BY
+ }
+
+
+ /**
+ * Get the output stream containing transformed messages produced by this operator.
+ * @return the output stream containing transformed messages produced by this operator.
+ */
+ MessageStreamImpl<OM> getNextStream();
+
+ /**
+ * Init method to initialize the context for this {@link OperatorSpec}. The default implementation is NO-OP.
+ *
+ * @param config the {@link Config} object for this task
+ * @param context the {@link TaskContext} object for this task
+ */
+ default void init(Config config, TaskContext context) { }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java b/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java
new file mode 100644
index 0000000..d626852
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java
@@ -0,0 +1,210 @@
+/*
+ * 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 java.util.Collection;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.OutputStream;
+import org.apache.samza.operators.StreamGraphImpl;
+import org.apache.samza.operators.functions.FilterFunction;
+import org.apache.samza.operators.functions.FlatMapFunction;
+import org.apache.samza.operators.functions.MapFunction;
+import org.apache.samza.operators.functions.PartialJoinFunction;
+import org.apache.samza.operators.functions.SinkFunction;
+import org.apache.samza.operators.windows.WindowPane;
+import org.apache.samza.operators.windows.internal.WindowInternal;
+
+import java.util.ArrayList;
+import org.apache.samza.task.TaskContext;
+
+
+/**
+ * Factory methods for creating {@link OperatorSpec} instances.
+ */
+public class OperatorSpecs {
+
+ private OperatorSpecs() {}
+
+ /**
+ * Creates a {@link StreamOperatorSpec} for {@link MapFunction}
+ *
+ * @param mapFn the map function
+ * @param graph the {@link StreamGraphImpl} object
+ * @param output the output {@link MessageStreamImpl} object
+ * @param <M> type of input message
+ * @param <OM> type of output message
+ * @return the {@link StreamOperatorSpec}
+ */
+ public static <M, OM> StreamOperatorSpec<M, OM> createMapOperatorSpec(MapFunction<M, OM> mapFn, StreamGraphImpl graph, MessageStreamImpl<OM> output) {
+ return new StreamOperatorSpec<>(new FlatMapFunction<M, OM>() {
+ @Override
+ public Collection<OM> apply(M message) {
+ return new ArrayList<OM>() {
+ {
+ OM r = mapFn.apply(message);
+ if (r != null) {
+ this.add(r);
+ }
+ }
+ };
+ }
+
+ @Override
+ public void init(Config config, TaskContext context) {
+ mapFn.init(config, context);
+ }
+ }, output, OperatorSpec.OpCode.MAP, graph.getNextOpId());
+ }
+
+ /**
+ * Creates a {@link StreamOperatorSpec} for {@link FilterFunction}
+ *
+ * @param filterFn the transformation function
+ * @param graph the {@link StreamGraphImpl} object
+ * @param output the output {@link MessageStreamImpl} object
+ * @param <M> type of input message
+ * @return the {@link StreamOperatorSpec}
+ */
+ public static <M> StreamOperatorSpec<M, M> createFilterOperatorSpec(FilterFunction<M> filterFn, StreamGraphImpl graph, MessageStreamImpl<M> output) {
+ return new StreamOperatorSpec<>(new FlatMapFunction<M, M>() {
+ @Override
+ public Collection<M> apply(M message) {
+ return new ArrayList<M>() {
+ {
+ if (filterFn.apply(message)) {
+ this.add(message);
+ }
+ }
+ };
+ }
+
+ @Override
+ public void init(Config config, TaskContext context) {
+ filterFn.init(config, context);
+ }
+ }, output, OperatorSpec.OpCode.FILTER, graph.getNextOpId());
+ }
+
+ /**
+ * Creates a {@link StreamOperatorSpec}.
+ *
+ * @param transformFn the transformation function
+ * @param graph the {@link StreamGraphImpl} object
+ * @param output the output {@link MessageStreamImpl} object
+ * @param <M> type of input message
+ * @param <OM> type of output message
+ * @return the {@link StreamOperatorSpec}
+ */
+ public static <M, OM> StreamOperatorSpec<M, OM> createStreamOperatorSpec(
+ FlatMapFunction<M, OM> transformFn, StreamGraphImpl graph, MessageStreamImpl<OM> output) {
+ return new StreamOperatorSpec<>(transformFn, output, OperatorSpec.OpCode.FLAT_MAP, graph.getNextOpId());
+ }
+
+ /**
+ * Creates a {@link SinkOperatorSpec}.
+ *
+ * @param sinkFn the sink function
+ * @param <M> type of input message
+ * @param graph the {@link StreamGraphImpl} object
+ * @return the {@link SinkOperatorSpec}
+ */
+ public static <M> SinkOperatorSpec<M> createSinkOperatorSpec(SinkFunction<M> sinkFn, StreamGraphImpl graph) {
+ return new SinkOperatorSpec<>(sinkFn, OperatorSpec.OpCode.SINK, graph.getNextOpId());
+ }
+
+ /**
+ * Creates a {@link SinkOperatorSpec}.
+ *
+ * @param sinkFn the sink function
+ * @param graph the {@link StreamGraphImpl} object
+ * @param stream the {@link OutputStream} where the message is sent to
+ * @param <M> type of input message
+ * @return the {@link SinkOperatorSpec}
+ */
+ public static <M> SinkOperatorSpec<M> createSendToOperatorSpec(SinkFunction<M> sinkFn, StreamGraphImpl graph, OutputStream<M> stream) {
+ return new SinkOperatorSpec<>(sinkFn, OperatorSpec.OpCode.SEND_TO, graph.getNextOpId(), stream);
+ }
+
+ /**
+ * Creates a {@link SinkOperatorSpec}.
+ *
+ * @param sinkFn the sink function
+ * @param graph the {@link StreamGraphImpl} object
+ * @param stream the {@link OutputStream} where the message is sent to
+ * @param <M> type of input message
+ * @return the {@link SinkOperatorSpec}
+ */
+ public static <M> SinkOperatorSpec<M> createPartitionOperatorSpec(SinkFunction<M> sinkFn, StreamGraphImpl graph, OutputStream<M> stream) {
+ return new SinkOperatorSpec<>(sinkFn, OperatorSpec.OpCode.PARTITION_BY, graph.getNextOpId(), stream);
+ }
+
+ /**
+ * Creates a {@link WindowOperatorSpec}.
+ *
+ * @param window the description of the window.
+ * @param graph the {@link StreamGraphImpl} object
+ * @param wndOutput the window output {@link MessageStreamImpl} object
+ * @param <M> the type of input message
+ * @param <WK> the type of key in the {@link WindowPane}
+ * @param <WV> the type of value in the window
+ * @return the {@link WindowOperatorSpec}
+ */
+
+ public static <M, WK, WV> WindowOperatorSpec<M, WK, WV> createWindowOperatorSpec(
+ WindowInternal<M, WK, WV> window, StreamGraphImpl graph, MessageStreamImpl<WindowPane<WK, WV>> wndOutput) {
+ return new WindowOperatorSpec<>(window, wndOutput, graph.getNextOpId());
+ }
+
+ /**
+ * Creates a {@link PartialJoinOperatorSpec}.
+ *
+ * @param partialJoinFn the join function
+ * @param graph the {@link StreamGraphImpl} object
+ * @param joinOutput the output {@link MessageStreamImpl}
+ * @param <M> type of input message
+ * @param <K> type of join key
+ * @param <JM> the type of message in the other join stream
+ * @param <OM> the type of message in the join output
+ * @return the {@link PartialJoinOperatorSpec}
+ */
+ public static <M, K, JM, OM> PartialJoinOperatorSpec<M, K, JM, OM> createPartialJoinOperatorSpec(
+ PartialJoinFunction<K, M, JM, OM> partialJoinFn, StreamGraphImpl graph, MessageStreamImpl<OM> joinOutput) {
+ return new PartialJoinOperatorSpec<>(partialJoinFn, joinOutput, graph.getNextOpId());
+ }
+
+ /**
+ * Creates a {@link StreamOperatorSpec} with a merger function.
+ *
+ * @param graph the {@link StreamGraphImpl} object
+ * @param mergeOutput the output {@link MessageStreamImpl} from the merger
+ * @param <M> the type of input message
+ * @return the {@link StreamOperatorSpec} for the merge
+ */
+ public static <M> StreamOperatorSpec<M, M> createMergeOperatorSpec(StreamGraphImpl graph, MessageStreamImpl<M> mergeOutput) {
+ return new StreamOperatorSpec<M, M>(message ->
+ new ArrayList<M>() {
+ {
+ this.add(message);
+ }
+ },
+ mergeOutput, OperatorSpec.OpCode.MERGE, graph.getNextOpId());
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-core/src/main/java/org/apache/samza/operators/spec/PartialJoinOperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/PartialJoinOperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/PartialJoinOperatorSpec.java
new file mode 100644
index 0000000..e057c2b
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/operators/spec/PartialJoinOperatorSpec.java
@@ -0,0 +1,86 @@
+/*
+ * 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.config.Config;
+import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.functions.PartialJoinFunction;
+import org.apache.samza.task.TaskContext;
+
+
+/**
+ * Spec for the partial join operator that takes messages from one input stream, joins with buffered
+ * messages from another stream, and produces join results to an output {@link MessageStreamImpl}.
+ *
+ * @param <M> the type of input message
+ * @param <K> the type of join key
+ * @param <JM> the type of message in the other join stream
+ * @param <RM> the type of message in the join output stream
+ */
+public class PartialJoinOperatorSpec<M, K, JM, RM> implements OperatorSpec<RM> {
+
+ private final MessageStreamImpl<RM> joinOutput;
+
+ /**
+ * The transformation function of {@link PartialJoinOperatorSpec} that takes an input message of
+ * type {@code M}, joins with a stream of buffered messages of type {@code JM} from another stream,
+ * and generates a joined result message of type {@code RM}.
+ */
+ private final PartialJoinFunction<K, M, JM, RM> transformFn;
+
+
+ /**
+ * The unique ID for this operator.
+ */
+ private final int opId;
+
+ /**
+ * Default constructor for a {@link PartialJoinOperatorSpec}.
+ *
+ * @param partialJoinFn partial join function that take type {@code M} of input message and join
+ * w/ type {@code JM} of buffered message from another stream
+ * @param joinOutput the output {@link MessageStreamImpl} of the join results
+ */
+ PartialJoinOperatorSpec(PartialJoinFunction<K, M, JM, RM> partialJoinFn, MessageStreamImpl<RM> joinOutput, int opId) {
+ this.joinOutput = joinOutput;
+ this.transformFn = partialJoinFn;
+ this.opId = opId;
+ }
+
+ @Override
+ public MessageStreamImpl<RM> getNextStream() {
+ return this.joinOutput;
+ }
+
+ public PartialJoinFunction<K, M, JM, RM> getTransformFn() {
+ return this.transformFn;
+ }
+
+ public OperatorSpec.OpCode getOpCode() {
+ return OpCode.JOIN;
+ }
+
+ public int getOpId() {
+ return this.opId;
+ }
+
+ @Override public void init(Config config, TaskContext context) {
+ this.transformFn.init(config, context);
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-core/src/main/java/org/apache/samza/operators/spec/SinkOperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/SinkOperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/SinkOperatorSpec.java
new file mode 100644
index 0000000..ba30d67
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/operators/spec/SinkOperatorSpec.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.config.Config;
+import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.OutputStream;
+import org.apache.samza.operators.functions.SinkFunction;
+import org.apache.samza.task.TaskContext;
+
+
+/**
+ * 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 message
+ */
+public class SinkOperatorSpec<M> implements OperatorSpec {
+
+ /**
+ * {@link OpCode} for this {@link SinkOperatorSpec}
+ */
+ private final OperatorSpec.OpCode opCode;
+
+ /**
+ * The unique ID for this operator.
+ */
+ private final int opId;
+
+ /**
+ * The user-defined sink function
+ */
+ private final SinkFunction<M> sinkFn;
+
+ /**
+ * Potential output stream defined by the {@link SinkFunction}
+ */
+ private final OutputStream<M> outStream;
+
+ /**
+ * Default constructor for a {@link SinkOperatorSpec} w/o an output stream. (e.g. output is sent to remote database)
+ *
+ * @param sinkFn a user defined {@link SinkFunction} that will be called with the output message,
+ * the output {@link org.apache.samza.task.MessageCollector} and the
+ * {@link org.apache.samza.task.TaskCoordinator}.
+ * @param opCode the specific {@link OpCode} for this {@link SinkOperatorSpec}. It could be {@link OpCode#SINK}, {@link OpCode#SEND_TO},
+ * or {@link OpCode#PARTITION_BY}
+ * @param opId the unique id of this {@link SinkOperatorSpec} in the {@link org.apache.samza.operators.StreamGraph}
+ */
+ SinkOperatorSpec(SinkFunction<M> sinkFn, OperatorSpec.OpCode opCode, int opId) {
+ this(sinkFn, opCode, opId, null);
+ }
+
+ /**
+ * Default constructor for a {@link SinkOperatorSpec} that sends the output to an {@link OutputStream}
+ *
+ * @param sinkFn a user defined {@link SinkFunction} that will be called with the output message,
+ * the output {@link org.apache.samza.task.MessageCollector} and the
+ * {@link org.apache.samza.task.TaskCoordinator}.
+ * @param opCode the specific {@link OpCode} for this {@link SinkOperatorSpec}. It could be {@link OpCode#SINK}, {@link OpCode#SEND_TO},
+ * or {@link OpCode#PARTITION_BY}
+ * @param opId the unique id of this {@link SinkOperatorSpec} in the {@link org.apache.samza.operators.StreamGraph}
+ * @param opId the {@link OutputStream} for this {@link SinkOperatorSpec}
+ */
+ SinkOperatorSpec(SinkFunction<M> sinkFn, OperatorSpec.OpCode opCode, int opId, OutputStream<M> outStream) {
+ this.sinkFn = sinkFn;
+ this.opCode = opCode;
+ this.opId = opId;
+ this.outStream = outStream;
+ }
+
+ /**
+ * This is a terminal operator and doesn't allow further operator chaining.
+ * @return null
+ */
+ @Override
+ public MessageStreamImpl<M> getNextStream() {
+ return null;
+ }
+
+ public SinkFunction<M> getSinkFn() {
+ return this.sinkFn;
+ }
+
+ public OperatorSpec.OpCode getOpCode() {
+ return this.opCode;
+ }
+
+ public int getOpId() {
+ return this.opId;
+ }
+
+ public OutputStream<M> getOutStream() {
+ return this.outStream;
+ }
+
+ @Override public void init(Config config, TaskContext context) {
+ this.sinkFn.init(config, context);
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-core/src/main/java/org/apache/samza/operators/spec/StreamOperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/StreamOperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/StreamOperatorSpec.java
new file mode 100644
index 0000000..d7813f7
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/operators/spec/StreamOperatorSpec.java
@@ -0,0 +1,91 @@
+/*
+ * 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.config.Config;
+import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.functions.FlatMapFunction;
+import org.apache.samza.task.TaskContext;
+
+
+/**
+ * The spec for a linear stream operator that outputs 0 or more messages for each input message.
+ *
+ * @param <M> the type of input message
+ * @param <OM> the type of output message
+ */
+public class StreamOperatorSpec<M, OM> implements OperatorSpec<OM> {
+
+ /**
+ * {@link OpCode} for this {@link StreamOperatorSpec}
+ */
+ private final OperatorSpec.OpCode opCode;
+
+ /**
+ * The unique ID for this operator.
+ */
+ private final int opId;
+
+ /**
+ * The output {@link MessageStreamImpl} from this {@link StreamOperatorSpec}
+ */
+ private final MessageStreamImpl<OM> outputStream;
+
+ /**
+ * Transformation function applied in this {@link StreamOperatorSpec}
+ */
+ private final FlatMapFunction<M, OM> transformFn;
+
+ /**
+ * Constructor for a {@link StreamOperatorSpec} that accepts an output {@link MessageStreamImpl}.
+ *
+ * @param transformFn the transformation function
+ * @param outputStream the output {@link MessageStreamImpl}
+ * @param opCode the {@link OpCode} for this {@link StreamOperatorSpec}
+ * @param opId the unique id for this {@link StreamOperatorSpec} in a {@link org.apache.samza.operators.StreamGraph}
+ */
+ StreamOperatorSpec(FlatMapFunction<M, OM> transformFn, MessageStreamImpl outputStream, OperatorSpec.OpCode opCode, int opId) {
+ this.outputStream = outputStream;
+ this.transformFn = transformFn;
+ this.opCode = opCode;
+ this.opId = opId;
+ }
+
+ @Override
+ public MessageStreamImpl<OM> getNextStream() {
+ return this.outputStream;
+ }
+
+ public FlatMapFunction<M, OM> getTransformFn() {
+ return this.transformFn;
+ }
+
+ public OperatorSpec.OpCode getOpCode() {
+ return this.opCode;
+ }
+
+ public int getOpId() {
+ return this.opId;
+ }
+
+ @Override
+ public void init(Config config, TaskContext context) {
+ this.transformFn.init(config, context);
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-core/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java
new file mode 100644
index 0000000..46417ed
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.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.spec;
+
+import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.windows.WindowPane;
+import org.apache.samza.operators.windows.internal.WindowInternal;
+
+
+/**
+ * Default window operator spec object
+ *
+ * @param <M> the type of input message to the window
+ * @param <WK> the type of key of the window
+ * @param <WV> the type of aggregated value in the window output {@link WindowPane}
+ */
+public class WindowOperatorSpec<M, WK, WV> implements OperatorSpec<WindowPane<WK, WV>> {
+
+ private final WindowInternal<M, WK, WV> window;
+
+ private final MessageStreamImpl<WindowPane<WK, WV>> outputStream;
+
+ private final int opId;
+
+
+ /**
+ * Constructor for {@link WindowOperatorSpec}.
+ *
+ * @param window the window function
+ * @param outputStream the output {@link MessageStreamImpl} from this {@link WindowOperatorSpec}
+ * @param opId auto-generated unique ID of this operator
+ */
+ WindowOperatorSpec(WindowInternal<M, WK, WV> window, MessageStreamImpl<WindowPane<WK, WV>> outputStream, int opId) {
+ this.outputStream = outputStream;
+ this.window = window;
+ this.opId = opId;
+ }
+
+ @Override
+ public MessageStreamImpl<WindowPane<WK, WV>> getNextStream() {
+ return this.outputStream;
+ }
+
+ public WindowInternal getWindow() {
+ return window;
+ }
+
+ public OpCode getOpCode() {
+ return OpCode.WINDOW;
+ }
+
+ public int getOpId() {
+ return this.opId;
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-core/src/main/java/org/apache/samza/operators/spec/WindowState.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/WindowState.java b/samza-core/src/main/java/org/apache/samza/operators/spec/WindowState.java
new file mode 100644
index 0000000..53bca2e
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/operators/spec/WindowState.java
@@ -0,0 +1,81 @@
+/*
+ * 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.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/8515448a/samza-core/src/main/java/org/apache/samza/system/RemoteExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/system/RemoteExecutionEnvironment.java b/samza-core/src/main/java/org/apache/samza/system/RemoteExecutionEnvironment.java
new file mode 100644
index 0000000..fafa2cb
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/system/RemoteExecutionEnvironment.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.system;
+
+import org.apache.samza.operators.StreamGraphBuilder;
+import org.apache.samza.config.Config;
+
+/**
+ * This class implements the {@link ExecutionEnvironment} that runs the applications in YARN environment
+ */
+public class RemoteExecutionEnvironment implements ExecutionEnvironment {
+
+ @Override public void run(StreamGraphBuilder app, Config config) {
+ // TODO: add description of ProcessContext that is going to create a sub-DAG of the {@code graph}
+ // TODO: actually instantiate the tasks and run the job, i.e.
+ // 1. create all input/output/intermediate topics
+ // 2. create the single job configuration
+ // 3. execute JobRunner to submit the single job for the whole graph
+ }
+
+}
[10/13] samza git commit: SAMZA-1097: update master branch to use
0.13.0-SNAPSHOT version
Posted by jm...@apache.org.
SAMZA-1097: update master branch to use 0.13.0-SNAPSHOT version
Author: Yi Pan (Data Infrastructure) <ni...@gmail.com>
Reviewers: jagadish <jv...@linkedin.com>
Closes #59 from nickpan47/SAMZA-1097
Project: http://git-wip-us.apache.org/repos/asf/samza/repo
Commit: http://git-wip-us.apache.org/repos/asf/samza/commit/eab18001
Tree: http://git-wip-us.apache.org/repos/asf/samza/tree/eab18001
Diff: http://git-wip-us.apache.org/repos/asf/samza/diff/eab18001
Branch: refs/heads/samza-fluent-api-v1
Commit: eab180010e9e4cd3511b55c838681b0b12e21f94
Parents: 062ac56
Author: Yi Pan (Data Infrastructure) <ni...@gmail.com>
Authored: Tue Feb 21 17:16:58 2017 -0800
Committer: Yi Pan (Data Infrastructure) <ni...@gmail.com>
Committed: Tue Feb 21 17:16:58 2017 -0800
----------------------------------------------------------------------
gradle.properties | 2 +-
samza-test/src/main/python/configs/tests.json | 2 +-
2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/samza/blob/eab18001/gradle.properties
----------------------------------------------------------------------
diff --git a/gradle.properties b/gradle.properties
index 82fa1ed..026f406 100644
--- a/gradle.properties
+++ b/gradle.properties
@@ -15,7 +15,7 @@
# specific language governing permissions and limitations
# under the License.
group=org.apache.samza
-version=0.12.0
+version=0.13.0-SNAPSHOT
scalaVersion=2.11
gradleVersion=2.8
http://git-wip-us.apache.org/repos/asf/samza/blob/eab18001/samza-test/src/main/python/configs/tests.json
----------------------------------------------------------------------
diff --git a/samza-test/src/main/python/configs/tests.json b/samza-test/src/main/python/configs/tests.json
index 5dcb724..a9f7881 100644
--- a/samza-test/src/main/python/configs/tests.json
+++ b/samza-test/src/main/python/configs/tests.json
@@ -1,5 +1,5 @@
{
- "samza_executable": "samza-test_2.11-0.12.0.tgz",
+ "samza_executable": "samza-test_2.11-0.13.0-SNAPSHOT.tgz",
"samza_install_path": "deploy/smoke_tests",
"samza_config_factory": "org.apache.samza.config.factories.PropertiesConfigFactory"
}
[07/13] samza git commit: SAMZA-1075: Refactor SystemAdmin Interface
to expose a common method to create streams
Posted by jm...@apache.org.
SAMZA-1075: Refactor SystemAdmin Interface to expose a common method to create streams
Author: Jacob Maes <jm...@linkedin.com>
Reviewers: Yi Pan (Data Infrastructure) <ni...@gmail.com>
Closes #53 from jmakes/samza-1075
Project: http://git-wip-us.apache.org/repos/asf/samza/repo
Commit: http://git-wip-us.apache.org/repos/asf/samza/commit/7e7747fb
Tree: http://git-wip-us.apache.org/repos/asf/samza/tree/7e7747fb
Diff: http://git-wip-us.apache.org/repos/asf/samza/diff/7e7747fb
Branch: refs/heads/samza-fluent-api-v1
Commit: 7e7747fb99b30aa25078b5a59911c39d32f95921
Parents: 8515448
Author: Jacob Maes <jm...@linkedin.com>
Authored: Fri Feb 17 12:49:19 2017 -0800
Committer: Jacob Maes <jm...@linkedin.com>
Committed: Fri Feb 17 12:49:19 2017 -0800
----------------------------------------------------------------------
.../org/apache/samza/system/StreamSpec.java | 203 +++++++++++++++++++
.../samza/system/StreamValidationException.java | 30 +++
.../org/apache/samza/system/SystemAdmin.java | 29 ++-
.../samza/system/kafka/KafkaStreamSpec.java | 141 +++++++++++++
.../org/apache/samza/config/KafkaConfig.scala | 115 ++++++-----
.../samza/system/kafka/KafkaSystemAdmin.scala | 143 +++++++------
.../system/kafka/TestKafkaSystemAdminJava.java | 145 +++++++++++++
.../system/kafka/TestKafkaSystemAdmin.scala | 24 ++-
8 files changed, 700 insertions(+), 130 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/samza/blob/7e7747fb/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java b/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java
new file mode 100644
index 0000000..d8a2144
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/system/StreamSpec.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.system;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+
+/**
+ * StreamSpec is a blueprint for creating, validating, or simply describing a stream in the runtime environment.
+ *
+ * It has specific attributes for common behaviors that Samza uses.
+ *
+ * It also includes a map of configurations which may be system-specific.
+ *
+ * It is immutable by design.
+ */
+public class StreamSpec {
+
+ private static final int DEFAULT_PARTITION_COUNT = 1;
+
+ /**
+ * Unique identifier for the stream in a Samza application.
+ * This identifier is used as a key for stream properties in the
+ * job config and to distinguish between streams in a graph.
+ */
+ private final String id;
+
+ /**
+ * The System name on which this stream will exist. Corresponds to a named implementation of the
+ * Samza System abstraction.
+ */
+ private final String systemName;
+
+ /**
+ * The physical identifier for the stream. This is the identifier that will be used in remote
+ * systems to identify the stream. In Kafka this would be the topic name whereas in HDFS it
+ * might be a file URN.
+ */
+ private final String physicalName;
+
+ /**
+ * The number of partitions for the stream.
+ */
+ private final int partitionCount;
+
+ /**
+ * A set of all system-specific configurations for the stream.
+ */
+ private final Map<String, String> config;
+
+ /**
+ * @param id The application-unique logical identifier for the stream. It is used to distinguish between
+ * streams in a Samza application so it must be unique in the context of one deployable unit.
+ * It does not need to be globally unique or unique with respect to a host.
+ *
+ * @param physicalName The physical identifier for the stream. This is the identifier that will be used in remote
+ * systems to identify the stream. In Kafka this would be the topic name whereas in HDFS it
+ * might be a file URN.
+ *
+ * @param systemName The System name on which this stream will exist. Corresponds to a named implementation of the
+ * Samza System abstraction. See {@link SystemFactory}
+ */
+ public StreamSpec(String id, String physicalName, String systemName) {
+ this(id, physicalName, systemName, DEFAULT_PARTITION_COUNT, Collections.emptyMap());
+ }
+
+ /**
+ *
+ * @param id The application-unique logical identifier for the stream. It is used to distinguish between
+ * streams in a Samza application so it must be unique in the context of one deployable unit.
+ * It does not need to be globally unique or unique with respect to a host.
+ *
+ * @param physicalName The physical identifier for the stream. This is the identifier that will be used in remote
+ * systems to identify the stream. In Kafka this would be the topic name whereas in HDFS it
+ * might be a file URN.
+ *
+ * @param systemName The System name on which this stream will exist. Corresponds to a named implementation of the
+ * Samza System abstraction. See {@link SystemFactory}
+ *
+ * @param partitionCount The number of partitionts for the stream. A value of {@code 1} indicates unpartitioned.
+ */
+ public StreamSpec(String id, String physicalName, String systemName, int partitionCount) {
+ this(id, physicalName, systemName, partitionCount, Collections.emptyMap());
+ }
+
+ /**
+ * @param id The application-unique logical identifier for the stream. It is used to distinguish between
+ * streams in a Samza application so it must be unique in the context of one deployable unit.
+ * It does not need to be globally unique or unique with respect to a host.
+ *
+ * @param physicalName The physical identifier for the stream. This is the identifier that will be used in remote
+ * systems to identify the stream. In Kafka this would be the topic name whereas in HDFS it
+ * might be a file URN.
+ *
+ * @param systemName The System name on which this stream will exist. Corresponds to a named implementation of the
+ * Samza System abstraction. See {@link SystemFactory}
+ *
+ * @param config A map of properties for the stream. These may be System-specfic.
+ */
+ public StreamSpec(String id, String physicalName, String systemName, Map<String, String> config) {
+ this(id, physicalName, systemName, DEFAULT_PARTITION_COUNT, config);
+ }
+
+ /**
+ * @param id The application-unique logical identifier for the stream. It is used to distinguish between
+ * streams in a Samza application so it must be unique in the context of one deployable unit.
+ * It does not need to be globally unique or unique with respect to a host.
+ *
+ * @param physicalName The physical identifier for the stream. This is the identifier that will be used in remote
+ * systems to identify the stream. In Kafka this would be the topic name whereas in HDFS it
+ * might be a file URN.
+ *
+ * @param systemName The System name on which this stream will exist. Corresponds to a named implementation of the
+ * Samza System abstraction. See {@link SystemFactory}
+ *
+ * @param partitionCount The number of partitionts for the stream. A value of {@code 1} indicates unpartitioned.
+ *
+ * @param config A map of properties for the stream. These may be System-specfic.
+ */
+ public StreamSpec(String id, String physicalName, String systemName, int partitionCount, Map<String, String> config) {
+ if (id == null) {
+ throw new NullPointerException("Parameter 'id' must not be null");
+ }
+
+ if (systemName == null) {
+ throw new NullPointerException("Parameter 'systemName' must not be null");
+ }
+
+ if (partitionCount < 1) {
+ throw new NullPointerException("Parameter 'partitionCount' must not be greater than 0");
+ }
+
+ this.id = id;
+ this.systemName = systemName;
+ this.physicalName = physicalName;
+ this.partitionCount = partitionCount;
+
+ if (config != null) {
+ this.config = Collections.unmodifiableMap(new HashMap<>(config));
+ } else {
+ this.config = Collections.emptyMap();
+ }
+ }
+
+ /**
+ * Copies this StreamSpec, but applies a new partitionCount.
+ *
+ * This method is not static s.t. subclasses can override it.
+ *
+ * @param partitionCount The partitionCount for the returned StreamSpec.
+ * @return A copy of this StreamSpec with the specified partitionCount.
+ */
+ public StreamSpec copyWithPartitionCount(int partitionCount) {
+ return new StreamSpec(id, physicalName, systemName, partitionCount, config);
+ }
+
+ public String getId() {
+ return id;
+ }
+
+ public String getSystemName() {
+ return systemName;
+ }
+
+ public String getPhysicalName() {
+ return physicalName;
+ }
+
+ public int getPartitionCount() {
+ return partitionCount;
+ }
+
+ public Map<String, String> getConfig() {
+ return config;
+ }
+
+ public String get(String propertyName) {
+ return config.get(propertyName);
+ }
+
+ public String getOrDefault(String propertyName, String defaultValue) {
+ return config.getOrDefault(propertyName, defaultValue);
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/7e7747fb/samza-api/src/main/java/org/apache/samza/system/StreamValidationException.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/StreamValidationException.java b/samza-api/src/main/java/org/apache/samza/system/StreamValidationException.java
new file mode 100644
index 0000000..fef4148
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/system/StreamValidationException.java
@@ -0,0 +1,30 @@
+/*
+ * 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.system;
+
+import org.apache.samza.SamzaException;
+
+
+public class StreamValidationException extends SamzaException {
+ private static final long serialVersionUID = 1L;
+
+ public StreamValidationException(String s) {
+ super(s);
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/7e7747fb/samza-api/src/main/java/org/apache/samza/system/SystemAdmin.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/SystemAdmin.java b/samza-api/src/main/java/org/apache/samza/system/SystemAdmin.java
index ef99893..b180712 100644
--- a/samza-api/src/main/java/org/apache/samza/system/SystemAdmin.java
+++ b/samza-api/src/main/java/org/apache/samza/system/SystemAdmin.java
@@ -28,7 +28,6 @@ import java.util.Set;
* utility methods that Samza needs in order to interact with a system.
*/
public interface SystemAdmin {
-
/**
* Fetches the offsets for the messages immediately after the supplied offsets
* for a group of SystemStreamPartitions.
@@ -52,11 +51,12 @@ public interface SystemAdmin {
/**
* An API to create a change log stream
- *
+ *
* @param streamName
* The name of the stream to be created in the underlying stream
* @param numOfPartitions
* The number of partitions in the changelog stream
+ * @deprecated since 0.12.1, use {@link #createStream(StreamSpec)}
*/
void createChangelogStream(String streamName, int numOfPartitions);
@@ -67,6 +67,7 @@ public interface SystemAdmin {
* The name of the stream to be created in the underlying stream
* @param numOfPartitions
* The number of partitions in the changelog stream
+ * @deprecated since 0.12.1, use {@link #validateStream(StreamSpec)}
*/
void validateChangelogStream(String streamName, int numOfPartitions);
@@ -76,6 +77,7 @@ public interface SystemAdmin {
*
* @param streamName
* The name of the coordinator stream to create.
+ * @deprecated since 0.12.1, use {@link #createStream(StreamSpec)}
*/
void createCoordinatorStream(String streamName);
@@ -89,4 +91,27 @@ public interface SystemAdmin {
* @return -1 if offset1 < offset2; 0 if offset1 == offset2; 1 if offset1 > offset2. Null if not comparable
*/
Integer offsetComparator(String offset1, String offset2);
+
+ /**
+ * Create a stream described by the spec.
+ *
+ * @param streamSpec The spec, or blueprint from which the physical stream will be created on the system.
+ * @return {@code true} if the stream was actually created and not pre-existing.
+ * {@code false} if the stream was pre-existing.
+ * A RuntimeException will be thrown if creation fails.
+ */
+ default boolean createStream(StreamSpec streamSpec) {
+ throw new UnsupportedOperationException();
+ }
+
+ /**
+ * Validates the stream described by the streamSpec on the system.
+ * A {@link StreamValidationException} should be thrown for any validation error.
+ *
+ * @param streamSpec The spec, or blueprint for the physical stream on the system.
+ * @throws StreamValidationException if validation fails.
+ */
+ default void validateStream(StreamSpec streamSpec) throws StreamValidationException {
+ throw new UnsupportedOperationException();
+ }
}
http://git-wip-us.apache.org/repos/asf/samza/blob/7e7747fb/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaStreamSpec.java
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaStreamSpec.java b/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaStreamSpec.java
new file mode 100644
index 0000000..3255f70
--- /dev/null
+++ b/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaStreamSpec.java
@@ -0,0 +1,141 @@
+/*
+ * 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.system.kafka;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+import org.apache.samza.config.KafkaConfig;
+import org.apache.samza.system.StreamSpec;
+
+
+/**
+ * Extends StreamSpec with the ability to easily get the topic replication factor.
+ */
+public class KafkaStreamSpec extends StreamSpec {
+ private static final int DEFAULT_REPLICATION_FACTOR = 2;
+
+ /**
+ * The number of replicas for stream durability.
+ */
+ private final int replicationFactor;
+
+ /**
+ * Convenience method to convert a config map to Properties.
+ * @param map The Map to convert.
+ * @return The Properties instance.
+ */
+ private static Properties mapToProperties(Map<String, String> map) {
+ Properties props = new Properties();
+ props.putAll(map);
+ return props;
+ }
+
+ /**
+ * Convenience method to convert Properties to a config map.
+ * @param properties The Properties to convert.
+ * @return The Map instance.
+ */
+ private static Map<String, String> propertiesToMap(Properties properties) {
+ Map<String, String> map = new HashMap<String, String>();
+ for (final String name: properties.stringPropertyNames()) {
+ map.put(name, properties.getProperty(name));
+ }
+ return map;
+ }
+
+ /**
+ * Converts any StreamSpec to a KafkaStreamSpec.
+ * If the original spec already is a KafkaStreamSpec, it is simply returned.
+ *
+ * @param originalSpec The StreamSpec instance to convert to KafkaStreamSpec.
+ * @return A KafkaStreamSpec instance.
+ */
+ public static KafkaStreamSpec fromSpec(StreamSpec originalSpec) {
+ if (originalSpec instanceof KafkaStreamSpec) {
+ return ((KafkaStreamSpec) originalSpec);
+ }
+
+ int replicationFactor = Integer.parseInt(originalSpec.getOrDefault( KafkaConfig.TOPIC_REPLICATION_FACTOR(),
+ KafkaConfig.TOPIC_DEFAULT_REPLICATION_FACTOR()));
+
+ return new KafkaStreamSpec( originalSpec.getId(),
+ originalSpec.getPhysicalName(),
+ originalSpec.getSystemName(),
+ originalSpec.getPartitionCount(),
+ replicationFactor,
+ mapToProperties(originalSpec.getConfig()));
+ }
+
+ /**
+ * Convenience constructor to create a KafkaStreamSpec with just a topicName, systemName, and partitionCount.
+ *
+ * @param topicName The name of the topic.
+ * @param systemName The name of the System. See {@link org.apache.samza.system.SystemFactory}
+ * @param partitionCount The number of partitions.
+ */
+ public KafkaStreamSpec(String topicName, String systemName, int partitionCount) {
+ this(topicName, topicName, systemName, partitionCount, DEFAULT_REPLICATION_FACTOR, new Properties());
+ }
+
+ /**
+ * Constructs a StreamSpec with a replication factor.
+ *
+ * @param id The application-unique logical identifier for the stream. It is used to distinguish between
+ * streams in a Samza application so it must be unique in the context of one deployable unit.
+ * It does not need to be globally unique or unique with respect to a host.
+ *
+ * @param topicName The physical identifier for the stream. This is the identifier that will be used in remote
+ * systems to identify the stream. In Kafka this would be the topic name whereas in HDFS it
+ * might be a file URN.
+ *
+ * @param systemName The System name on which this stream will exist. Corresponds to a named implementation of the
+ * Samza System abstraction. See {@link org.apache.samza.system.SystemFactory}
+ *
+ * @param partitionCount The number of partitionts for the stream. A value of {@code 1} indicates unpartitioned.
+ *
+ * @param replicationFactor The number of topic replicas in the Kafka cluster for durability.
+ *
+ * @param properties A set of properties for the stream. These may be System-specfic.
+ */
+ public KafkaStreamSpec(String id, String topicName, String systemName, int partitionCount, int replicationFactor,
+ Properties properties) {
+ super(id, topicName, systemName, partitionCount, propertiesToMap(properties));
+
+ if (replicationFactor <= 0) {
+ throw new IllegalArgumentException(
+ String.format("Replication factor %d must be greater than 0.", replicationFactor));
+ }
+ this.replicationFactor = replicationFactor;
+ }
+
+ @Override
+ public StreamSpec copyWithPartitionCount(int partitionCount) {
+ return new KafkaStreamSpec(getId(), getPhysicalName(), getSystemName(), partitionCount, getReplicationFactor(), getProperties());
+ }
+
+ public int getReplicationFactor() {
+ return replicationFactor;
+ }
+
+ public Properties getProperties() {
+ return mapToProperties(getConfig());
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/7e7747fb/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConfig.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConfig.scala b/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConfig.scala
index 770220c..e355e7e 100644
--- a/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConfig.scala
+++ b/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConfig.scala
@@ -24,27 +24,33 @@ import java.util.regex.Pattern
import org.apache.samza.util.Util
import org.apache.samza.util.Logging
+
import scala.collection.JavaConversions._
import kafka.consumer.ConsumerConfig
import java.util.{Properties, UUID}
+
import org.apache.kafka.clients.producer.ProducerConfig
import org.apache.samza.SamzaException
import java.util
+
import scala.collection.JavaConverters._
import org.apache.samza.system.kafka.KafkaSystemFactory
import org.apache.samza.config.SystemConfig.Config2System
import org.apache.kafka.common.serialization.ByteArraySerializer
object KafkaConfig {
+ val TOPIC_REPLICATION_FACTOR = "replication.factor"
+ val TOPIC_DEFAULT_REPLICATION_FACTOR = "2"
+
val REGEX_RESOLVED_STREAMS = "job.config.rewriter.%s.regex"
val REGEX_RESOLVED_SYSTEM = "job.config.rewriter.%s.system"
val REGEX_INHERITED_CONFIG = "job.config.rewriter.%s.config"
val CHECKPOINT_SYSTEM = "task.checkpoint.system"
- val CHECKPOINT_REPLICATION_FACTOR = "task.checkpoint.replication.factor"
+ val CHECKPOINT_REPLICATION_FACTOR = "task.checkpoint." + TOPIC_REPLICATION_FACTOR
val CHECKPOINT_SEGMENT_BYTES = "task.checkpoint.segment.bytes"
- val CHANGELOG_STREAM_REPLICATION_FACTOR = "stores.%s.changelog.replication.factor"
+ val CHANGELOG_STREAM_REPLICATION_FACTOR = "stores.%s.changelog." + TOPIC_REPLICATION_FACTOR
val CHANGELOG_STREAM_KAFKA_SETTINGS = "stores.%s.changelog.kafka."
// The default segment size to use for changelog topics
val CHANGELOG_DEFAULT_SEGMENT_SIZE = "536870912"
@@ -53,20 +59,20 @@ object KafkaConfig {
val CHANGELOG_STREAM_NAMES_REGEX = "stores\\.(.*)\\.changelog$"
/**
- * Defines how low a queue can get for a single system/stream/partition
- * combination before trying to fetch more messages for it.
- */
+ * Defines how low a queue can get for a single system/stream/partition
+ * combination before trying to fetch more messages for it.
+ */
val CONSUMER_FETCH_THRESHOLD = SystemConfig.SYSTEM_PREFIX + "samza.fetch.threshold"
val DEFAULT_CHECKPOINT_SEGMENT_BYTES = 26214400
/**
- * Defines how many bytes to use for the buffered prefetch messages for job as a whole.
- * The bytes for a single system/stream/partition are computed based on this.
- * This fetches wholes messages, hence this bytes limit is a soft one, and the actual usage can be
- * the bytes limit + size of max message in the partition for a given stream.
- * If the value of this property is > 0 then this takes precedence over CONSUMER_FETCH_THRESHOLD config.
- */
+ * Defines how many bytes to use for the buffered prefetch messages for job as a whole.
+ * The bytes for a single system/stream/partition are computed based on this.
+ * This fetches wholes messages, hence this bytes limit is a soft one, and the actual usage can be
+ * the bytes limit + size of max message in the partition for a given stream.
+ * If the value of this property is > 0 then this takes precedence over CONSUMER_FETCH_THRESHOLD config.
+ */
val CONSUMER_FETCH_THRESHOLD_BYTES = SystemConfig.SYSTEM_PREFIX + "samza.fetch.threshold.bytes"
implicit def Config2Kafka(config: Config) = new KafkaConfig(config)
@@ -75,18 +81,23 @@ object KafkaConfig {
class KafkaConfig(config: Config) extends ScalaMapConfig(config) {
// checkpoints
def getCheckpointSystem = getOption(KafkaConfig.CHECKPOINT_SYSTEM)
+
def getCheckpointReplicationFactor() = getOption(KafkaConfig.CHECKPOINT_REPLICATION_FACTOR)
+
def getCheckpointSegmentBytes() = getInt(KafkaConfig.CHECKPOINT_SEGMENT_BYTES, KafkaConfig.DEFAULT_CHECKPOINT_SEGMENT_BYTES)
+
// custom consumer config
def getConsumerFetchThreshold(name: String) = getOption(KafkaConfig.CONSUMER_FETCH_THRESHOLD format name)
+
def getConsumerFetchThresholdBytes(name: String) = getOption(KafkaConfig.CONSUMER_FETCH_THRESHOLD_BYTES format name)
+
def isConsumerFetchThresholdBytesEnabled(name: String): Boolean = getConsumerFetchThresholdBytes(name).getOrElse("-1").toLong > 0
/**
- * Returns a map of topic -> fetch.message.max.bytes value for all streams that
- * are defined with this property in the config.
- */
+ * Returns a map of topic -> fetch.message.max.bytes value for all streams that
+ * are defined with this property in the config.
+ */
def getFetchMessageMaxBytesTopics(systemName: String) = {
val subConf = config.subset("systems.%s.streams." format systemName, true)
subConf
@@ -98,9 +109,9 @@ class KafkaConfig(config: Config) extends ScalaMapConfig(config) {
}
/**
- * Returns a map of topic -> auto.offset.reset value for all streams that
- * are defined with this property in the config.
- */
+ * Returns a map of topic -> auto.offset.reset value for all streams that
+ * are defined with this property in the config.
+ */
def getAutoOffsetResetTopics(systemName: String) = {
val subConf = config.subset("systems.%s.streams." format systemName, true)
subConf
@@ -113,8 +124,11 @@ class KafkaConfig(config: Config) extends ScalaMapConfig(config) {
// regex resolver
def getRegexResolvedStreams(rewriterName: String) = getOption(KafkaConfig.REGEX_RESOLVED_STREAMS format rewriterName)
+
def getRegexResolvedSystem(rewriterName: String) = getOption(KafkaConfig.REGEX_RESOLVED_SYSTEM format rewriterName)
+
def getRegexResolvedInheritedConfig(rewriterName: String) = config.subset((KafkaConfig.REGEX_INHERITED_CONFIG format rewriterName) + ".", true)
+
def getChangelogStreamReplicationFactor(name: String) = getOption(KafkaConfig.CHANGELOG_STREAM_REPLICATION_FACTOR format name)
// The method returns a map of storenames to changelog topic names, which are configured to use kafka as the changelog stream
@@ -124,16 +138,16 @@ class KafkaConfig(config: Config) extends ScalaMapConfig(config) {
val storageConfig = new StorageConfig(config)
val pattern = Pattern.compile(KafkaConfig.CHANGELOG_STREAM_NAMES_REGEX)
- for((changelogConfig, cn) <- changelogConfigs){
+ for ((changelogConfig, cn) <- changelogConfigs) {
// Lookup the factory for this particular stream and verify if it's a kafka system
val matcher = pattern.matcher(changelogConfig)
- val storeName = if(matcher.find()) matcher.group(1) else throw new SamzaException("Unable to find store name in the changelog configuration: " + changelogConfig + " with SystemStream: " + cn)
+ val storeName = if (matcher.find()) matcher.group(1) else throw new SamzaException("Unable to find store name in the changelog configuration: " + changelogConfig + " with SystemStream: " + cn)
val changelogName = storageConfig.getChangelogStream(storeName).getOrElse(throw new SamzaException("unable to get SystemStream for store:" + changelogConfig));
val systemStream = Util.getSystemStreamFromNames(changelogName)
val factoryName = config.getSystemFactory(systemStream.getSystem).getOrElse(new SamzaException("Unable to determine factory for system: " + systemStream.getSystem))
- if(classOf[KafkaSystemFactory].getCanonicalName == factoryName){
+ if (classOf[KafkaSystemFactory].getCanonicalName == factoryName) {
storeToChangelog += storeName -> systemStream.getStream
}
}
@@ -147,16 +161,22 @@ class KafkaConfig(config: Config) extends ScalaMapConfig(config) {
kafkaChangeLogProperties.setProperty("cleanup.policy", "compact")
kafkaChangeLogProperties.setProperty("segment.bytes", KafkaConfig.CHANGELOG_DEFAULT_SEGMENT_SIZE)
kafkaChangeLogProperties.setProperty("delete.retention.ms", String.valueOf(new StorageConfig(config).getChangeLogDeleteRetentionInMs(name)))
- filteredConfigs.foreach{kv => kafkaChangeLogProperties.setProperty(kv._1, kv._2)}
+ filteredConfigs.foreach { kv => kafkaChangeLogProperties.setProperty(kv._1, kv._2) }
kafkaChangeLogProperties
}
+ def getTopicKafkaProperties(systemName: String, streamName: String) = {
+ val filteredConfigs = config.subset(StreamConfig.STREAM_PREFIX format(systemName, streamName), true)
+ val topicProperties = new Properties
+ filteredConfigs.foreach { kv => topicProperties.setProperty(kv._1, kv._2) }
+ topicProperties
+ }
+
// kafka config
- def getKafkaSystemConsumerConfig(
- systemName: String,
- clientId: String,
- groupId: String = "undefined-samza-consumer-group-%s" format UUID.randomUUID.toString,
- injectedProps: Map[String, String] = Map()) = {
+ def getKafkaSystemConsumerConfig( systemName: String,
+ clientId: String,
+ groupId: String = "undefined-samza-consumer-group-%s" format UUID.randomUUID.toString,
+ injectedProps: Map[String, String] = Map()) = {
val subConf = config.subset("systems.%s.consumer." format systemName, true)
val consumerProps = new Properties()
@@ -167,10 +187,9 @@ class KafkaConfig(config: Config) extends ScalaMapConfig(config) {
new ConsumerConfig(consumerProps)
}
- def getKafkaSystemProducerConfig(
- systemName: String,
- clientId: String,
- injectedProps: Map[String, String] = Map()) = {
+ def getKafkaSystemProducerConfig( systemName: String,
+ clientId: String,
+ injectedProps: Map[String, String] = Map()) = {
val subConf = config.subset("systems.%s.producer." format systemName, true)
val producerProps = new util.HashMap[String, Object]()
@@ -197,45 +216,45 @@ class KafkaProducerConfig(val systemName: String,
val producerProperties: java.util.Map[String, Object] = new util.HashMap[String, Object]()
producerProperties.putAll(properties)
- if(!producerProperties.containsKey(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG)) {
- debug("%s undefined. Defaulting to %s." format (ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, byteArraySerializerClassName))
+ if (!producerProperties.containsKey(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG)) {
+ debug("%s undefined. Defaulting to %s." format(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, byteArraySerializerClassName))
producerProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, byteArraySerializerClassName)
}
- if(!producerProperties.containsKey(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG)) {
- debug("%s undefined. Defaulting to %s." format (ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, byteArraySerializerClassName))
+ if (!producerProperties.containsKey(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG)) {
+ debug("%s undefined. Defaulting to %s." format(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, byteArraySerializerClassName))
producerProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, byteArraySerializerClassName)
}
- if(producerProperties.containsKey(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION)
- && producerProperties.get(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION).asInstanceOf[String].toInt > MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION_DEFAULT) {
- warn("Setting '%s' to a value other than %d does not guarantee message ordering because new messages will be sent without waiting for previous ones to be acknowledged."
- format (ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION_DEFAULT))
+ if (producerProperties.containsKey(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION)
+ && producerProperties.get(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION).asInstanceOf[String].toInt > MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION_DEFAULT) {
+ warn("Setting '%s' to a value other than %d does not guarantee message ordering because new messages will be sent without waiting for previous ones to be acknowledged."
+ format(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION_DEFAULT))
} else {
producerProperties.put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION_DEFAULT)
}
- if(producerProperties.containsKey(ProducerConfig.RETRIES_CONFIG)
- && producerProperties.get(ProducerConfig.RETRIES_CONFIG).asInstanceOf[String].toInt < RETRIES_DEFAULT) {
- warn("Samza does not provide producer failure handling. Consider setting '%s' to a large value, like Int.MAX." format ProducerConfig.RETRIES_CONFIG)
+ if (producerProperties.containsKey(ProducerConfig.RETRIES_CONFIG)
+ && producerProperties.get(ProducerConfig.RETRIES_CONFIG).asInstanceOf[String].toInt < RETRIES_DEFAULT) {
+ warn("Samza does not provide producer failure handling. Consider setting '%s' to a large value, like Int.MAX." format ProducerConfig.RETRIES_CONFIG)
} else {
// Retries config is set to Max so that when all attempts fail, Samza also fails the send. We do not have any special handler
// for producer failure
producerProperties.put(ProducerConfig.RETRIES_CONFIG, RETRIES_DEFAULT)
}
-
+
producerProperties
}
- val reconnectIntervalMs = Option(properties.get(ProducerConfig.RECONNECT_BACKOFF_MS_CONFIG))
- .getOrElse(RECONNECT_BACKOFF_MS_DEFAULT).asInstanceOf[Long]
+ val reconnectIntervalMs = Option(properties.get(ProducerConfig.RECONNECT_BACKOFF_MS_CONFIG))
+ .getOrElse(RECONNECT_BACKOFF_MS_DEFAULT).asInstanceOf[Long]
val bootsrapServers = {
- if(properties.containsKey("metadata.broker.list"))
+ if (properties.containsKey("metadata.broker.list"))
warn("Kafka producer configuration contains 'metadata.broker.list'. This configuration is deprecated . Samza has been upgraded " +
- "to use Kafka's new producer API. Please update your configurations based on the documentation at http://kafka.apache.org/documentation.html#newproducerconfigs")
+ "to use Kafka's new producer API. Please update your configurations based on the documentation at http://kafka.apache.org/documentation.html#newproducerconfigs")
Option(properties.get("bootstrap.servers"))
- .getOrElse(throw new SamzaException("No bootstrap servers defined in config for %s." format systemName))
- .asInstanceOf[String]
+ .getOrElse(throw new SamzaException("No bootstrap servers defined in config for %s." format systemName))
+ .asInstanceOf[String]
}
}
http://git-wip-us.apache.org/repos/asf/samza/blob/7e7747fb/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemAdmin.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemAdmin.scala b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemAdmin.scala
index 955fa44..309b653 100644
--- a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemAdmin.scala
+++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemAdmin.scala
@@ -20,23 +20,21 @@
package org.apache.samza.system.kafka
import java.util
+import java.util.{Properties, UUID}
-import org.apache.samza.Partition
-import org.apache.samza.SamzaException
-import org.apache.samza.system.{ExtendedSystemAdmin, SystemStreamMetadata, SystemStreamPartition}
-import org.apache.samza.util.{ ClientUtilTopicMetadataStore, ExponentialSleepStrategy, Logging, KafkaUtil }
+import kafka.admin.AdminUtils
import kafka.api._
-import kafka.consumer.SimpleConsumer
-import kafka.common.{ TopicExistsException, TopicAndPartition }
-import kafka.consumer.ConsumerConfig
+import kafka.common.{TopicAndPartition, TopicExistsException}
+import kafka.consumer.{ConsumerConfig, SimpleConsumer}
import kafka.utils.ZkUtils
-import java.util.{ Properties, UUID }
+import org.apache.samza.config.KafkaConfig
+import org.apache.samza.system.SystemStreamMetadata.SystemStreamPartitionMetadata
+import org.apache.samza.system._
+import org.apache.samza.util.{ClientUtilTopicMetadataStore, ExponentialSleepStrategy, KafkaUtil, Logging}
+import org.apache.samza.{Partition, SamzaException}
+
import scala.collection.JavaConversions
import scala.collection.JavaConversions._
-import org.apache.samza.system.SystemStreamMetadata.{OffsetType, SystemStreamPartitionMetadata}
-import kafka.consumer.ConsumerConfig
-import kafka.admin.AdminUtils
-import org.apache.samza.util.KafkaUtil
object KafkaSystemAdmin extends Logging {
@@ -269,12 +267,12 @@ class KafkaSystemAdmin(
}
/**
- * Returns the newest offset for the specified SSP.
- * This method is fast and targeted. It minimizes the number of kafka requests.
- * It does not retry indefinitely if there is any failure.
- * It returns null if the topic is empty. To get the offsets for *all*
- * partitions, it would be more efficient to call getSystemStreamMetadata
- */
+ * Returns the newest offset for the specified SSP.
+ * This method is fast and targeted. It minimizes the number of kafka requests.
+ * It does not retry indefinitely if there is any failure.
+ * It returns null if the topic is empty. To get the offsets for *all*
+ * partitions, it would be more efficient to call getSystemStreamMetadata
+ */
override def getNewestOffset(ssp: SystemStreamPartition, maxRetries: Integer) = {
debug("Fetching newest offset for: %s" format ssp)
var offset: String = null
@@ -334,34 +332,14 @@ class KafkaSystemAdmin(
override def createCoordinatorStream(streamName: String) {
info("Attempting to create coordinator stream %s." format streamName)
- new ExponentialSleepStrategy(initialDelayMs = 500).run(
- loop => {
- val zkClient = connectZk()
- try {
- AdminUtils.createTopic(
- zkClient,
- streamName,
- 1, // Always one partition for coordinator stream.
- coordinatorStreamReplicationFactor,
- coordinatorStreamProperties)
- } finally {
- zkClient.close
- }
- info("Created coordinator stream %s." format streamName)
- loop.done
- },
+ val streamSpec = new KafkaStreamSpec(streamName, streamName, systemName, 1, coordinatorStreamReplicationFactor, coordinatorStreamProperties)
- (exception, loop) => {
- exception match {
- case e: TopicExistsException =>
- info("Coordinator stream %s already exists." format streamName)
- loop.done
- case e: Exception =>
- warn("Failed to create topic %s: %s. Retrying." format (streamName, e))
- debug("Exception detail:", e)
- }
- })
+ if (createStream(streamSpec)) {
+ info("Created coordinator stream %s." format streamName)
+ } else {
+ info("Coordinator stream %s already exists." format streamName)
+ }
}
/**
@@ -435,44 +413,57 @@ class KafkaSystemAdmin(
offsets
}
- private def createTopicInKafka(topicName: String, numKafkaChangelogPartitions: Int) {
- val retryBackoff: ExponentialSleepStrategy = new ExponentialSleepStrategy
- info("Attempting to create change log topic %s." format topicName)
- info("Using partition count " + numKafkaChangelogPartitions + " for creating change log topic")
- val topicMetaInfo = topicMetaInformation.getOrElse(topicName, throw new KafkaChangelogException("Unable to find topic information for topic " + topicName))
- retryBackoff.run(
+ /**
+ * @inheritdoc
+ */
+ override def createStream(spec: StreamSpec): Boolean = {
+ val kSpec = KafkaStreamSpec.fromSpec(spec);
+ var streamCreated = false
+
+ new ExponentialSleepStrategy(initialDelayMs = 500).run(
loop => {
val zkClient = connectZk()
try {
AdminUtils.createTopic(
zkClient,
- topicName,
- numKafkaChangelogPartitions,
- topicMetaInfo.replicationFactor,
- topicMetaInfo.kafkaProps)
+ kSpec.getPhysicalName,
+ kSpec.getPartitionCount,
+ kSpec.getReplicationFactor,
+ kSpec.getProperties)
} finally {
zkClient.close
}
- info("Created changelog topic %s." format topicName)
+ streamCreated = true
loop.done
},
(exception, loop) => {
exception match {
case e: TopicExistsException =>
- info("Changelog topic %s already exists." format topicName)
+ streamCreated = false
loop.done
case e: Exception =>
- warn("Failed to create topic %s: %s. Retrying." format (topicName, e))
+ warn("Failed to create topic %s: %s. Retrying." format (spec.getPhysicalName, e))
debug("Exception detail:", e)
}
})
+
+ streamCreated
}
- private def validateTopicInKafka(topicName: String, numKafkaChangelogPartitions: Int) {
+ /**
+ * @inheritdoc
+ *
+ * Validates a stream in Kafka. Should not be called before createStream(),
+ * since ClientUtils.fetchTopicMetadata(), used by different Kafka clients,
+ * is not read-only and will auto-create a new topic.
+ */
+ override def validateStream(spec: StreamSpec): Unit = {
+ val topicName = spec.getPhysicalName
+ info("Validating topic %s." format topicName)
+
val retryBackoff: ExponentialSleepStrategy = new ExponentialSleepStrategy
- info("Validating changelog topic %s." format topicName)
var metadataTTL = Long.MaxValue // Trust the cache until we get an exception
retryBackoff.run(
loop => {
@@ -482,17 +473,17 @@ class KafkaSystemAdmin(
KafkaUtil.maybeThrowException(topicMetadata.errorCode)
val partitionCount = topicMetadata.partitionsMetadata.length
- if (partitionCount < numKafkaChangelogPartitions) {
- throw new KafkaChangelogException("Changelog topic validation failed for topic %s because partition count %s did not match expected partition count of %d" format (topicName, topicMetadata.partitionsMetadata.length, numKafkaChangelogPartitions))
+ if (partitionCount != spec.getPartitionCount) {
+ throw new StreamValidationException("Topic validation failed for topic %s because partition count %s did not match expected partition count of %d" format (topicName, topicMetadata.partitionsMetadata.length, spec.getPartitionCount))
}
- info("Successfully validated changelog topic %s." format topicName)
+ info("Successfully validated topic %s." format topicName)
loop.done
},
(exception, loop) => {
exception match {
- case e: KafkaChangelogException => throw e
+ case e: StreamValidationException => throw e
case e: Exception =>
warn("While trying to validate topic %s: %s. Retrying." format (topicName, e))
debug("Exception detail:", e)
@@ -502,24 +493,32 @@ class KafkaSystemAdmin(
}
/**
- * Exception to be thrown when the change log stream creation or validation has failed
- */
+ * Exception to be thrown when the change log stream creation or validation has failed
+ */
class KafkaChangelogException(s: String, t: Throwable) extends SamzaException(s, t) {
def this(s: String) = this(s, null)
}
override def createChangelogStream(topicName: String, numKafkaChangelogPartitions: Int) = {
- createTopicInKafka(topicName, numKafkaChangelogPartitions)
- validateChangelogStream(topicName, numKafkaChangelogPartitions)
+ val topicMeta = topicMetaInformation.getOrElse(topicName, throw new KafkaChangelogException("Unable to find topic information for topic " + topicName))
+ val spec = new KafkaStreamSpec(topicName, topicName, systemName, numKafkaChangelogPartitions, topicMeta.replicationFactor, topicMeta.kafkaProps)
+
+ if (createStream(spec)) {
+ info("Created changelog stream %s." format topicName)
+ } else {
+ info("Changelog stream %s already exists." format topicName)
+ }
+
+ validateStream(spec)
}
/**
- * Validates change log stream in Kafka. Should not be called before createChangelogStream(),
- * since ClientUtils.fetchTopicMetadata(), used by different Kafka clients, is not read-only and
- * will auto-create a new topic.
- */
+ * Validates a stream in Kafka. Should not be called before createStream(),
+ * since ClientUtils.fetchTopicMetadata(), used by different Kafka clients, is not read-only and
+ * will auto-create a new topic.
+ */
override def validateChangelogStream(topicName: String, numKafkaChangelogPartitions: Int) = {
- validateTopicInKafka(topicName, numKafkaChangelogPartitions)
+ validateStream(new KafkaStreamSpec(topicName, systemName, numKafkaChangelogPartitions))
}
/**
http://git-wip-us.apache.org/repos/asf/samza/blob/7e7747fb/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaSystemAdminJava.java
----------------------------------------------------------------------
diff --git a/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaSystemAdminJava.java b/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaSystemAdminJava.java
new file mode 100644
index 0000000..a786468
--- /dev/null
+++ b/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaSystemAdminJava.java
@@ -0,0 +1,145 @@
+/*
+ * 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.system.kafka;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+import org.apache.samza.system.StreamSpec;
+import org.apache.samza.system.StreamValidationException;
+import org.apache.samza.system.SystemAdmin;
+import org.apache.samza.util.Util;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
+
+import static org.junit.Assert.*;
+
+
+public class TestKafkaSystemAdminJava extends TestKafkaSystemAdmin {
+
+ KafkaSystemAdmin basicSystemAdmin = createSystemAdmin();
+
+
+ @Test
+ public void testCreateCoordinatorStreamDelegatesToCreateStream() {
+ KafkaSystemAdmin systemAdmin = createSystemAdmin();//coordProps, 3, new scala.collection.immutable.HashMap<>(), 1000);
+ SystemAdmin admin = Mockito.spy(systemAdmin);
+ StreamSpec spec = new StreamSpec("testId", "testCoordinatorStream", "testSystem");
+
+ admin.createCoordinatorStream(spec.getPhysicalName());
+ admin.validateStream(spec);
+
+ Mockito.verify(admin).createStream(Mockito.any());
+ }
+
+ @Test
+ public void testCreateChangelogStreamDelegatesToCreateStream() {
+ final String STREAM = "testChangeLogStream";
+ final int PARTITIONS = 12;
+ final int REP_FACTOR = 3;
+
+ Properties coordProps = new Properties();
+ Properties changeLogProps = new Properties();
+ changeLogProps.setProperty("cleanup.policy", "compact");
+ changeLogProps.setProperty("segment.bytes", "139");
+ Map<String, ChangelogInfo> changeLogMap = new HashMap<>();
+ changeLogMap.put(STREAM, new ChangelogInfo(REP_FACTOR, changeLogProps));
+
+ SystemAdmin admin = Mockito.spy(createSystemAdmin(coordProps, 3, Util.javaMapAsScalaMap(changeLogMap)));
+ StreamSpec spec = new StreamSpec(STREAM, STREAM, SYSTEM(), PARTITIONS);
+ admin.createChangelogStream(STREAM, PARTITIONS);
+ admin.validateStream(spec);
+
+ ArgumentCaptor<StreamSpec> specCaptor = ArgumentCaptor.forClass(StreamSpec.class);
+ Mockito.verify(admin).createStream(specCaptor.capture());
+
+ StreamSpec internalSpec = specCaptor.getValue();
+ assertTrue(internalSpec instanceof KafkaStreamSpec); // KafkaStreamSpec is used to carry replication factor
+ assertEquals(STREAM, internalSpec.getId());
+ assertEquals(SYSTEM(), internalSpec.getSystemName());
+ assertEquals(STREAM, internalSpec.getPhysicalName());
+ assertEquals(REP_FACTOR, ((KafkaStreamSpec) internalSpec).getReplicationFactor());
+ assertEquals(PARTITIONS, internalSpec.getPartitionCount());
+ assertEquals(changeLogProps, ((KafkaStreamSpec) internalSpec).getProperties());
+ }
+
+ @Test
+ public void testValidateChangelogStreamDelegatesToValidateStream() {
+ final String STREAM = "testChangeLogValidate";
+ Properties coordProps = new Properties();
+ Map<String, ChangelogInfo> changeLogMap = new HashMap<>();
+ changeLogMap.put(STREAM, new ChangelogInfo(3, new Properties()));
+
+ KafkaSystemAdmin systemAdmin = createSystemAdmin(coordProps, 3, Util.javaMapAsScalaMap(changeLogMap));
+ SystemAdmin admin = Mockito.spy(systemAdmin);
+ StreamSpec spec = new StreamSpec("testId", STREAM, "testSystem", 12);
+
+ admin.createChangelogStream(spec.getPhysicalName(), spec.getPartitionCount());
+ admin.validateStream(spec);
+ admin.validateChangelogStream(spec.getPhysicalName(), spec.getPartitionCount());
+
+ Mockito.verify(admin).createStream(Mockito.any());
+ Mockito.verify(admin, Mockito.times(3)).validateStream(Mockito.any());
+ }
+
+ @Test
+ public void testCreateStream() {
+ SystemAdmin admin = this.basicSystemAdmin;
+ StreamSpec spec = new StreamSpec("testId", "testStream", "testSystem", 8);
+
+ assertTrue("createStream should return true if the stream does not exist and then is created.", admin.createStream(spec));
+ admin.validateStream(spec);
+
+ assertFalse("createStream should return false if the stream already exists.", admin.createStream(spec));
+ }
+
+ @Test(expected = StreamValidationException.class)
+ public void testValidateStreamDoesNotExist() {
+ SystemAdmin admin = this.basicSystemAdmin;
+
+ StreamSpec spec = new StreamSpec("testId", "testStreamNameExist", "testSystem", 8);
+
+ admin.validateStream(spec);
+ }
+
+ @Test(expected = StreamValidationException.class)
+ public void testValidateStreamWrongPartitionCount() {
+ SystemAdmin admin = this.basicSystemAdmin;
+ StreamSpec spec1 = new StreamSpec("testId", "testStreamPartition", "testSystem", 8);
+ StreamSpec spec2 = new StreamSpec("testId", "testStreamPartition", "testSystem", 4);
+
+ assertTrue("createStream should return true if the stream does not exist and then is created.", admin.createStream(spec1));
+
+ admin.validateStream(spec2);
+ }
+
+ @Test(expected = StreamValidationException.class)
+ public void testValidateStreamWrongName() {
+ SystemAdmin admin = this.basicSystemAdmin;
+ StreamSpec spec1 = new StreamSpec("testId", "testStreamName1", "testSystem", 8);
+ StreamSpec spec2 = new StreamSpec("testId", "testStreamName2", "testSystem", 8);
+
+ assertTrue("createStream should return true if the stream does not exist and then is created.", admin.createStream(spec1));
+
+ admin.validateStream(spec2);
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/7e7747fb/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemAdmin.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemAdmin.scala b/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemAdmin.scala
index 0e3c9b5..be7db97 100644
--- a/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemAdmin.scala
+++ b/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemAdmin.scala
@@ -21,18 +21,16 @@
package org.apache.samza.system.kafka
-import java.util
-import java.util.Properties
+import java.util.{Properties, UUID}
import kafka.admin.AdminUtils
import kafka.common.{ErrorMapping, LeaderNotAvailableException}
import kafka.consumer.{Consumer, ConsumerConfig}
-import kafka.server.{KafkaConfig, KafkaServer}
-import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
-import kafka.utils.{TestUtils, ZkUtils}
import kafka.integration.KafkaServerTestHarness
+import kafka.server.KafkaConfig
+import kafka.utils.{TestUtils, ZkUtils}
+import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
import org.apache.kafka.common.security.JaasUtils
-
import org.apache.samza.Partition
import org.apache.samza.config.KafkaProducerConfig
import org.apache.samza.system.SystemStreamMetadata.SystemStreamPartitionMetadata
@@ -43,7 +41,9 @@ import org.junit._
import scala.collection.JavaConversions._
-
+/**
+ * README: New tests should be added to the Java tests. See TestKafkaSystemAdminJava
+ */
object TestKafkaSystemAdmin extends KafkaServerTestHarness {
val SYSTEM = "kafka"
@@ -136,6 +136,14 @@ object TestKafkaSystemAdmin extends KafkaServerTestHarness {
Consumer.create(consumerConfig)
}
+ def createSystemAdmin: KafkaSystemAdmin = {
+ new KafkaSystemAdmin(SYSTEM, brokers, connectZk = () => ZkUtils(zkConnect, 6000, 6000, zkSecure))
+ }
+
+ def createSystemAdmin(coordinatorStreamProperties: Properties, coordinatorStreamReplicationFactor: Int, topicMetaInformation: Map[String, ChangelogInfo]): KafkaSystemAdmin = {
+ new KafkaSystemAdmin(SYSTEM, brokers, connectZk = () => ZkUtils(zkConnect, 6000, 6000, zkSecure), coordinatorStreamProperties, coordinatorStreamReplicationFactor, 10000, ConsumerConfig.SocketBufferSize, UUID.randomUUID.toString, topicMetaInformation)
+ }
+
}
/**
@@ -146,7 +154,7 @@ class TestKafkaSystemAdmin {
import TestKafkaSystemAdmin._
// Provide a random zkAddress, the system admin tries to connect only when a topic is created/validated
- val systemAdmin = new KafkaSystemAdmin(SYSTEM, brokers, connectZk = () => ZkUtils(zkConnect, 6000, 6000, zkSecure))
+ val systemAdmin = createSystemAdmin
@Test
def testShouldAssembleMetadata {
[04/13] samza git commit: SAMZA-1073: moving all operator classes
into samza-core
Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImpls.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImpls.java b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImpls.java
new file mode 100644
index 0000000..02637a3
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImpls.java
@@ -0,0 +1,235 @@
+/*
+ * 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 java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.StreamGraphImpl;
+import org.apache.samza.operators.TestMessageEnvelope;
+import org.apache.samza.operators.TestMessageStreamImplUtil;
+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.JoinFunction;
+import org.apache.samza.operators.functions.PartialJoinFunction;
+import org.apache.samza.operators.functions.SinkFunction;
+import org.apache.samza.operators.spec.OperatorSpec;
+import org.apache.samza.operators.spec.WindowOperatorSpec;
+import org.apache.samza.operators.spec.PartialJoinOperatorSpec;
+import org.apache.samza.operators.spec.SinkOperatorSpec;
+import org.apache.samza.operators.spec.StreamOperatorSpec;
+import org.apache.samza.operators.windows.Windows;
+import org.apache.samza.operators.windows.internal.WindowInternal;
+import org.apache.samza.task.TaskContext;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.lang.reflect.Field;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.Set;
+
+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;
+ Method createOpMethod = null;
+ Method createOpsMethod = null;
+
+ @Before
+ public void prep() throws NoSuchFieldException, NoSuchMethodException {
+ nextOperatorsField = OperatorImpl.class.getDeclaredField("nextOperators");
+ nextOperatorsField.setAccessible(true);
+
+ createOpMethod = OperatorGraph.class.getDeclaredMethod("createOperatorImpl", MessageStreamImpl.class,
+ OperatorSpec.class, Config.class, TaskContext.class);
+ createOpMethod.setAccessible(true);
+
+ createOpsMethod = OperatorGraph.class.getDeclaredMethod("createOperatorImpls", MessageStreamImpl.class, Config.class, TaskContext.class);
+ createOpsMethod.setAccessible(true);
+ }
+
+ @Test
+ public void testCreateOperator() throws NoSuchFieldException, IllegalAccessException, InvocationTargetException {
+ // get window operator
+ WindowOperatorSpec mockWnd = mock(WindowOperatorSpec.class);
+ WindowInternal<TestMessageEnvelope, String, Integer> windowInternal = new WindowInternal<>(null, null, null, null);
+ when(mockWnd.getWindow()).thenReturn(windowInternal);
+ MessageStreamImpl<TestMessageEnvelope> mockStream = mock(MessageStreamImpl.class);
+ Config mockConfig = mock(Config.class);
+ TaskContext mockContext = mock(TaskContext.class);
+
+ OperatorGraph opGraph = new OperatorGraph();
+ OperatorImpl<TestMessageEnvelope, ? extends MessageEnvelope> opImpl = (OperatorImpl<TestMessageEnvelope, ? extends MessageEnvelope>)
+ createOpMethod.invoke(opGraph, mockStream, mockWnd, mockConfig, mockContext);
+ assertTrue(opImpl instanceof WindowOperatorImpl);
+ Field wndInternalField = WindowOperatorImpl.class.getDeclaredField("window");
+ wndInternalField.setAccessible(true);
+ WindowInternal wndInternal = (WindowInternal) wndInternalField.get(opImpl);
+ assertEquals(wndInternal, windowInternal);
+
+ // get simple operator
+ StreamOperatorSpec<TestMessageEnvelope, TestOutputMessageEnvelope> mockSimpleOp = mock(StreamOperatorSpec.class);
+ FlatMapFunction<TestMessageEnvelope, TestOutputMessageEnvelope> mockTxfmFn = mock(FlatMapFunction.class);
+ when(mockSimpleOp.getTransformFn()).thenReturn(mockTxfmFn);
+ opImpl = (OperatorImpl<TestMessageEnvelope, ? extends MessageEnvelope>) createOpMethod.invoke(opGraph, mockStream, mockSimpleOp, mockConfig, mockContext);
+ 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 = (OperatorImpl<TestMessageEnvelope, ? extends MessageEnvelope>) createOpMethod.invoke(opGraph, mockStream, sinkOp, mockConfig, mockContext);
+ 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);
+ PartialJoinFunction<String, TestMessageEnvelope, TestMessageEnvelope, TestOutputMessageEnvelope> joinFn = mock(PartialJoinFunction.class);
+ when(joinOp.getTransformFn()).thenReturn(joinFn);
+ opImpl = (OperatorImpl<TestMessageEnvelope, ? extends MessageEnvelope>) createOpMethod.invoke(opGraph, mockStream, joinOp, mockConfig, mockContext);
+ assertTrue(opImpl instanceof PartialJoinOperatorImpl);
+ }
+
+ @Test
+ public void testEmptyChain() throws InvocationTargetException, IllegalAccessException {
+ // test creation of empty chain
+ MessageStreamImpl<TestMessageEnvelope> testStream = mock(MessageStreamImpl.class);
+ TaskContext mockContext = mock(TaskContext.class);
+ Config mockConfig = mock(Config.class);
+ OperatorGraph opGraph = new OperatorGraph();
+ RootOperatorImpl operatorChain = (RootOperatorImpl) createOpsMethod.invoke(opGraph, testStream, mockConfig, mockContext);
+ assertTrue(operatorChain != null);
+ }
+
+ @Test
+ public void testLinearChain() throws IllegalAccessException, InvocationTargetException {
+ // test creation of linear chain
+ StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+ MessageStreamImpl<TestMessageEnvelope> testInput = TestMessageStreamImplUtil.<TestMessageEnvelope>getMessageStreamImpl(mockGraph);
+ TaskContext mockContext = mock(TaskContext.class);
+ Config mockConfig = mock(Config.class);
+ testInput.map(m -> m).window(Windows.keyedSessionWindow(TestMessageEnvelope::getKey, Duration.ofMinutes(10)));
+ OperatorGraph opGraph = new OperatorGraph();
+ RootOperatorImpl operatorChain = (RootOperatorImpl) createOpsMethod.invoke(opGraph, testInput, mockConfig, 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, InvocationTargetException {
+ // test creation of broadcast chain
+ StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+ MessageStreamImpl<TestMessageEnvelope> testInput = TestMessageStreamImplUtil.<TestMessageEnvelope>getMessageStreamImpl(mockGraph);
+ TaskContext mockContext = mock(TaskContext.class);
+ Config mockConfig = mock(Config.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);
+ OperatorGraph opGraph = new OperatorGraph();
+ RootOperatorImpl operatorChain = (RootOperatorImpl) createOpsMethod.invoke(opGraph, testInput, mockConfig, 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, InvocationTargetException {
+ // test creation of join chain
+ StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+ MessageStreamImpl<TestMessageEnvelope> input1 = TestMessageStreamImplUtil.getMessageStreamImpl(mockGraph);
+ MessageStreamImpl<TestMessageEnvelope> input2 = TestMessageStreamImplUtil.getMessageStreamImpl(mockGraph);
+ TaskContext mockContext = mock(TaskContext.class);
+ Config mockConfig = mock(Config.class);
+ input1
+ .join(input2,
+ new JoinFunction<String, TestMessageEnvelope, TestMessageEnvelope, TestOutputMessageEnvelope>() {
+ @Override
+ public TestOutputMessageEnvelope apply(TestMessageEnvelope m1, TestMessageEnvelope m2) {
+ return new TestOutputMessageEnvelope(m1.getKey(), m1.getMessage().getValue().length() + m2.getMessage().getValue().length());
+ }
+
+ @Override
+ public String getFirstKey(TestMessageEnvelope message) {
+ return message.getKey();
+ }
+
+ @Override
+ public String getSecondKey(TestMessageEnvelope message) {
+ return message.getKey();
+ }
+ })
+ .map(m -> m);
+ OperatorGraph opGraph = new OperatorGraph();
+ // now, we create chained operators from each input sources
+ RootOperatorImpl chain1 = (RootOperatorImpl) createOpsMethod.invoke(opGraph, input1, mockConfig, mockContext);
+ RootOperatorImpl chain2 = (RootOperatorImpl) createOpsMethod.invoke(opGraph, input2, mockConfig, 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/8515448a/samza-core/src/test/java/org/apache/samza/operators/impl/TestSinkOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/TestSinkOperatorImpl.java b/samza-core/src/test/java/org/apache/samza/operators/impl/TestSinkOperatorImpl.java
new file mode 100644
index 0000000..ce9fdd2
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/operators/impl/TestSinkOperatorImpl.java
@@ -0,0 +1,50 @@
+/*
+ * 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.config.Config;
+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.TaskContext;
+import org.apache.samza.task.TaskCoordinator;
+import org.junit.Test;
+
+import static org.mockito.Mockito.*;
+
+
+public class TestSinkOperatorImpl {
+
+ @Test
+ public void testSinkOperator() {
+ SinkOperatorSpec<TestOutputMessageEnvelope> sinkOp = mock(SinkOperatorSpec.class);
+ SinkFunction<TestOutputMessageEnvelope> sinkFn = mock(SinkFunction.class);
+ when(sinkOp.getSinkFn()).thenReturn(sinkFn);
+ Config mockConfig = mock(Config.class);
+ TaskContext mockContext = mock(TaskContext.class);
+ SinkOperatorImpl<TestOutputMessageEnvelope> sinkImpl = new SinkOperatorImpl<>(sinkOp, mockConfig, mockContext);
+ TestOutputMessageEnvelope mockMsg = mock(TestOutputMessageEnvelope.class);
+ MessageCollector mockCollector = mock(MessageCollector.class);
+ TaskCoordinator mockCoordinator = mock(TaskCoordinator.class);
+
+ sinkImpl.onNext(mockMsg, mockCollector, mockCoordinator);
+ verify(sinkFn, times(1)).apply(mockMsg, mockCollector, mockCoordinator);
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-core/src/test/java/org/apache/samza/operators/impl/TestStreamOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/TestStreamOperatorImpl.java b/samza-core/src/test/java/org/apache/samza/operators/impl/TestStreamOperatorImpl.java
new file mode 100644
index 0000000..010a210
--- /dev/null
+++ b/samza-core/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 java.util.ArrayList;
+import java.util.Collection;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.MessageStreamImpl;
+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.TaskContext;
+import org.apache.samza.task.TaskCoordinator;
+import org.junit.Test;
+
+import static org.mockito.Mockito.*;
+
+
+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);
+ MessageStreamImpl<TestMessageEnvelope> mockInput = mock(MessageStreamImpl.class);
+ Config mockConfig = mock(Config.class);
+ TaskContext mockContext = mock(TaskContext.class);
+ StreamOperatorImpl<TestMessageEnvelope, TestOutputMessageEnvelope> opImpl = spy(new StreamOperatorImpl<>(mockOp, mockInput, mockConfig, mockContext));
+ 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/8515448a/samza-core/src/test/java/org/apache/samza/operators/spec/TestOperatorSpecs.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/operators/spec/TestOperatorSpecs.java b/samza-core/src/test/java/org/apache/samza/operators/spec/TestOperatorSpecs.java
new file mode 100644
index 0000000..31257a4
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/operators/spec/TestOperatorSpecs.java
@@ -0,0 +1,127 @@
+/*
+ * 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.MessageStreamImpl;
+import org.apache.samza.operators.StreamGraphImpl;
+import org.apache.samza.operators.TestMessageEnvelope;
+import org.apache.samza.operators.TestMessageStreamImplUtil;
+import org.apache.samza.operators.data.MessageEnvelope;
+import org.apache.samza.operators.functions.FlatMapFunction;
+import org.apache.samza.operators.functions.PartialJoinFunction;
+import org.apache.samza.operators.functions.SinkFunction;
+import org.apache.samza.operators.windows.internal.WindowInternal;
+import org.apache.samza.operators.windows.WindowPane;
+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.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+
+
+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));
+ } };
+ MessageStreamImpl<TestMessageEnvelope> mockOutput = mock(MessageStreamImpl.class);
+ StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+ StreamOperatorSpec<MessageEnvelope, TestMessageEnvelope> strmOp = OperatorSpecs.createStreamOperatorSpec(transformFn, mockGraph, mockOutput);
+ assertEquals(strmOp.getTransformFn(), transformFn);
+ assertEquals(strmOp.getNextStream(), mockOutput);
+ }
+
+ @Test
+ public void testGetSinkOperator() {
+ SinkFunction<TestMessageEnvelope> sinkFn = (TestMessageEnvelope message, MessageCollector messageCollector,
+ TaskCoordinator taskCoordinator) -> { };
+ StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+ SinkOperatorSpec<TestMessageEnvelope> sinkOp = OperatorSpecs.createSinkOperatorSpec(sinkFn, mockGraph);
+ assertEquals(sinkOp.getSinkFn(), sinkFn);
+ assertTrue(sinkOp.getNextStream() == null);
+ }
+
+ @Test
+ public void testGetWindowOperator() throws Exception {
+ Function<TestMessageEnvelope, String> keyExtractor = m -> "globalkey";
+ BiFunction<TestMessageEnvelope, Integer, Integer> aggregator = (m, c) -> c + 1;
+
+ //instantiate a window using reflection
+ WindowInternal window = new WindowInternal(null, aggregator, keyExtractor, null);
+
+ StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+ MessageStreamImpl<WindowPane<String, Integer>> mockWndOut = mock(MessageStreamImpl.class);
+ WindowOperatorSpec spec = OperatorSpecs.<TestMessageEnvelope, String, Integer>createWindowOperatorSpec(window, mockGraph, mockWndOut);
+ assertEquals(spec.getWindow(), window);
+ assertEquals(spec.getWindow().getKeyExtractor(), keyExtractor);
+ assertEquals(spec.getWindow().getFoldFunction(), aggregator);
+ }
+
+ @Test
+ public void testGetPartialJoinOperator() {
+ PartialJoinFunction<Object, MessageEnvelope<Object, ?>, MessageEnvelope<Object, ?>, TestMessageEnvelope> merger =
+ new PartialJoinFunction<Object, MessageEnvelope<Object, ?>, MessageEnvelope<Object, ?>, TestMessageEnvelope>() {
+ @Override
+ public TestMessageEnvelope apply(MessageEnvelope<Object, ?> m1, MessageEnvelope<Object, ?> m2) {
+ return new TestMessageEnvelope(m1.getKey().toString(), m2.getMessage().toString(), System.nanoTime());
+ }
+
+ @Override
+ public Object getKey(MessageEnvelope<Object, ?> message) {
+ return message.getKey();
+ }
+
+ @Override
+ public Object getOtherKey(MessageEnvelope<Object, ?> message) {
+ return message.getKey();
+ }
+ };
+
+ StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+ MessageStreamImpl<TestMessageEnvelope> joinOutput = TestMessageStreamImplUtil.<TestMessageEnvelope>getMessageStreamImpl(mockGraph);
+ PartialJoinOperatorSpec<MessageEnvelope<Object, ?>, Object, MessageEnvelope<Object, ?>, TestMessageEnvelope> partialJoin =
+ OperatorSpecs.createPartialJoinOperatorSpec(merger, mockGraph, joinOutput);
+
+ assertEquals(partialJoin.getNextStream(), joinOutput);
+ MessageEnvelope<Object, Object> m = mock(MessageEnvelope.class);
+ MessageEnvelope<Object, Object> s = mock(MessageEnvelope.class);
+ assertEquals(partialJoin.getTransformFn(), merger);
+ }
+
+ @Test
+ public void testGetMergeOperator() {
+ StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
+ MessageStreamImpl<TestMessageEnvelope> output = TestMessageStreamImplUtil.<TestMessageEnvelope>getMessageStreamImpl(mockGraph);
+ StreamOperatorSpec<TestMessageEnvelope, TestMessageEnvelope> mergeOp = OperatorSpecs.createMergeOperatorSpec(mockGraph, 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.getNextStream(), output);
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-operator/README.md
----------------------------------------------------------------------
diff --git a/samza-operator/README.md b/samza-operator/README.md
deleted file mode 100644
index 15d2092..0000000
--- a/samza-operator/README.md
+++ /dev/null
@@ -1,17 +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.
--->
-samza-operator is an experimental module that is under development (SAMZA-552).
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/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
deleted file mode 100644
index 830e4a5..0000000
--- a/samza-operator/src/main/java/org/apache/samza/operators/MessageStreamImpl.java
+++ /dev/null
@@ -1,182 +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 java.util.Collection;
-import java.util.Collections;
-import java.util.function.Function;
-import java.util.HashSet;
-import java.util.Set;
-import org.apache.samza.config.Config;
-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.PartialJoinFunction;
-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.WindowPane;
-import org.apache.samza.operators.windows.internal.WindowInternal;
-import org.apache.samza.task.TaskContext;
-
-
-/**
- * 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 messages in this {@link MessageStream}
- */
-public class MessageStreamImpl<M> implements MessageStream<M> {
- /**
- * The {@link StreamGraphImpl} object that contains this {@link MessageStreamImpl}
- */
- private final StreamGraphImpl graph;
-
- /**
- * The set of operators that consume the messages in this {@link MessageStream}
- */
- private final Set<OperatorSpec> registeredOperatorSpecs = new HashSet<>();
-
- /**
- * Default constructor
- *
- * @param graph the {@link StreamGraphImpl} object that this stream belongs to
- */
- MessageStreamImpl(StreamGraphImpl graph) {
- this.graph = graph;
- }
-
- @Override public <TM> MessageStream<TM> map(MapFunction<M, TM> mapFn) {
- OperatorSpec<TM> op = OperatorSpecs.<M, TM>createMapOperatorSpec(mapFn, this.graph, new MessageStreamImpl<>(this.graph));
- this.registeredOperatorSpecs.add(op);
- return op.getNextStream();
- }
-
- @Override public MessageStream<M> filter(FilterFunction<M> filterFn) {
- OperatorSpec<M> op = OperatorSpecs.<M>createFilterOperatorSpec(filterFn, this.graph, new MessageStreamImpl<>(this.graph));
- this.registeredOperatorSpecs.add(op);
- return op.getNextStream();
- }
-
- @Override
- public <TM> MessageStream<TM> flatMap(FlatMapFunction<M, TM> flatMapFn) {
- OperatorSpec<TM> op = OperatorSpecs.createStreamOperatorSpec(flatMapFn, this.graph, new MessageStreamImpl<>(this.graph));
- this.registeredOperatorSpecs.add(op);
- return op.getNextStream();
- }
-
- @Override
- public void sink(SinkFunction<M> sinkFn) {
- this.registeredOperatorSpecs.add(OperatorSpecs.createSinkOperatorSpec(sinkFn, this.graph));
- }
-
- @Override public void sendTo(OutputStream<M> stream) {
- this.registeredOperatorSpecs.add(OperatorSpecs.createSendToOperatorSpec(stream.getSinkFunction(), this.graph, stream));
- }
-
- @Override
- public <K, WV> MessageStream<WindowPane<K, WV>> window(Window<M, K, WV> window) {
- OperatorSpec<WindowPane<K, WV>> wndOp = OperatorSpecs.createWindowOperatorSpec((WindowInternal<M, K, WV>) window,
- this.graph, new MessageStreamImpl<>(this.graph));
- this.registeredOperatorSpecs.add(wndOp);
- return wndOp.getNextStream();
- }
-
- @Override public <K, OM, RM> MessageStream<RM> join(MessageStream<OM> otherStream, JoinFunction<K, M, OM, RM> joinFn) {
- MessageStreamImpl<RM> outputStream = new MessageStreamImpl<>(this.graph);
-
- PartialJoinFunction<K, M, OM, RM> parJoin1 = new PartialJoinFunction<K, M, OM, RM>() {
- @Override
- public RM apply(M m1, OM om) {
- return joinFn.apply(m1, om);
- }
-
- @Override
- public K getKey(M message) {
- return joinFn.getFirstKey(message);
- }
-
- @Override
- public K getOtherKey(OM message) {
- return joinFn.getSecondKey(message);
- }
-
- @Override
- public void init(Config config, TaskContext context) {
- joinFn.init(config, context);
- }
- };
-
- PartialJoinFunction<K, OM, M, RM> parJoin2 = new PartialJoinFunction<K, OM, M, RM>() {
- @Override
- public RM apply(OM m1, M m) {
- return joinFn.apply(m, m1);
- }
-
- @Override
- public K getKey(OM message) {
- return joinFn.getSecondKey(message);
- }
-
- @Override
- public K getOtherKey(M message) {
- return joinFn.getFirstKey(message);
- }
- };
-
- // TODO: need to add default store functions for the two partial join functions
-
- ((MessageStreamImpl<OM>) otherStream).registeredOperatorSpecs.add(
- OperatorSpecs.<OM, K, M, RM>createPartialJoinOperatorSpec(parJoin2, this.graph, outputStream));
- this.registeredOperatorSpecs.add(OperatorSpecs.<M, K, OM, RM>createPartialJoinOperatorSpec(parJoin1, this.graph, outputStream));
- return outputStream;
- }
-
- @Override
- public MessageStream<M> merge(Collection<MessageStream<M>> otherStreams) {
- MessageStreamImpl<M> outputStream = new MessageStreamImpl<>(this.graph);
-
- otherStreams.add(this);
- otherStreams.forEach(other -> ((MessageStreamImpl<M>) other).registeredOperatorSpecs.
- add(OperatorSpecs.createMergeOperatorSpec(this.graph, outputStream)));
- return outputStream;
- }
-
- @Override
- public <K> MessageStream<M> partitionBy(Function<M, K> parKeyExtractor) {
- MessageStreamImpl<M> intStream = this.graph.createIntStream(parKeyExtractor);
- OutputStream<M> outputStream = this.graph.getOutputStream(intStream);
- this.registeredOperatorSpecs.add(OperatorSpecs.createPartitionOperatorSpec(outputStream.getSinkFunction(),
- this.graph, outputStream));
- return intStream;
- }
- /**
- * 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/8515448a/samza-operator/src/main/java/org/apache/samza/operators/StreamGraphImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/StreamGraphImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/StreamGraphImpl.java
deleted file mode 100644
index dca3469..0000000
--- a/samza-operator/src/main/java/org/apache/samza/operators/StreamGraphImpl.java
+++ /dev/null
@@ -1,260 +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 java.util.Properties;
-import java.util.function.Function;
-import org.apache.samza.operators.data.MessageEnvelope;
-import org.apache.samza.operators.functions.SinkFunction;
-import org.apache.samza.serializers.Serde;
-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 java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * The implementation of {@link StreamGraph} interface. This class provides implementation of methods to allow users to
- * create system input/output/intermediate streams.
- */
-public class StreamGraphImpl implements StreamGraph {
-
- /**
- * Unique identifier for each {@link org.apache.samza.operators.spec.OperatorSpec} added to transform the {@link MessageEnvelope}
- * in the input {@link MessageStream}s.
- */
- private int opId = 0;
-
- private class InputStreamImpl<K, V, M extends MessageEnvelope<K, V>> extends MessageStreamImpl<M> {
- final StreamSpec spec;
- final Serde<K> keySerde;
- final Serde<V> msgSerde;
-
- InputStreamImpl(StreamGraphImpl graph, StreamSpec streamSpec, Serde<K> keySerde, Serde<V> msgSerde) {
- super(graph);
- this.spec = streamSpec;
- this.keySerde = keySerde;
- this.msgSerde = msgSerde;
- }
-
- StreamSpec getSpec() {
- return this.spec;
- }
-
- }
-
- private class OutputStreamImpl<K, V, M extends MessageEnvelope<K, V>> implements OutputStream<M> {
- final StreamSpec spec;
- final Serde<K> keySerde;
- final Serde<V> msgSerde;
-
- OutputStreamImpl(StreamGraphImpl graph, StreamSpec streamSpec, Serde<K> keySerde, Serde<V> msgSerde) {
- this.spec = streamSpec;
- this.keySerde = keySerde;
- this.msgSerde = msgSerde;
- }
-
- StreamSpec getSpec() {
- return this.spec;
- }
-
- @Override
- public SinkFunction<M> getSinkFunction() {
- return (M message, MessageCollector mc, TaskCoordinator tc) -> {
- // TODO: need to find a way to directly pass in the serde class names
- // mc.send(new OutgoingMessageEnvelope(this.spec.getSystemStream(), this.keySerde.getClass().getName(), this.msgSerde.getClass().getName(),
- // message.getKey(), message.getKey(), message.getMessage()));
- mc.send(new OutgoingMessageEnvelope(this.spec.getSystemStream(), message.getKey(), message.getMessage()));
- };
- }
- }
-
- private class IntermediateStreamImpl<PK, K, V, M extends MessageEnvelope<K, V>> extends InputStreamImpl<K, V, M> implements OutputStream<M> {
- final Function<M, PK> parKeyFn;
-
- /**
- * Default constructor
- *
- * @param graph the {@link StreamGraphImpl} object that this stream belongs to
- */
- IntermediateStreamImpl(StreamGraphImpl graph, StreamSpec streamSpec, Serde<K> keySerde, Serde<V> msgSerde) {
- this(graph, streamSpec, keySerde, msgSerde, null);
- }
-
- IntermediateStreamImpl(StreamGraphImpl graph, StreamSpec streamSpec, Serde<K> keySerde, Serde<V> msgSerde, Function<M, PK> parKeyFn) {
- super(graph, streamSpec, keySerde, msgSerde);
- this.parKeyFn = parKeyFn;
- }
-
- @Override
- public SinkFunction<M> getSinkFunction() {
- return (M message, MessageCollector mc, TaskCoordinator tc) -> {
- // TODO: need to find a way to directly pass in the serde class names
- // mc.send(new OutgoingMessageEnvelope(this.spec.getSystemStream(), this.keySerde.getClass().getName(), this.msgSerde.getClass().getName(),
- // message.getKey(), message.getKey(), message.getMessage()));
- if (this.parKeyFn == null) {
- mc.send(new OutgoingMessageEnvelope(this.spec.getSystemStream(), message.getKey(), message.getMessage()));
- } else {
- // apply partition key function
- mc.send(new OutgoingMessageEnvelope(this.spec.getSystemStream(), this.parKeyFn.apply(message), message.getKey(), message.getMessage()));
- }
- };
- }
- }
-
- /**
- * Maps keeping all {@link SystemStream}s that are input and output of operators in {@link StreamGraphImpl}
- */
- private final Map<SystemStream, MessageStream> inStreams = new HashMap<>();
- private final Map<SystemStream, OutputStream> outStreams = new HashMap<>();
-
- private ContextManager contextManager = new ContextManager() { };
-
- @Override
- public <K, V, M extends MessageEnvelope<K, V>> MessageStream<M> createInStream(StreamSpec streamSpec, Serde<K> keySerde, Serde<V> msgSerde) {
- if (!this.inStreams.containsKey(streamSpec.getSystemStream())) {
- this.inStreams.putIfAbsent(streamSpec.getSystemStream(), new InputStreamImpl<K, V, M>(this, streamSpec, keySerde, msgSerde));
- }
- return this.inStreams.get(streamSpec.getSystemStream());
- }
-
- /**
- * Helper method to be used by {@link MessageStreamImpl} class
- *
- * @param streamSpec the {@link StreamSpec} object defining the {@link SystemStream} as the output
- * @param <M> the type of {@link MessageEnvelope}s in the output {@link SystemStream}
- * @return the {@link MessageStreamImpl} object
- */
- @Override
- public <K, V, M extends MessageEnvelope<K, V>> OutputStream<M> createOutStream(StreamSpec streamSpec, Serde<K> keySerde, Serde<V> msgSerde) {
- if (!this.outStreams.containsKey(streamSpec.getSystemStream())) {
- this.outStreams.putIfAbsent(streamSpec.getSystemStream(), new OutputStreamImpl<K, V, M>(this, streamSpec, keySerde, msgSerde));
- }
- return this.outStreams.get(streamSpec.getSystemStream());
- }
-
- /**
- * Helper method to be used by {@link MessageStreamImpl} class
- *
- * @param streamSpec the {@link StreamSpec} object defining the {@link SystemStream} as an intermediate {@link SystemStream}
- * @param <M> the type of {@link MessageEnvelope}s in the output {@link SystemStream}
- * @return the {@link MessageStreamImpl} object
- */
- @Override
- public <K, V, M extends MessageEnvelope<K, V>> OutputStream<M> createIntStream(StreamSpec streamSpec, Serde<K> keySerde, Serde<V> msgSerde) {
- if (!this.inStreams.containsKey(streamSpec.getSystemStream())) {
- this.inStreams.putIfAbsent(streamSpec.getSystemStream(), new IntermediateStreamImpl<K, K, V, M>(this, streamSpec, keySerde, msgSerde));
- }
- IntermediateStreamImpl<K, K, V, M> intStream = (IntermediateStreamImpl<K, K, V, M>) this.inStreams.get(streamSpec.getSystemStream());
- if (!this.outStreams.containsKey(streamSpec.getSystemStream())) {
- this.outStreams.putIfAbsent(streamSpec.getSystemStream(), intStream);
- }
- return intStream;
- }
-
- @Override public Map<StreamSpec, MessageStream> getInStreams() {
- Map<StreamSpec, MessageStream> inStreamMap = new HashMap<>();
- this.inStreams.forEach((ss, entry) -> inStreamMap.put(((InputStreamImpl) entry).getSpec(), entry));
- return Collections.unmodifiableMap(inStreamMap);
- }
-
- @Override public Map<StreamSpec, OutputStream> getOutStreams() {
- Map<StreamSpec, OutputStream> outStreamMap = new HashMap<>();
- this.outStreams.forEach((ss, entry) -> outStreamMap.put(((OutputStreamImpl) entry).getSpec(), entry));
- return Collections.unmodifiableMap(outStreamMap);
- }
-
- @Override
- public StreamGraph withContextManager(ContextManager manager) {
- this.contextManager = manager;
- return this;
- }
-
- public int getNextOpId() {
- return this.opId++;
- }
-
- public ContextManager getContextManager() {
- return this.contextManager;
- }
-
- /**
- * Helper method to be get the input stream via {@link SystemStream}
- *
- * @param systemStream the {@link SystemStream}
- * @return a {@link MessageStreamImpl} object corresponding to the {@code systemStream}
- */
- public MessageStreamImpl getInputStream(SystemStream systemStream) {
- if (this.inStreams.containsKey(systemStream)) {
- return (MessageStreamImpl) this.inStreams.get(systemStream);
- }
- return null;
- }
-
- <M> OutputStream<M> getOutputStream(MessageStreamImpl<M> intStream) {
- if (this.outStreams.containsValue(intStream)) {
- return (OutputStream<M>) intStream;
- }
- return null;
- }
-
- <M> MessageStream<M> getIntStream(OutputStream<M> outStream) {
- if (this.inStreams.containsValue(outStream)) {
- return (MessageStream<M>) outStream;
- }
- return null;
- }
-
- /**
- * Method to create intermediate topics for {@link MessageStreamImpl#partitionBy(Function)} method.
- *
- * @param parKeyFn the function to extract the partition key from the input message
- * @param <PK> the type of partition key
- * @param <M> the type of input message
- * @return the {@link OutputStream} object for the re-partitioned stream
- */
- <PK, M> MessageStreamImpl<M> createIntStream(Function<M, PK> parKeyFn) {
- // TODO: placeholder to auto-generate intermediate streams via {@link StreamSpec}
- StreamSpec streamSpec = new StreamSpec() {
- @Override
- public SystemStream getSystemStream() {
- // TODO: should auto-generate intermedaite stream name here
- return new SystemStream("intermediate", String.format("par-%d", StreamGraphImpl.this.opId));
- }
-
- @Override
- public Properties getProperties() {
- return null;
- }
- };
-
- if (!this.inStreams.containsKey(streamSpec.getSystemStream())) {
- this.inStreams.putIfAbsent(streamSpec.getSystemStream(), new IntermediateStreamImpl(this, streamSpec, null, null, parKeyFn));
- }
- IntermediateStreamImpl intStream = (IntermediateStreamImpl) this.inStreams.get(streamSpec.getSystemStream());
- if (!this.outStreams.containsKey(streamSpec.getSystemStream())) {
- this.outStreams.putIfAbsent(streamSpec.getSystemStream(), intStream);
- }
- return intStream;
- }
-
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-operator/src/main/java/org/apache/samza/operators/functions/PartialJoinFunction.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/functions/PartialJoinFunction.java b/samza-operator/src/main/java/org/apache/samza/operators/functions/PartialJoinFunction.java
deleted file mode 100644
index 809a70a..0000000
--- a/samza-operator/src/main/java/org/apache/samza/operators/functions/PartialJoinFunction.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.functions;
-
-import org.apache.samza.annotation.InterfaceStability;
-
-
-/**
- * This defines the interface function a two-way join functions that takes input messages from two input
- * {@link org.apache.samza.operators.MessageStream}s and merge them into a single output joined message in the join output
- */
-@InterfaceStability.Unstable
-public interface PartialJoinFunction<K, M, OM, RM> extends InitableFunction {
-
- /**
- * Method to perform join method on the two input messages
- *
- * @param m1 message from the first input stream
- * @param om message from the second input stream
- * @return the joined message in the output stream
- */
- RM apply(M m1, OM om);
-
- /**
- * Method to get the key from the input message
- *
- * @param message the input message from the first strean
- * @return the join key in the {@code message}
- */
- K getKey(M message);
-
- /**
- * Method to get the key from the input message in the other stream
- *
- * @param message the input message from the other stream
- * @return the join key in the {@code message}
- */
- K getOtherKey(OM message);
-
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorGraph.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorGraph.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorGraph.java
deleted file mode 100644
index 66336f8..0000000
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorGraph.java
+++ /dev/null
@@ -1,164 +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.config.Config;
-import org.apache.samza.operators.MessageStreamImpl;
-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.system.SystemStream;
-import org.apache.samza.task.TaskContext;
-
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-
-
-/**
- * Instantiates the DAG of {@link OperatorImpl}s corresponding to the {@link OperatorSpec}s for a
- * {@link MessageStreamImpl}
- */
-public class OperatorGraph {
-
- /**
- * A {@link Map} from {@link OperatorSpec} to {@link OperatorImpl}. This map registers all {@link OperatorImpl} in the DAG
- * of {@link OperatorImpl} in a {@link org.apache.samza.container.TaskInstance}. Each {@link OperatorImpl} is created
- * according to a single instance of {@link OperatorSpec}.
- */
- private final Map<OperatorSpec, OperatorImpl> operators = new HashMap<>();
-
- /**
- * This {@link Map} describes the DAG of {@link OperatorImpl} that are chained together to process the input messages.
- */
- private final Map<SystemStream, RootOperatorImpl> operatorGraph = new HashMap<>();
-
- /**
- * Initialize the whole DAG of {@link OperatorImpl}s, based on the input {@link MessageStreamImpl} from the {@link org.apache.samza.operators.StreamGraph}.
- * This method will traverse each input {@link org.apache.samza.operators.MessageStream} in the {@code inputStreams} and
- * instantiate the corresponding {@link OperatorImpl} chains that take the {@link org.apache.samza.operators.MessageStream} as input.
- *
- * @param inputStreams the map of input {@link org.apache.samza.operators.MessageStream}s
- * @param config the {@link Config} required to instantiate operators
- * @param context the {@link TaskContext} required to instantiate operators
- */
- public void init(Map<SystemStream, MessageStreamImpl> inputStreams, Config config, TaskContext context) {
- inputStreams.forEach((ss, mstream) -> this.operatorGraph.put(ss, this.createOperatorImpls(mstream, config, context)));
- }
-
- /**
- * Method to get the corresponding {@link RootOperatorImpl}
- *
- * @param ss input {@link SystemStream}
- * @param <M> the type of input message
- * @return the {@link OperatorImpl} that starts processing the input message
- */
- public <M> OperatorImpl<M, M> get(SystemStream ss) {
- return this.operatorGraph.get(ss);
- }
-
- /**
- * 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 messagess in the {@code source} {@link MessageStreamImpl}
- * @param config the {@link Config} required to instantiate operators
- * @param context the {@link TaskContext} required to instantiate operators
- * @return root node for the {@link OperatorImpl} DAG
- */
- private <M> RootOperatorImpl<M> createOperatorImpls(MessageStreamImpl<M> source, Config config,
- 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, ?> operatorImpl =
- this.createAndRegisterOperatorImpl(registeredOperator, source, config, 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 <M> type of input message
- * @param config the {@link Config} required to instantiate operators
- * @param context the {@link TaskContext} required to instantiate operators
- * @return the operator implementation for the operatorSpec
- */
- private <M> OperatorImpl<M, ?> createAndRegisterOperatorImpl(OperatorSpec operatorSpec,
- MessageStreamImpl<M> source, Config config, TaskContext context) {
- if (!operators.containsKey(operatorSpec)) {
- OperatorImpl<M, ?> operatorImpl = createOperatorImpl(source, operatorSpec, config, context);
- if (operators.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.
- // initialize the corresponding operator function
- operatorSpec.init(config, context);
- MessageStreamImpl nextStream = operatorSpec.getNextStream();
- if (nextStream != null) {
- Collection<OperatorSpec> registeredSpecs = nextStream.getRegisteredOperatorSpecs();
- registeredSpecs.forEach(registeredSpec -> {
- OperatorImpl subImpl = this.createAndRegisterOperatorImpl(registeredSpec, nextStream, config, context);
- operatorImpl.registerNextOperator(subImpl);
- });
- }
- return operatorImpl;
- }
- }
-
- // the implementation corresponding to operatorSpec has already been instantiated
- // and registered, so we do not need to traverse the DAG further.
- return operators.get(operatorSpec);
- }
-
- /**
- * Creates a new {@link OperatorImpl} instance for the provided {@link OperatorSpec}.
- *
- * @param source the source {@link MessageStreamImpl}
- * @param <M> type of input message
- * @param operatorSpec the immutable {@link OperatorSpec} definition.
- * @param config the {@link Config} required to instantiate operators
- * @param context the {@link TaskContext} required to instantiate operators
- * @return the {@link OperatorImpl} implementation instance
- */
- private static <M> OperatorImpl<M, ?> createOperatorImpl(MessageStreamImpl<M> source, OperatorSpec operatorSpec, Config config, TaskContext context) {
- if (operatorSpec instanceof StreamOperatorSpec) {
- StreamOperatorSpec<M, ?> streamOpSpec = (StreamOperatorSpec<M, ?>) operatorSpec;
- return new StreamOperatorImpl<>(streamOpSpec, source, config, context);
- } else if (operatorSpec instanceof SinkOperatorSpec) {
- return new SinkOperatorImpl<>((SinkOperatorSpec<M>) operatorSpec, config, context);
- } else if (operatorSpec instanceof WindowOperatorSpec) {
- return new WindowOperatorImpl<>((WindowOperatorSpec<M, ?, ?>) operatorSpec, source, config, context);
- } else if (operatorSpec instanceof PartialJoinOperatorSpec) {
- return new PartialJoinOperatorImpl<>((PartialJoinOperatorSpec) operatorSpec, source, config, context);
- }
- throw new IllegalArgumentException(
- String.format("Unsupported OperatorSpec: %s", operatorSpec.getClass().getName()));
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/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
deleted file mode 100644
index abb1fa9..0000000
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.samza.operators.impl;
-
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskCoordinator;
-
-import java.util.HashSet;
-import java.util.Set;
-
-
-/**
- * Abstract base class for all stream operator implementations.
- */
-public abstract class OperatorImpl<M, RM> {
-
- private final Set<OperatorImpl<RM, ?>> 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, ?> nextOperator) {
- nextOperators.add(nextOperator);
- }
-
- /**
- * 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 message
- * @param collector the {@link MessageCollector} in the context
- * @param coordinator the {@link TaskCoordinator} in the context
- */
- public abstract void onNext(M message, MessageCollector collector, TaskCoordinator coordinator);
-
- /**
- * Helper method to propagate the output of this operator to all registered downstream operators.
- *
- * This method <b>must</b> be called from {@link #onNext} to propagate the operator output correctly.
- *
- * @param outputMessage output message
- * @param collector the {@link MessageCollector} in the context
- * @param coordinator the {@link TaskCoordinator} in the context
- */
- 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/8515448a/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
deleted file mode 100644
index c8515e1..0000000
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/PartialJoinOperatorImpl.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.impl;
-
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.MessageStreamImpl;
-import org.apache.samza.operators.spec.PartialJoinOperatorSpec;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskContext;
-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 messages in the input stream
- * @param <JM> type of messages in the stream to join with
- * @param <RM> type of messages in the joined stream
- */
-class PartialJoinOperatorImpl<M, K, JM, RM> extends OperatorImpl<M, RM> {
-
- PartialJoinOperatorImpl(PartialJoinOperatorSpec<M, K, JM, RM> joinOp, MessageStreamImpl<M> source, Config config, TaskContext context) {
- // 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/8515448a/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
deleted file mode 100644
index 4b30a5d..0000000
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/RootOperatorImpl.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.impl;
-
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskCoordinator;
-
-
-/**
- * A no-op operator implementation that forwards incoming messages to all of its subscribers.
- * @param <M> type of incoming messages
- */
-final class RootOperatorImpl<M> 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/8515448a/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
deleted file mode 100644
index 2bb362c..0000000
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/SessionWindowOperatorImpl.java
+++ /dev/null
@@ -1,52 +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.config.Config;
-import org.apache.samza.operators.MessageStreamImpl;
-import org.apache.samza.operators.spec.WindowOperatorSpec;
-import org.apache.samza.operators.windows.WindowPane;
-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 message
- * @param <RK> the type of window key
- * @param <WV> the type of window state
- */
-class SessionWindowOperatorImpl<M, RK, WV> extends OperatorImpl<M, WindowPane<RK, WV>> {
-
- private final WindowOperatorSpec<M, RK, WV> windowSpec;
-
- SessionWindowOperatorImpl(WindowOperatorSpec<M, RK, WV> windowSpec, MessageStreamImpl<M> source, Config config, TaskContext context) {
- this.windowSpec = windowSpec;
- }
-
- @Override
- public void onNext(M message, MessageCollector collector, TaskCoordinator coordinator) {
- }
-
- 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/8515448a/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
deleted file mode 100644
index 41d1778..0000000
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/SinkOperatorImpl.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;
-
-import org.apache.samza.config.Config;
-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.TaskContext;
-import org.apache.samza.task.TaskCoordinator;
-
-
-/**
- * Implementation for {@link SinkOperatorSpec}
- */
-class SinkOperatorImpl<M> extends OperatorImpl<M, M> {
-
- private final SinkFunction<M> sinkFn;
-
- SinkOperatorImpl(SinkOperatorSpec<M> sinkOp, Config config, TaskContext context) {
- this.sinkFn = sinkOp.getSinkFn();
- }
-
- @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/8515448a/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
deleted file mode 100644
index 644de20..0000000
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/StreamOperatorImpl.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.config.Config;
-import org.apache.samza.operators.MessageStreamImpl;
-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.TaskContext;
-import org.apache.samza.task.TaskCoordinator;
-
-
-/**
- * A StreamOperator that accepts a 1:n transform function and applies it to each incoming message.
- *
- * @param <M> type of message in the input stream
- * @param <RM> type of message in the output stream
- */
-class StreamOperatorImpl<M, RM> extends OperatorImpl<M, RM> {
-
- private final FlatMapFunction<M, RM> transformFn;
-
- StreamOperatorImpl(StreamOperatorSpec<M, RM> streamOperatorSpec, MessageStreamImpl<M> source, Config config, TaskContext context) {
- 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/8515448a/samza-operator/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java b/samza-operator/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java
deleted file mode 100644
index af00553..0000000
--- a/samza-operator/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java
+++ /dev/null
@@ -1,43 +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.config.Config;
-import org.apache.samza.operators.MessageStreamImpl;
-import org.apache.samza.operators.spec.WindowOperatorSpec;
-import org.apache.samza.operators.windows.WindowPane;
-import org.apache.samza.operators.windows.internal.WindowInternal;
-import org.apache.samza.task.MessageCollector;
-import org.apache.samza.task.TaskContext;
-import org.apache.samza.task.TaskCoordinator;
-
-public class WindowOperatorImpl<M, WK, WV> extends OperatorImpl<M, WindowPane<WK, WV>> {
-
- private final WindowInternal<M, WK, WV> window;
-
- public WindowOperatorImpl(WindowOperatorSpec spec, MessageStreamImpl<M> source, Config config, TaskContext context) {
- // source, config, and context are used to initialize the window kv-store
- window = spec.getWindow();
- }
-
- @Override
- public void onNext(M message, MessageCollector collector, TaskCoordinator coordinator) {
-
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/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
deleted file mode 100644
index 1444662..0000000
--- a/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.samza.operators.spec;
-
-import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.MessageStreamImpl;
-import org.apache.samza.task.TaskContext;
-
-
-/**
- * A stateless serializable stream operator specification that holds all the information required
- * to transform the input {@link MessageStreamImpl} and produce the output {@link MessageStreamImpl}.
- *
- * @param <OM> the type of output message from the operator
- */
-@InterfaceStability.Unstable
-public interface OperatorSpec<OM> {
-
- enum OpCode {
- MAP,
- FLAT_MAP,
- FILTER,
- SINK,
- SEND_TO,
- JOIN,
- WINDOW,
- MERGE,
- PARTITION_BY
- }
-
-
- /**
- * Get the output stream containing transformed messages produced by this operator.
- * @return the output stream containing transformed messages produced by this operator.
- */
- MessageStreamImpl<OM> getNextStream();
-
- /**
- * Init method to initialize the context for this {@link OperatorSpec}. The default implementation is NO-OP.
- *
- * @param config the {@link Config} object for this task
- * @param context the {@link TaskContext} object for this task
- */
- default void init(Config config, TaskContext context) { }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/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
deleted file mode 100644
index d626852..0000000
--- a/samza-operator/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java
+++ /dev/null
@@ -1,210 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.operators.spec;
-
-import java.util.Collection;
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.MessageStreamImpl;
-import org.apache.samza.operators.OutputStream;
-import org.apache.samza.operators.StreamGraphImpl;
-import org.apache.samza.operators.functions.FilterFunction;
-import org.apache.samza.operators.functions.FlatMapFunction;
-import org.apache.samza.operators.functions.MapFunction;
-import org.apache.samza.operators.functions.PartialJoinFunction;
-import org.apache.samza.operators.functions.SinkFunction;
-import org.apache.samza.operators.windows.WindowPane;
-import org.apache.samza.operators.windows.internal.WindowInternal;
-
-import java.util.ArrayList;
-import org.apache.samza.task.TaskContext;
-
-
-/**
- * Factory methods for creating {@link OperatorSpec} instances.
- */
-public class OperatorSpecs {
-
- private OperatorSpecs() {}
-
- /**
- * Creates a {@link StreamOperatorSpec} for {@link MapFunction}
- *
- * @param mapFn the map function
- * @param graph the {@link StreamGraphImpl} object
- * @param output the output {@link MessageStreamImpl} object
- * @param <M> type of input message
- * @param <OM> type of output message
- * @return the {@link StreamOperatorSpec}
- */
- public static <M, OM> StreamOperatorSpec<M, OM> createMapOperatorSpec(MapFunction<M, OM> mapFn, StreamGraphImpl graph, MessageStreamImpl<OM> output) {
- return new StreamOperatorSpec<>(new FlatMapFunction<M, OM>() {
- @Override
- public Collection<OM> apply(M message) {
- return new ArrayList<OM>() {
- {
- OM r = mapFn.apply(message);
- if (r != null) {
- this.add(r);
- }
- }
- };
- }
-
- @Override
- public void init(Config config, TaskContext context) {
- mapFn.init(config, context);
- }
- }, output, OperatorSpec.OpCode.MAP, graph.getNextOpId());
- }
-
- /**
- * Creates a {@link StreamOperatorSpec} for {@link FilterFunction}
- *
- * @param filterFn the transformation function
- * @param graph the {@link StreamGraphImpl} object
- * @param output the output {@link MessageStreamImpl} object
- * @param <M> type of input message
- * @return the {@link StreamOperatorSpec}
- */
- public static <M> StreamOperatorSpec<M, M> createFilterOperatorSpec(FilterFunction<M> filterFn, StreamGraphImpl graph, MessageStreamImpl<M> output) {
- return new StreamOperatorSpec<>(new FlatMapFunction<M, M>() {
- @Override
- public Collection<M> apply(M message) {
- return new ArrayList<M>() {
- {
- if (filterFn.apply(message)) {
- this.add(message);
- }
- }
- };
- }
-
- @Override
- public void init(Config config, TaskContext context) {
- filterFn.init(config, context);
- }
- }, output, OperatorSpec.OpCode.FILTER, graph.getNextOpId());
- }
-
- /**
- * Creates a {@link StreamOperatorSpec}.
- *
- * @param transformFn the transformation function
- * @param graph the {@link StreamGraphImpl} object
- * @param output the output {@link MessageStreamImpl} object
- * @param <M> type of input message
- * @param <OM> type of output message
- * @return the {@link StreamOperatorSpec}
- */
- public static <M, OM> StreamOperatorSpec<M, OM> createStreamOperatorSpec(
- FlatMapFunction<M, OM> transformFn, StreamGraphImpl graph, MessageStreamImpl<OM> output) {
- return new StreamOperatorSpec<>(transformFn, output, OperatorSpec.OpCode.FLAT_MAP, graph.getNextOpId());
- }
-
- /**
- * Creates a {@link SinkOperatorSpec}.
- *
- * @param sinkFn the sink function
- * @param <M> type of input message
- * @param graph the {@link StreamGraphImpl} object
- * @return the {@link SinkOperatorSpec}
- */
- public static <M> SinkOperatorSpec<M> createSinkOperatorSpec(SinkFunction<M> sinkFn, StreamGraphImpl graph) {
- return new SinkOperatorSpec<>(sinkFn, OperatorSpec.OpCode.SINK, graph.getNextOpId());
- }
-
- /**
- * Creates a {@link SinkOperatorSpec}.
- *
- * @param sinkFn the sink function
- * @param graph the {@link StreamGraphImpl} object
- * @param stream the {@link OutputStream} where the message is sent to
- * @param <M> type of input message
- * @return the {@link SinkOperatorSpec}
- */
- public static <M> SinkOperatorSpec<M> createSendToOperatorSpec(SinkFunction<M> sinkFn, StreamGraphImpl graph, OutputStream<M> stream) {
- return new SinkOperatorSpec<>(sinkFn, OperatorSpec.OpCode.SEND_TO, graph.getNextOpId(), stream);
- }
-
- /**
- * Creates a {@link SinkOperatorSpec}.
- *
- * @param sinkFn the sink function
- * @param graph the {@link StreamGraphImpl} object
- * @param stream the {@link OutputStream} where the message is sent to
- * @param <M> type of input message
- * @return the {@link SinkOperatorSpec}
- */
- public static <M> SinkOperatorSpec<M> createPartitionOperatorSpec(SinkFunction<M> sinkFn, StreamGraphImpl graph, OutputStream<M> stream) {
- return new SinkOperatorSpec<>(sinkFn, OperatorSpec.OpCode.PARTITION_BY, graph.getNextOpId(), stream);
- }
-
- /**
- * Creates a {@link WindowOperatorSpec}.
- *
- * @param window the description of the window.
- * @param graph the {@link StreamGraphImpl} object
- * @param wndOutput the window output {@link MessageStreamImpl} object
- * @param <M> the type of input message
- * @param <WK> the type of key in the {@link WindowPane}
- * @param <WV> the type of value in the window
- * @return the {@link WindowOperatorSpec}
- */
-
- public static <M, WK, WV> WindowOperatorSpec<M, WK, WV> createWindowOperatorSpec(
- WindowInternal<M, WK, WV> window, StreamGraphImpl graph, MessageStreamImpl<WindowPane<WK, WV>> wndOutput) {
- return new WindowOperatorSpec<>(window, wndOutput, graph.getNextOpId());
- }
-
- /**
- * Creates a {@link PartialJoinOperatorSpec}.
- *
- * @param partialJoinFn the join function
- * @param graph the {@link StreamGraphImpl} object
- * @param joinOutput the output {@link MessageStreamImpl}
- * @param <M> type of input message
- * @param <K> type of join key
- * @param <JM> the type of message in the other join stream
- * @param <OM> the type of message in the join output
- * @return the {@link PartialJoinOperatorSpec}
- */
- public static <M, K, JM, OM> PartialJoinOperatorSpec<M, K, JM, OM> createPartialJoinOperatorSpec(
- PartialJoinFunction<K, M, JM, OM> partialJoinFn, StreamGraphImpl graph, MessageStreamImpl<OM> joinOutput) {
- return new PartialJoinOperatorSpec<>(partialJoinFn, joinOutput, graph.getNextOpId());
- }
-
- /**
- * Creates a {@link StreamOperatorSpec} with a merger function.
- *
- * @param graph the {@link StreamGraphImpl} object
- * @param mergeOutput the output {@link MessageStreamImpl} from the merger
- * @param <M> the type of input message
- * @return the {@link StreamOperatorSpec} for the merge
- */
- public static <M> StreamOperatorSpec<M, M> createMergeOperatorSpec(StreamGraphImpl graph, MessageStreamImpl<M> mergeOutput) {
- return new StreamOperatorSpec<M, M>(message ->
- new ArrayList<M>() {
- {
- this.add(message);
- }
- },
- mergeOutput, OperatorSpec.OpCode.MERGE, graph.getNextOpId());
- }
-}
[12/13] samza git commit: SAMZA-1096: StreamSpec constructors in the
ExecutionEnvironments
Posted by jm...@apache.org.
SAMZA-1096: StreamSpec constructors in the ExecutionEnvironments
Project: http://git-wip-us.apache.org/repos/asf/samza/repo
Commit: http://git-wip-us.apache.org/repos/asf/samza/commit/674e5231
Tree: http://git-wip-us.apache.org/repos/asf/samza/tree/674e5231
Diff: http://git-wip-us.apache.org/repos/asf/samza/diff/674e5231
Branch: refs/heads/samza-fluent-api-v1
Commit: 674e5231bec56a00a274be46ea249cb5b3962f0e
Parents: ea37b74
Author: Jacob Maes <jm...@linkedin.com>
Authored: Thu Feb 23 10:57:32 2017 -0800
Committer: Jacob Maes <jm...@linkedin.com>
Committed: Thu Feb 23 10:58:12 2017 -0800
----------------------------------------------------------------------
.../samza/system/ExecutionEnvironment.java | 9 +-
.../org/apache/samza/system/StreamProvider.java | 78 +++++
.../org/apache/samza/system/StreamSpec.java | 22 +-
.../system/AbstractExecutionEnvironment.java | 67 ++++
.../system/RemoteExecutionEnvironment.java | 6 +-
.../system/StandaloneExecutionEnvironment.java | 6 +-
.../org/apache/samza/config/JobConfig.scala | 11 +
.../org/apache/samza/config/StreamConfig.scala | 73 ++++-
.../TestAbstractExecutionEnvironment.java | 308 +++++++++++++++++++
.../org/apache/samza/config/KafkaConfig.scala | 8 +-
10 files changed, 567 insertions(+), 21 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/samza/blob/674e5231/samza-api/src/main/java/org/apache/samza/system/ExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/ExecutionEnvironment.java b/samza-api/src/main/java/org/apache/samza/system/ExecutionEnvironment.java
index ad37eb3..ef46626 100644
--- a/samza-api/src/main/java/org/apache/samza/system/ExecutionEnvironment.java
+++ b/samza-api/src/main/java/org/apache/samza/system/ExecutionEnvironment.java
@@ -18,6 +18,7 @@
*/
package org.apache.samza.system;
+import java.lang.reflect.Constructor;
import org.apache.samza.annotation.InterfaceStability;
import org.apache.samza.config.ConfigException;
import org.apache.samza.operators.StreamGraphBuilder;
@@ -28,7 +29,7 @@ import org.apache.samza.config.Config;
* Interface to be implemented by physical execution engine to deploy the config and jobs to run the {@link org.apache.samza.operators.StreamGraph}
*/
@InterfaceStability.Unstable
-public interface ExecutionEnvironment {
+public interface ExecutionEnvironment extends StreamProvider {
String ENVIRONMENT_CONFIG = "job.execution.environment.class";
String DEFAULT_ENVIRONMENT_CLASS = "org.apache.samza.system.StandaloneExecutionEnvironment";
@@ -51,8 +52,10 @@ public interface ExecutionEnvironment {
*/
static ExecutionEnvironment fromConfig(Config config) {
try {
- if (ExecutionEnvironment.class.isAssignableFrom(Class.forName(config.get(ENVIRONMENT_CONFIG, DEFAULT_ENVIRONMENT_CLASS)))) {
- return (ExecutionEnvironment) Class.forName(config.get(ENVIRONMENT_CONFIG, DEFAULT_ENVIRONMENT_CLASS)).newInstance();
+ Class<?> environmentClass = Class.forName(config.get(ENVIRONMENT_CONFIG, DEFAULT_ENVIRONMENT_CLASS));
+ if (ExecutionEnvironment.class.isAssignableFrom(environmentClass)) {
+ Constructor<?> constructor = environmentClass.getConstructor(Config.class); // *sigh*
+ return (ExecutionEnvironment) constructor.newInstance(config);
}
} catch (Exception e) {
throw new ConfigException(String.format("Problem in loading ExecutionEnvironment class %s", config.get(ENVIRONMENT_CONFIG)), e);
http://git-wip-us.apache.org/repos/asf/samza/blob/674e5231/samza-api/src/main/java/org/apache/samza/system/StreamProvider.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/StreamProvider.java b/samza-api/src/main/java/org/apache/samza/system/StreamProvider.java
new file mode 100644
index 0000000..62c2ec4
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/system/StreamProvider.java
@@ -0,0 +1,78 @@
+/*
+ * 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.system;
+
+/**
+ * Describes the common interface for classes that construct instances of {@link StreamSpec}.
+ */
+public interface StreamProvider {
+
+ /**
+ * Constructs a {@link StreamSpec} from the configuration for the specified streamId.
+ *
+ * The stream configurations are read from the following properties in the config:
+ * {@code streams.{$streamId}.*}
+ * <br>
+ * All properties matching this pattern are assumed to be system-specific with two exceptions. The following two
+ * properties are Samza properties which are used to bind the stream to a system and a physical resource on that system.
+ *
+ * <ul>
+ * <li>system - The name of the System on which this stream will be used. If this property isn't defined
+ * the stream will be associated with the System defined in {@code job.default.system}</li>
+ * <li>physicalName - The system-specific name for this stream. It could be a file URN, topic name, or other identifer.</li>
+ * </ul>
+ *
+ * @param streamId The logical identifier for the stream in Samza.
+ * @return The {@link StreamSpec} instance.
+ */
+ StreamSpec streamFromConfig(String streamId);
+
+ /**
+ * Constructs a {@link StreamSpec} from the configuration for the specified streamId.
+ *
+ * The stream configurations are read from the following properties in the config:
+ * {@code streams.{$streamId}.*}
+ * <br>
+ * All properties matching this pattern are assumed to be system-specific with one exception. The following
+ * property is a Samza property which is used to bind the stream to a system.
+ *
+ * <ul>
+ * <li>system - The name of the System on which this stream will be used. If this property isn't defined
+ * the stream will be associated with the System defined in {@code job.default.system}</li>
+ * </ul>
+ *
+ * @param streamId The logical identifier for the stream in Samza.
+ * @param physicalName The system-specific name for this stream. It could be a file URN, topic name, or other identifer.
+ * @return The {@link StreamSpec} instance.
+ */
+ StreamSpec streamFromConfig(String streamId, String physicalName);
+
+ /**
+ * Constructs a {@link StreamSpec} from the configuration for the specified streamId.
+ *
+ * The stream configurations are read from the following properties in the config:
+ * {@code streams.{$streamId}.*}
+ *
+ * @param streamId The logical identifier for the stream in Samza.
+ * @param physicalName The system-specific name for this stream. It could be a file URN, topic name, or other identifer.
+ * @param systemName The name of the System on which this stream will be used.
+ * @return The {@link StreamSpec} instance.
+ */
+ StreamSpec streamFromConfig(String streamId, String physicalName, String systemName);
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/674e5231/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java b/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java
index d8a2144..3bd0076 100644
--- a/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java
+++ b/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java
@@ -137,16 +137,11 @@ public class StreamSpec {
* @param config A map of properties for the stream. These may be System-specfic.
*/
public StreamSpec(String id, String physicalName, String systemName, int partitionCount, Map<String, String> config) {
- if (id == null) {
- throw new NullPointerException("Parameter 'id' must not be null");
- }
-
- if (systemName == null) {
- throw new NullPointerException("Parameter 'systemName' must not be null");
- }
+ validateLogicalIdentifier("id", id);
+ validateLogicalIdentifier("systemName", systemName);
if (partitionCount < 1) {
- throw new NullPointerException("Parameter 'partitionCount' must not be greater than 0");
+ throw new IllegalArgumentException("Parameter 'partitionCount' must be greater than 0");
}
this.id = id;
@@ -200,4 +195,15 @@ public class StreamSpec {
public String getOrDefault(String propertyName, String defaultValue) {
return config.getOrDefault(propertyName, defaultValue);
}
+
+ private void validateLogicalIdentifier(String identifierName, String identifierValue) {
+ /*if (identifier == null) {
+ throw new NullPointerException();
+ } else if (identifier.isEmpty()) {
+
+ } else*/
+ if (!identifierValue.matches("[A-Za-z0-9_-]+")) {
+ throw new IllegalArgumentException(String.format("Identifier '%s' must match the expression [A-Za-z0-9_-]+", identifierName));
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/samza/blob/674e5231/samza-core/src/main/java/org/apache/samza/system/AbstractExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/system/AbstractExecutionEnvironment.java b/samza-core/src/main/java/org/apache/samza/system/AbstractExecutionEnvironment.java
new file mode 100644
index 0000000..c066bdd
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/system/AbstractExecutionEnvironment.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.system;
+
+import java.util.Map;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.StreamConfig;
+
+
+public abstract class AbstractExecutionEnvironment implements ExecutionEnvironment {
+
+ private final Config config;
+
+ public AbstractExecutionEnvironment(Config config) {
+ if (config == null) {
+ throw new NullPointerException();
+ }
+
+ this.config = config;
+ }
+
+ @Override
+ public StreamSpec streamFromConfig(String streamId) {
+ StreamConfig streamConfig = new StreamConfig(config);
+
+ String system = streamConfig.getSystem(streamId);
+ String physicalName = streamConfig.getPhysicalName(streamId, streamId);
+ Map<String, String> properties = streamConfig.getStreamProperties(streamId);
+
+ return new StreamSpec(streamId, physicalName, system, properties);
+ }
+
+ @Override
+ public StreamSpec streamFromConfig(String streamId, String physicalName) {
+ StreamConfig streamConfig = new StreamConfig(config);
+
+ String system = streamConfig.getSystem(streamId);
+ Map<String, String> properties = streamConfig.getStreamProperties(streamId);
+
+ return new StreamSpec(streamId, physicalName, system, properties);
+ }
+
+ @Override
+ public StreamSpec streamFromConfig(String streamId, String physicalName, String system) {
+ StreamConfig streamConfig = new StreamConfig(config);
+
+ Map<String, String> properties = streamConfig.getStreamProperties(streamId);
+
+ return new StreamSpec(streamId, physicalName, system, properties);
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/674e5231/samza-core/src/main/java/org/apache/samza/system/RemoteExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/system/RemoteExecutionEnvironment.java b/samza-core/src/main/java/org/apache/samza/system/RemoteExecutionEnvironment.java
index 1dbc5f4..851c7f3 100644
--- a/samza-core/src/main/java/org/apache/samza/system/RemoteExecutionEnvironment.java
+++ b/samza-core/src/main/java/org/apache/samza/system/RemoteExecutionEnvironment.java
@@ -33,7 +33,11 @@ import org.slf4j.LoggerFactory;
/**
* This class implements the {@link ExecutionEnvironment} that runs the applications in YARN environment
*/
-public class RemoteExecutionEnvironment implements ExecutionEnvironment {
+public class RemoteExecutionEnvironment extends AbstractExecutionEnvironment {
+
+ public RemoteExecutionEnvironment(Config config) {
+ super(config);
+ }
private static final Logger log = LoggerFactory.getLogger(RemoteExecutionEnvironment.class);
@Override public void run(StreamGraphBuilder app, Config config) {
http://git-wip-us.apache.org/repos/asf/samza/blob/674e5231/samza-core/src/main/java/org/apache/samza/system/StandaloneExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/system/StandaloneExecutionEnvironment.java b/samza-core/src/main/java/org/apache/samza/system/StandaloneExecutionEnvironment.java
index f0f6ef2..71d60ef 100644
--- a/samza-core/src/main/java/org/apache/samza/system/StandaloneExecutionEnvironment.java
+++ b/samza-core/src/main/java/org/apache/samza/system/StandaloneExecutionEnvironment.java
@@ -28,7 +28,11 @@ import org.apache.samza.operators.StreamGraphImpl;
/**
* This class implements the {@link ExecutionEnvironment} that runs the applications in standalone environment
*/
-public class StandaloneExecutionEnvironment implements ExecutionEnvironment {
+public class StandaloneExecutionEnvironment extends AbstractExecutionEnvironment {
+
+ public StandaloneExecutionEnvironment(Config config) {
+ super(config);
+ }
// TODO: may want to move this to a common base class for all {@link ExecutionEnvironment}
StreamGraph createGraph(StreamGraphBuilder app, Config config) {
http://git-wip-us.apache.org/repos/asf/samza/blob/674e5231/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala b/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala
index a797ac2..6b1473c 100644
--- a/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala
+++ b/samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala
@@ -43,6 +43,8 @@ object JobConfig {
val SAMZA_FWK_PATH = "samza.fwk.path"
val SAMZA_FWK_VERSION = "samza.fwk.version"
val JOB_COORDINATOR_SYSTEM = "job.coordinator.system"
+ val JOB_METADATA_DEFAULT_SYSTEM = "job.metadata.system"
+ val JOB_DEFAULT_SYSTEM = "job.default.system"
val JOB_CONTAINER_COUNT = "job.container.count"
val jOB_CONTAINER_THREAD_POOL_SIZE = "job.container.thread.pool.size"
val JOB_CONTAINER_SINGLE_THREAD_MODE = "job.container.single.thread.mode"
@@ -108,6 +110,15 @@ class JobConfig(config: Config) extends ScalaMapConfig(config) with Logging {
def getCoordinatorSystemName = getOption(JobConfig.JOB_COORDINATOR_SYSTEM).getOrElse(
throw new ConfigException("Missing job.coordinator.system configuration. Cannot proceed with job execution."))
+ def getMetadataSystemName = {
+ getOption(JobConfig.JOB_METADATA_DEFAULT_SYSTEM) match {
+ case Some(system) => Some(system)
+ case _ => getDefaultSystem
+ }
+ }
+
+ def getDefaultSystem = getOption(JobConfig.JOB_DEFAULT_SYSTEM)
+
def getContainerCount = {
getOption(JobConfig.JOB_CONTAINER_COUNT) match {
case Some(count) => count.toInt
http://git-wip-us.apache.org/repos/asf/samza/blob/674e5231/samza-core/src/main/scala/org/apache/samza/config/StreamConfig.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/config/StreamConfig.scala b/samza-core/src/main/scala/org/apache/samza/config/StreamConfig.scala
index 0ccc7df..c376681 100644
--- a/samza-core/src/main/scala/org/apache/samza/config/StreamConfig.scala
+++ b/samza-core/src/main/scala/org/apache/samza/config/StreamConfig.scala
@@ -19,13 +19,23 @@
package org.apache.samza.config
+import org.apache.samza.config.JobConfig.Config2Job
+import org.apache.samza.system.SystemStream
import org.apache.samza.util.Logging
+
import scala.collection.JavaConversions._
-import org.apache.samza.system.SystemStream
object StreamConfig {
// stream config constants
val STREAM_PREFIX = "systems.%s.streams.%s."
+
+ val SYSTEM = "system"
+ val PHYSICAL_NAME = "physicalName"
+ val STREAM_PREFIX_BY_ID = "streams.%s."
+ val SYSTEM_FOR_STREAM_ID = STREAM_PREFIX_BY_ID + SYSTEM
+ val PHYSICAL_NAME_FOR_STREAM_ID = STREAM_PREFIX_BY_ID + PHYSICAL_NAME
+ val SAMZA_STREAM_PROPERTIES = Set(StreamConfig.SYSTEM, StreamConfig.PHYSICAL_NAME)
+
val MSG_SERDE = STREAM_PREFIX + "samza.msg.serde"
val KEY_SERDE = STREAM_PREFIX + "samza.key.serde"
val CONSUMER_RESET_OFFSET = STREAM_PREFIX + "samza.reset.offset"
@@ -79,4 +89,65 @@ class StreamConfig(config: Config) extends ScalaMapConfig(config) with Logging {
new SystemStream(systemName, streamName)
}).toSet
}
+
+ /**
+ * Gets the stream properties from the legacy config style:
+ * systems.{system}.streams.{streams}.*
+ *
+ * @param systemName the system name under which the properties are configured
+ * @param streamName the stream name
+ * @return the map of properties for the stream
+ */
+ def getSystemStreamProperties(systemName: String, streamName: String) = {
+ config.subset(StreamConfig.STREAM_PREFIX format(systemName, streamName), true)
+ }
+
+ /**
+ * Gets the properties for the specified streamId from the config.
+ * This method supercedes {@link StreamConfig.#getSystemStreamProperties}
+ * It first applies any legacy configs from this config location:
+ * systems.{system}.streams.{stream}.*
+ *
+ * It then overrides them with properties of the new config format:
+ * streams.{streamId}.*
+ *
+ * @param streamId the identifier for the stream in the config.
+ * @return the merged map of config properties from both the legacy and new config styles
+ */
+ def getStreamProperties(streamId: String) = {
+ val properties = subset(StreamConfig.STREAM_PREFIX_BY_ID format streamId)
+ val inheritedLegacyProperties:java.util.Map[String, String] = getSystemStreamProperties(getSystem(streamId), streamId)
+ val filteredStreamProperties:java.util.Map[String, String] = properties.filterKeys(k => !StreamConfig.SAMZA_STREAM_PROPERTIES.contains(k))
+ new MapConfig(java.util.Arrays.asList(inheritedLegacyProperties, filteredStreamProperties))
+ }
+
+ /**
+ * Gets the System associated with the specified streamId.
+ * It first looks for the property
+ * streams.{streamId}.system
+ *
+ * If no value was provided, it uses
+ * job.default.system
+ *
+ * @param streamId the identifier for the stream in the config.
+ * @return the system name associated with the stream.
+ */
+ def getSystem(streamId: String) = {
+ getOption(StreamConfig.SYSTEM_FOR_STREAM_ID format streamId) match {
+ case Some(system) => system
+ case _ => config.getDefaultSystem.getOrElse(throw new ConfigException("Missing %s configuration. Cannot bind stream to a system without it."
+ format(StreamConfig.SYSTEM_FOR_STREAM_ID format(streamId))))
+ }
+ }
+
+ /**
+ * Gets the physical name for the specified streamId.
+ *
+ * @param streamId the identifier for the stream in the config.
+ * @param defaultPhysicalName the default to use if the physical name is missing.
+ * @return the physical identifier for the stream or the default if it is undefined.
+ */
+ def getPhysicalName(streamId: String, defaultPhysicalName: String) = {
+ get(StreamConfig.PHYSICAL_NAME_FOR_STREAM_ID format streamId, defaultPhysicalName)
+ }
}
http://git-wip-us.apache.org/repos/asf/samza/blob/674e5231/samza-core/src/test/java/org/apache/samza/system/TestAbstractExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/system/TestAbstractExecutionEnvironment.java b/samza-core/src/test/java/org/apache/samza/system/TestAbstractExecutionEnvironment.java
new file mode 100644
index 0000000..e547322
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/system/TestAbstractExecutionEnvironment.java
@@ -0,0 +1,308 @@
+/*
+ * 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.system;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.ConfigException;
+import org.apache.samza.config.JobConfig;
+import org.apache.samza.config.MapConfig;
+import org.apache.samza.config.StreamConfig;
+import org.apache.samza.operators.StreamGraphBuilder;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+public class TestAbstractExecutionEnvironment {
+ private static final String STREAM_ID = "t3st-Stream_Id";
+ private static final String STREAM_ID_INVALID = "test#Str3amId!";
+
+ private static final String TEST_PHYSICAL_NAME = "t3st-Physical_Name";
+ private static final String TEST_PHYSICAL_NAME2 = "testPhysicalName2";
+ private static final String TEST_PHYSICAL_NAME_SPECIAL_CHARS = "test://Physical.Name?";
+
+ private static final String TEST_SYSTEM = "t3st-System_Name";
+ private static final String TEST_SYSTEM2 = "testSystemName2";
+ private static final String TEST_SYSTEM_INVALID = "test:System!Name@";
+
+ private static final String TEST_DEFAULT_SYSTEM = "testDefaultSystemName";
+
+
+ @Test(expected = NullPointerException.class)
+ public void testConfigValidation() {
+ new TestAbstractExecutionEnvironmentImpl(null);
+ }
+
+ // The physical name should be pulled from the StreamConfig.PHYSICAL_NAME property value.
+ @Test
+ public void testStreamFromConfigWithPhysicalNameInConfig() {
+ Config config = buildStreamConfig(STREAM_ID,
+ StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME,
+ StreamConfig.SYSTEM(), TEST_SYSTEM);
+
+ ExecutionEnvironment env = new TestAbstractExecutionEnvironmentImpl(config);
+ StreamSpec spec = env.streamFromConfig(STREAM_ID);
+
+ assertEquals(TEST_PHYSICAL_NAME, spec.getPhysicalName());
+ }
+
+ // The streamId should be used as the physicalName when the physical name is not specified.
+ // NOTE: its either this, set to null, or exception. This seems better for backward compatibility and API brevity.
+ @Test
+ public void testStreamFromConfigWithoutPhysicalNameInConfig() {
+ Config config = buildStreamConfig(STREAM_ID,
+ StreamConfig.SYSTEM(), TEST_SYSTEM);
+
+ ExecutionEnvironment env = new TestAbstractExecutionEnvironmentImpl(config);
+ StreamSpec spec = env.streamFromConfig(STREAM_ID);
+
+ assertEquals(STREAM_ID, spec.getPhysicalName());
+ }
+
+ // If the system is specified at the stream scope, use it
+ @Test
+ public void testStreamFromConfigWithSystemAtStreamScopeInConfig() {
+ Config config = buildStreamConfig(STREAM_ID,
+ StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME,
+ StreamConfig.SYSTEM(), TEST_SYSTEM);
+
+ ExecutionEnvironment env = new TestAbstractExecutionEnvironmentImpl(config);
+ StreamSpec spec = env.streamFromConfig(STREAM_ID);
+
+ assertEquals(TEST_SYSTEM, spec.getSystemName());
+ }
+
+ // If system isn't specified at stream scope, use the default system
+ @Test
+ public void testStreamFromConfigWithSystemAtDefaultScopeInConfig() {
+ Config config = addConfigs(buildStreamConfig(STREAM_ID,
+ StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME),
+ JobConfig.JOB_DEFAULT_SYSTEM(), TEST_DEFAULT_SYSTEM);
+
+ ExecutionEnvironment env = new TestAbstractExecutionEnvironmentImpl(config);
+ StreamSpec spec = env.streamFromConfig(STREAM_ID);
+
+ assertEquals(TEST_DEFAULT_SYSTEM, spec.getSystemName());
+ }
+
+ // Stream scope should override default scope
+ @Test
+ public void testStreamFromConfigWithSystemAtBothScopesInConfig() {
+ Config config = addConfigs(buildStreamConfig(STREAM_ID,
+ StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME,
+ StreamConfig.SYSTEM(), TEST_SYSTEM),
+ JobConfig.JOB_DEFAULT_SYSTEM(), TEST_DEFAULT_SYSTEM);
+
+ ExecutionEnvironment env = new TestAbstractExecutionEnvironmentImpl(config);
+ StreamSpec spec = env.streamFromConfig(STREAM_ID);
+
+ assertEquals(TEST_SYSTEM, spec.getSystemName());
+ }
+
+ // System is required. Throw if it cannot be determined.
+ @Test(expected = ConfigException.class)
+ public void testStreamFromConfigWithOutSystemInConfig() {
+ Config config = buildStreamConfig(STREAM_ID,
+ StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME);
+
+ ExecutionEnvironment env = new TestAbstractExecutionEnvironmentImpl(config);
+ StreamSpec spec = env.streamFromConfig(STREAM_ID);
+
+ assertEquals(TEST_SYSTEM, spec.getSystemName());
+
+ }
+
+ // The properties in the config "streams.{streamId}.*" should be passed through to the spec.
+ @Test
+ public void testStreamFromConfigPropertiesPassthrough() {
+ Config config = buildStreamConfig(STREAM_ID,
+ StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME,
+ StreamConfig.SYSTEM(), TEST_SYSTEM,
+ "systemProperty1", "systemValue1",
+ "systemProperty2", "systemValue2");
+
+ ExecutionEnvironment env = new TestAbstractExecutionEnvironmentImpl(config);
+ StreamSpec spec = env.streamFromConfig(STREAM_ID);
+
+ Map<String, String> properties = spec.getConfig();
+ assertEquals(2, properties.size());
+ assertEquals("systemValue1", properties.get("systemProperty1"));
+ assertEquals("systemValue2", properties.get("systemProperty2"));
+ assertEquals("systemValue1", spec.get("systemProperty1"));
+ assertEquals("systemValue2", spec.get("systemProperty2"));
+ }
+
+ // The samza properties (which are invalid for the underlying system) should be filtered out.
+ @Test
+ public void testStreamFromConfigSamzaPropertiesOmitted() {
+ Config config = buildStreamConfig(STREAM_ID,
+ StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME,
+ StreamConfig.SYSTEM(), TEST_SYSTEM,
+ "systemProperty1", "systemValue1",
+ "systemProperty2", "systemValue2");
+
+ ExecutionEnvironment env = new TestAbstractExecutionEnvironmentImpl(config);
+ StreamSpec spec = env.streamFromConfig(STREAM_ID);
+
+ Map<String, String> properties = spec.getConfig();
+ assertEquals(2, properties.size());
+ assertNull(properties.get(String.format(StreamConfig.PHYSICAL_NAME_FOR_STREAM_ID(), STREAM_ID)));
+ assertNull(properties.get(String.format(StreamConfig.SYSTEM_FOR_STREAM_ID(), STREAM_ID)));
+ assertNull(spec.get(String.format(StreamConfig.PHYSICAL_NAME_FOR_STREAM_ID(), STREAM_ID)));
+ assertNull(spec.get(String.format(StreamConfig.SYSTEM_FOR_STREAM_ID(), STREAM_ID)));
+ }
+
+ // When the physicalName argument is passed explicitly it should be used, regardless of whether it is also in the config
+ @Test
+ public void testStreamFromConfigPhysicalNameArgSimple() {
+ Config config = buildStreamConfig(STREAM_ID,
+ StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME2, // This should be ignored because of the explicit arg
+ StreamConfig.SYSTEM(), TEST_SYSTEM);
+
+ ExecutionEnvironment env = new TestAbstractExecutionEnvironmentImpl(config);
+ StreamSpec spec = env.streamFromConfig(STREAM_ID, TEST_PHYSICAL_NAME);
+
+ assertEquals(STREAM_ID, spec.getId());
+ assertEquals(TEST_PHYSICAL_NAME, spec.getPhysicalName());
+ assertEquals(TEST_SYSTEM, spec.getSystemName());
+ }
+
+ // Special characters are allowed for the physical name
+ @Test
+ public void testStreamFromConfigPhysicalNameArgSpecialCharacters() {
+ Config config = buildStreamConfig(STREAM_ID,
+ StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME2,
+ StreamConfig.SYSTEM(), TEST_SYSTEM);
+
+ ExecutionEnvironment env = new TestAbstractExecutionEnvironmentImpl(config);
+ StreamSpec spec = env.streamFromConfig(STREAM_ID, TEST_PHYSICAL_NAME_SPECIAL_CHARS);
+ assertEquals(TEST_PHYSICAL_NAME_SPECIAL_CHARS, spec.getPhysicalName());
+ }
+
+ // Null is allowed for the physical name
+ @Test
+ public void testStreamFromConfigPhysicalNameArgNull() {
+ Config config = buildStreamConfig(STREAM_ID,
+ StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME2,
+ StreamConfig.SYSTEM(), TEST_SYSTEM);
+
+ ExecutionEnvironment env = new TestAbstractExecutionEnvironmentImpl(config);
+ StreamSpec spec = env.streamFromConfig(STREAM_ID, null);
+ assertNull(spec.getPhysicalName());
+ }
+
+ // When the system name is provided explicitly, it should be used, regardless of whether it's also in the config
+ @Test
+ public void testStreamFromConfigSystemNameArgValid() {
+ Config config = buildStreamConfig(STREAM_ID,
+ StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME2, // This should be ignored because of the explicit arg
+ StreamConfig.SYSTEM(), TEST_SYSTEM2); // This too
+
+ ExecutionEnvironment env = new TestAbstractExecutionEnvironmentImpl(config);
+ StreamSpec spec = env.streamFromConfig(STREAM_ID, TEST_PHYSICAL_NAME, TEST_SYSTEM);
+
+ assertEquals(STREAM_ID, spec.getId());
+ assertEquals(TEST_PHYSICAL_NAME, spec.getPhysicalName());
+ assertEquals(TEST_SYSTEM, spec.getSystemName());
+ }
+
+ // Special characters are NOT allowed for system name, because it's used as an identifier in the config.
+ @Test(expected = IllegalArgumentException.class)
+ public void testStreamFromConfigSystemNameArgInvalid() {
+ Config config = buildStreamConfig(STREAM_ID,
+ StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME2,
+ StreamConfig.SYSTEM(), TEST_SYSTEM2);
+
+ ExecutionEnvironment env = new TestAbstractExecutionEnvironmentImpl(config);
+ env.streamFromConfig(STREAM_ID, TEST_PHYSICAL_NAME, TEST_SYSTEM_INVALID);
+ }
+
+ // Null is not allowed for system name.
+ @Test(expected = NullPointerException.class)
+ public void testStreamFromConfigSystemNameArgNull() {
+ Config config = buildStreamConfig(STREAM_ID,
+ StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME2,
+ StreamConfig.SYSTEM(), TEST_SYSTEM2);
+
+ ExecutionEnvironment env = new TestAbstractExecutionEnvironmentImpl(config);
+ env.streamFromConfig(STREAM_ID, TEST_PHYSICAL_NAME, null);
+ }
+
+ // Special characters are NOT allowed for streamId, because it's used as an identifier in the config.
+ @Test(expected = IllegalArgumentException.class)
+ public void testStreamFromConfigStreamIdInvalid() {
+ Config config = buildStreamConfig(STREAM_ID_INVALID,
+ StreamConfig.SYSTEM(), TEST_SYSTEM);
+
+ ExecutionEnvironment env = new TestAbstractExecutionEnvironmentImpl(config);
+ env.streamFromConfig(STREAM_ID_INVALID);
+ }
+
+ // Null is not allowed for streamId.
+ @Test(expected = NullPointerException.class)
+ public void testStreamFromConfigStreamIdNull() {
+ Config config = buildStreamConfig(null,
+ StreamConfig.SYSTEM(), TEST_SYSTEM);
+
+ ExecutionEnvironment env = new TestAbstractExecutionEnvironmentImpl(config);
+ env.streamFromConfig(null);
+ }
+
+
+ // Helper methods
+
+ private Config buildStreamConfig(String streamId, String... kvs) {
+ // inject streams.x. into each key
+ for (int i = 0; i < kvs.length - 1; i += 2) {
+ kvs[i] = String.format(StreamConfig.STREAM_PREFIX_BY_ID(), streamId) + kvs[i];
+ }
+ return buildConfig(kvs);
+ }
+
+ private Config buildConfig(String... kvs) {
+ if (kvs.length % 2 != 0) {
+ throw new IllegalArgumentException("There must be parity between the keys and values");
+ }
+
+ Map<String, String> configMap = new HashMap<>();
+ for (int i = 0; i < kvs.length - 1; i += 2) {
+ configMap.put(kvs[i], kvs[i + 1]);
+ }
+ return new MapConfig(configMap);
+ }
+
+ private Config addConfigs(Config original, String... kvs) {
+ Map<String, String> result = new HashMap<>();
+ result.putAll(original);
+ result.putAll(buildConfig(kvs));
+ return new MapConfig(result);
+ }
+
+ private class TestAbstractExecutionEnvironmentImpl extends AbstractExecutionEnvironment {
+
+ public TestAbstractExecutionEnvironmentImpl(Config config) {
+ super(config);
+ }
+
+ @Override
+ public void run(StreamGraphBuilder graphBuilder, Config config) {
+ // do nothing
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/samza/blob/674e5231/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConfig.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConfig.scala b/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConfig.scala
index e355e7e..7e9f18a 100644
--- a/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConfig.scala
+++ b/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConfig.scala
@@ -36,6 +36,7 @@ import java.util
import scala.collection.JavaConverters._
import org.apache.samza.system.kafka.KafkaSystemFactory
import org.apache.samza.config.SystemConfig.Config2System
+import org.apache.samza.config.StreamConfig.Config2Stream
import org.apache.kafka.common.serialization.ByteArraySerializer
object KafkaConfig {
@@ -165,13 +166,6 @@ class KafkaConfig(config: Config) extends ScalaMapConfig(config) {
kafkaChangeLogProperties
}
- def getTopicKafkaProperties(systemName: String, streamName: String) = {
- val filteredConfigs = config.subset(StreamConfig.STREAM_PREFIX format(systemName, streamName), true)
- val topicProperties = new Properties
- filteredConfigs.foreach { kv => topicProperties.setProperty(kv._1, kv._2) }
- topicProperties
- }
-
// kafka config
def getKafkaSystemConsumerConfig( systemName: String,
clientId: String,
[09/13] samza git commit: SAMZA-1075: fix partitionCount assertion
from PR53
Posted by jm...@apache.org.
SAMZA-1075: fix partitionCount assertion from PR53
nickpan47 here's the fix for the issue you found in PR53
Author: Jacob Maes <jm...@linkedin.com>
Reviewers: Yi Pan <ni...@gmail.com>
Closes #57 from jmakes/samza-1075-2
Project: http://git-wip-us.apache.org/repos/asf/samza/repo
Commit: http://git-wip-us.apache.org/repos/asf/samza/commit/062ac56d
Tree: http://git-wip-us.apache.org/repos/asf/samza/tree/062ac56d
Diff: http://git-wip-us.apache.org/repos/asf/samza/diff/062ac56d
Branch: refs/heads/samza-fluent-api-v1
Commit: 062ac56d1169ba4af273d7f92e3d6cb1cc553ca8
Parents: 6939b8c
Author: Jacob Maes <jm...@apache.org>
Authored: Tue Feb 21 16:49:37 2017 -0800
Committer: Jacob Maes <jm...@linkedin.com>
Committed: Tue Feb 21 16:49:37 2017 -0800
----------------------------------------------------------------------
samza-api/src/main/java/org/apache/samza/system/StreamSpec.java | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/samza/blob/062ac56d/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java b/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java
index d8a2144..e953d46 100644
--- a/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java
+++ b/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java
@@ -146,7 +146,7 @@ public class StreamSpec {
}
if (partitionCount < 1) {
- throw new NullPointerException("Parameter 'partitionCount' must not be greater than 0");
+ throw new IllegalArgumentException("Parameter 'partitionCount' must be greater than 0");
}
this.id = id;
[03/13] samza git commit: SAMZA-1073: moving all operator classes
into samza-core
Posted by jm...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/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
deleted file mode 100644
index e057c2b..0000000
--- a/samza-operator/src/main/java/org/apache/samza/operators/spec/PartialJoinOperatorSpec.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.samza.operators.spec;
-
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.MessageStreamImpl;
-import org.apache.samza.operators.functions.PartialJoinFunction;
-import org.apache.samza.task.TaskContext;
-
-
-/**
- * Spec for the partial join operator that takes messages from one input stream, joins with buffered
- * messages from another stream, and produces join results to an output {@link MessageStreamImpl}.
- *
- * @param <M> the type of input message
- * @param <K> the type of join key
- * @param <JM> the type of message in the other join stream
- * @param <RM> the type of message in the join output stream
- */
-public class PartialJoinOperatorSpec<M, K, JM, RM> implements OperatorSpec<RM> {
-
- private final MessageStreamImpl<RM> joinOutput;
-
- /**
- * The transformation function of {@link PartialJoinOperatorSpec} that takes an input message of
- * type {@code M}, joins with a stream of buffered messages of type {@code JM} from another stream,
- * and generates a joined result message of type {@code RM}.
- */
- private final PartialJoinFunction<K, M, JM, RM> transformFn;
-
-
- /**
- * The unique ID for this operator.
- */
- private final int opId;
-
- /**
- * Default constructor for a {@link PartialJoinOperatorSpec}.
- *
- * @param partialJoinFn partial join function that take type {@code M} of input message and join
- * w/ type {@code JM} of buffered message from another stream
- * @param joinOutput the output {@link MessageStreamImpl} of the join results
- */
- PartialJoinOperatorSpec(PartialJoinFunction<K, M, JM, RM> partialJoinFn, MessageStreamImpl<RM> joinOutput, int opId) {
- this.joinOutput = joinOutput;
- this.transformFn = partialJoinFn;
- this.opId = opId;
- }
-
- @Override
- public MessageStreamImpl<RM> getNextStream() {
- return this.joinOutput;
- }
-
- public PartialJoinFunction<K, M, JM, RM> getTransformFn() {
- return this.transformFn;
- }
-
- public OperatorSpec.OpCode getOpCode() {
- return OpCode.JOIN;
- }
-
- public int getOpId() {
- return this.opId;
- }
-
- @Override public void init(Config config, TaskContext context) {
- this.transformFn.init(config, context);
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/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
deleted file mode 100644
index ba30d67..0000000
--- a/samza-operator/src/main/java/org/apache/samza/operators/spec/SinkOperatorSpec.java
+++ /dev/null
@@ -1,116 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.samza.operators.spec;
-
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.MessageStreamImpl;
-import org.apache.samza.operators.OutputStream;
-import org.apache.samza.operators.functions.SinkFunction;
-import org.apache.samza.task.TaskContext;
-
-
-/**
- * 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 message
- */
-public class SinkOperatorSpec<M> implements OperatorSpec {
-
- /**
- * {@link OpCode} for this {@link SinkOperatorSpec}
- */
- private final OperatorSpec.OpCode opCode;
-
- /**
- * The unique ID for this operator.
- */
- private final int opId;
-
- /**
- * The user-defined sink function
- */
- private final SinkFunction<M> sinkFn;
-
- /**
- * Potential output stream defined by the {@link SinkFunction}
- */
- private final OutputStream<M> outStream;
-
- /**
- * Default constructor for a {@link SinkOperatorSpec} w/o an output stream. (e.g. output is sent to remote database)
- *
- * @param sinkFn a user defined {@link SinkFunction} that will be called with the output message,
- * the output {@link org.apache.samza.task.MessageCollector} and the
- * {@link org.apache.samza.task.TaskCoordinator}.
- * @param opCode the specific {@link OpCode} for this {@link SinkOperatorSpec}. It could be {@link OpCode#SINK}, {@link OpCode#SEND_TO},
- * or {@link OpCode#PARTITION_BY}
- * @param opId the unique id of this {@link SinkOperatorSpec} in the {@link org.apache.samza.operators.StreamGraph}
- */
- SinkOperatorSpec(SinkFunction<M> sinkFn, OperatorSpec.OpCode opCode, int opId) {
- this(sinkFn, opCode, opId, null);
- }
-
- /**
- * Default constructor for a {@link SinkOperatorSpec} that sends the output to an {@link OutputStream}
- *
- * @param sinkFn a user defined {@link SinkFunction} that will be called with the output message,
- * the output {@link org.apache.samza.task.MessageCollector} and the
- * {@link org.apache.samza.task.TaskCoordinator}.
- * @param opCode the specific {@link OpCode} for this {@link SinkOperatorSpec}. It could be {@link OpCode#SINK}, {@link OpCode#SEND_TO},
- * or {@link OpCode#PARTITION_BY}
- * @param opId the unique id of this {@link SinkOperatorSpec} in the {@link org.apache.samza.operators.StreamGraph}
- * @param opId the {@link OutputStream} for this {@link SinkOperatorSpec}
- */
- SinkOperatorSpec(SinkFunction<M> sinkFn, OperatorSpec.OpCode opCode, int opId, OutputStream<M> outStream) {
- this.sinkFn = sinkFn;
- this.opCode = opCode;
- this.opId = opId;
- this.outStream = outStream;
- }
-
- /**
- * This is a terminal operator and doesn't allow further operator chaining.
- * @return null
- */
- @Override
- public MessageStreamImpl<M> getNextStream() {
- return null;
- }
-
- public SinkFunction<M> getSinkFn() {
- return this.sinkFn;
- }
-
- public OperatorSpec.OpCode getOpCode() {
- return this.opCode;
- }
-
- public int getOpId() {
- return this.opId;
- }
-
- public OutputStream<M> getOutStream() {
- return this.outStream;
- }
-
- @Override public void init(Config config, TaskContext context) {
- this.sinkFn.init(config, context);
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/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
deleted file mode 100644
index d7813f7..0000000
--- a/samza-operator/src/main/java/org/apache/samza/operators/spec/StreamOperatorSpec.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.samza.operators.spec;
-
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.MessageStreamImpl;
-import org.apache.samza.operators.functions.FlatMapFunction;
-import org.apache.samza.task.TaskContext;
-
-
-/**
- * The spec for a linear stream operator that outputs 0 or more messages for each input message.
- *
- * @param <M> the type of input message
- * @param <OM> the type of output message
- */
-public class StreamOperatorSpec<M, OM> implements OperatorSpec<OM> {
-
- /**
- * {@link OpCode} for this {@link StreamOperatorSpec}
- */
- private final OperatorSpec.OpCode opCode;
-
- /**
- * The unique ID for this operator.
- */
- private final int opId;
-
- /**
- * The output {@link MessageStreamImpl} from this {@link StreamOperatorSpec}
- */
- private final MessageStreamImpl<OM> outputStream;
-
- /**
- * Transformation function applied in this {@link StreamOperatorSpec}
- */
- private final FlatMapFunction<M, OM> transformFn;
-
- /**
- * Constructor for a {@link StreamOperatorSpec} that accepts an output {@link MessageStreamImpl}.
- *
- * @param transformFn the transformation function
- * @param outputStream the output {@link MessageStreamImpl}
- * @param opCode the {@link OpCode} for this {@link StreamOperatorSpec}
- * @param opId the unique id for this {@link StreamOperatorSpec} in a {@link org.apache.samza.operators.StreamGraph}
- */
- StreamOperatorSpec(FlatMapFunction<M, OM> transformFn, MessageStreamImpl outputStream, OperatorSpec.OpCode opCode, int opId) {
- this.outputStream = outputStream;
- this.transformFn = transformFn;
- this.opCode = opCode;
- this.opId = opId;
- }
-
- @Override
- public MessageStreamImpl<OM> getNextStream() {
- return this.outputStream;
- }
-
- public FlatMapFunction<M, OM> getTransformFn() {
- return this.transformFn;
- }
-
- public OperatorSpec.OpCode getOpCode() {
- return this.opCode;
- }
-
- public int getOpId() {
- return this.opId;
- }
-
- @Override
- public void init(Config config, TaskContext context) {
- this.transformFn.init(config, context);
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/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
deleted file mode 100644
index 46417ed..0000000
--- a/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.samza.operators.spec;
-
-import org.apache.samza.operators.MessageStreamImpl;
-import org.apache.samza.operators.windows.WindowPane;
-import org.apache.samza.operators.windows.internal.WindowInternal;
-
-
-/**
- * Default window operator spec object
- *
- * @param <M> the type of input message to the window
- * @param <WK> the type of key of the window
- * @param <WV> the type of aggregated value in the window output {@link WindowPane}
- */
-public class WindowOperatorSpec<M, WK, WV> implements OperatorSpec<WindowPane<WK, WV>> {
-
- private final WindowInternal<M, WK, WV> window;
-
- private final MessageStreamImpl<WindowPane<WK, WV>> outputStream;
-
- private final int opId;
-
-
- /**
- * Constructor for {@link WindowOperatorSpec}.
- *
- * @param window the window function
- * @param outputStream the output {@link MessageStreamImpl} from this {@link WindowOperatorSpec}
- * @param opId auto-generated unique ID of this operator
- */
- WindowOperatorSpec(WindowInternal<M, WK, WV> window, MessageStreamImpl<WindowPane<WK, WV>> outputStream, int opId) {
- this.outputStream = outputStream;
- this.window = window;
- this.opId = opId;
- }
-
- @Override
- public MessageStreamImpl<WindowPane<WK, WV>> getNextStream() {
- return this.outputStream;
- }
-
- public WindowInternal getWindow() {
- return window;
- }
-
- public OpCode getOpCode() {
- return OpCode.WINDOW;
- }
-
- public int getOpId() {
- return this.opId;
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowState.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowState.java b/samza-operator/src/main/java/org/apache/samza/operators/spec/WindowState.java
deleted file mode 100644
index 53bca2e..0000000
--- a/samza-operator/src/main/java/org/apache/samza/operators/spec/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.spec;
-
-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/8515448a/samza-operator/src/main/java/org/apache/samza/system/RemoteExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/system/RemoteExecutionEnvironment.java b/samza-operator/src/main/java/org/apache/samza/system/RemoteExecutionEnvironment.java
deleted file mode 100644
index fafa2cb..0000000
--- a/samza-operator/src/main/java/org/apache/samza/system/RemoteExecutionEnvironment.java
+++ /dev/null
@@ -1,37 +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.system;
-
-import org.apache.samza.operators.StreamGraphBuilder;
-import org.apache.samza.config.Config;
-
-/**
- * This class implements the {@link ExecutionEnvironment} that runs the applications in YARN environment
- */
-public class RemoteExecutionEnvironment implements ExecutionEnvironment {
-
- @Override public void run(StreamGraphBuilder app, Config config) {
- // TODO: add description of ProcessContext that is going to create a sub-DAG of the {@code graph}
- // TODO: actually instantiate the tasks and run the job, i.e.
- // 1. create all input/output/intermediate topics
- // 2. create the single job configuration
- // 3. execute JobRunner to submit the single job for the whole graph
- }
-
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-operator/src/main/java/org/apache/samza/system/StandaloneExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/system/StandaloneExecutionEnvironment.java b/samza-operator/src/main/java/org/apache/samza/system/StandaloneExecutionEnvironment.java
deleted file mode 100644
index f0f6ef2..0000000
--- a/samza-operator/src/main/java/org/apache/samza/system/StandaloneExecutionEnvironment.java
+++ /dev/null
@@ -1,50 +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.system;
-
-import org.apache.samza.operators.StreamGraph;
-import org.apache.samza.operators.StreamGraphBuilder;
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.StreamGraphImpl;
-
-
-/**
- * This class implements the {@link ExecutionEnvironment} that runs the applications in standalone environment
- */
-public class StandaloneExecutionEnvironment implements ExecutionEnvironment {
-
- // TODO: may want to move this to a common base class for all {@link ExecutionEnvironment}
- StreamGraph createGraph(StreamGraphBuilder app, Config config) {
- StreamGraphImpl graph = new StreamGraphImpl();
- app.init(graph, config);
- return graph;
- }
-
- @Override public void run(StreamGraphBuilder app, Config config) {
- // 1. get logic graph for optimization
- // StreamGraph logicGraph = this.createGraph(app, config);
- // 2. potential optimization....
- // 3. create new instance of StreamGraphBuilder that would generate the optimized graph
- // 4. create all input/output/intermediate topics
- // 5. create the configuration for StreamProcessor
- // 6. start the StreamProcessor w/ optimized instance of StreamGraphBuilder
- }
-
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java b/samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java
deleted file mode 100644
index b007e3c..0000000
--- a/samza-operator/src/main/java/org/apache/samza/task/StreamOperatorTask.java
+++ /dev/null
@@ -1,111 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.samza.task;
-
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.ContextManager;
-import org.apache.samza.operators.MessageStreamImpl;
-import org.apache.samza.operators.StreamGraphBuilder;
-import org.apache.samza.operators.StreamGraphImpl;
-import org.apache.samza.operators.data.InputMessageEnvelope;
-import org.apache.samza.operators.impl.OperatorGraph;
-import org.apache.samza.system.IncomingMessageEnvelope;
-import org.apache.samza.system.SystemStream;
-
-import java.util.HashMap;
-import java.util.Map;
-
-
-/**
- * Execution of the logic sub-DAG
- *
- *
- * An {@link StreamTask} implementation that receives {@link InputMessageEnvelope}s and propagates them
- * through the user's stream transformations defined in {@link StreamGraphImpl} using the
- * {@link org.apache.samza.operators.MessageStream} APIs.
- * <p>
- * This class brings all the operator API implementation components together and feeds the
- * {@link InputMessageEnvelope}s into the transformation chains.
- * <p>
- * It accepts an instance of the user implemented factory {@link StreamGraphBuilder} as input parameter of the constructor.
- * When its own {@link #init(Config, TaskContext)} method is called during startup, it instantiate a user-defined {@link StreamGraphImpl}
- * from the {@link StreamGraphBuilder}, calls {@link StreamGraphImpl#getContextManager()} to initialize the task-wide context
- * for the graph, and creates a {@link MessageStreamImpl} corresponding to each of its input
- * {@link org.apache.samza.system.SystemStreamPartition}s. Each input {@link MessageStreamImpl}
- * will be corresponding to either an input stream or intermediate stream in {@link StreamGraphImpl}.
- * <p>
- * Then, this task calls {@link org.apache.samza.operators.impl.OperatorGraph#init(Map, Config, 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 org.apache.samza.system.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 StreamOperatorTask implements StreamTask, InitableTask, WindowableTask, ClosableTask {
-
- /**
- * A mapping from each {@link SystemStream} to the root node of its operator chain DAG.
- */
- private final OperatorGraph operatorGraph = new OperatorGraph();
-
- private final StreamGraphBuilder graphBuilder;
-
- private ContextManager contextManager;
-
- public StreamOperatorTask(StreamGraphBuilder graphBuilder) {
- this.graphBuilder = graphBuilder;
- }
-
- @Override
- public final void init(Config config, TaskContext context) throws Exception {
- // create the MessageStreamsImpl object and initialize app-specific logic DAG within the task
- StreamGraphImpl streams = new StreamGraphImpl();
- this.graphBuilder.init(streams, config);
- // get the context manager of the {@link StreamGraph} and initialize the task-specific context
- this.contextManager = streams.getContextManager();
-
- Map<SystemStream, MessageStreamImpl> inputBySystemStream = new HashMap<>();
- context.getSystemStreamPartitions().forEach(ssp -> {
- if (!inputBySystemStream.containsKey(ssp.getSystemStream())) {
- // create mapping from the physical input {@link SystemStream} to the logic {@link MessageStream}
- inputBySystemStream.putIfAbsent(ssp.getSystemStream(), streams.getInputStream(ssp.getSystemStream()));
- }
- });
- operatorGraph.init(inputBySystemStream, config, this.contextManager.initTaskContext(config, context));
- }
-
- @Override
- public final void process(IncomingMessageEnvelope ime, MessageCollector collector, TaskCoordinator coordinator) {
- this.operatorGraph.get(ime.getSystemStreamPartition().getSystemStream())
- .onNext(new InputMessageEnvelope(ime), collector, coordinator);
- }
-
- @Override
- public final void window(MessageCollector collector, TaskCoordinator coordinator) throws Exception {
- // TODO: invoke timer based triggers
- }
-
- @Override
- public void close() throws Exception {
- this.contextManager.finalizeTaskContext();
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-operator/src/test/java/org/apache/samza/example/KeyValueStoreExample.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/KeyValueStoreExample.java b/samza-operator/src/test/java/org/apache/samza/example/KeyValueStoreExample.java
deleted file mode 100644
index 85ebc6c..0000000
--- a/samza-operator/src/test/java/org/apache/samza/example/KeyValueStoreExample.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.example;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.samza.operators.*;
-import org.apache.samza.operators.StreamGraphBuilder;
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.data.MessageEnvelope;
-import org.apache.samza.operators.functions.FlatMapFunction;
-import org.apache.samza.serializers.JsonSerde;
-import org.apache.samza.serializers.StringSerde;
-import org.apache.samza.storage.kv.KeyValueStore;
-import org.apache.samza.system.ExecutionEnvironment;
-import org.apache.samza.system.SystemStream;
-import org.apache.samza.task.TaskContext;
-import org.apache.samza.util.CommandLine;
-
-import java.util.Properties;
-
-
-/**
- * Example code using {@link KeyValueStore} to implement event-time window
- */
-public class KeyValueStoreExample implements StreamGraphBuilder {
-
- /**
- * used by remote execution environment to launch the job in remote program. The remote program should follow the similar
- * invoking context as in standalone:
- *
- * public static void main(String args[]) throws Exception {
- * CommandLine cmdLine = new CommandLine();
- * Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
- * ExecutionEnvironment remoteEnv = ExecutionEnvironment.getRemoteEnvironment(config);
- * UserMainExample runnableApp = new UserMainExample();
- * runnableApp.run(remoteEnv, config);
- * }
- *
- */
- @Override public void init(StreamGraph graph, Config config) {
-
- MessageStream<PageViewEvent> pageViewEvents = graph.createInStream(input1, new StringSerde("UTF-8"), new JsonSerde<>());
- OutputStream<StatsOutput> pageViewPerMemberCounters = graph.createOutStream(output, new StringSerde("UTF-8"), new JsonSerde<StatsOutput>());
-
- pageViewEvents.
- partitionBy(m -> m.getMessage().memberId).
- flatMap(new MyStatsCounter()).
- sendTo(pageViewPerMemberCounters);
-
- }
-
- // standalone local program model
- public static void main(String[] args) throws Exception {
- CommandLine cmdLine = new CommandLine();
- Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
- ExecutionEnvironment standaloneEnv = ExecutionEnvironment.getLocalEnvironment(config);
- standaloneEnv.run(new KeyValueStoreExample(), config);
- }
-
- class MyStatsCounter implements FlatMapFunction<PageViewEvent, StatsOutput> {
- private final int timeoutMs = 10 * 60 * 1000;
-
- KeyValueStore<String, StatsWindowState> statsStore;
-
- class StatsWindowState {
- int lastCount = 0;
- long timeAtLastOutput = 0;
- int newCount = 0;
- }
-
- @Override
- public Collection<StatsOutput> apply(PageViewEvent message) {
- List<StatsOutput> outputStats = new ArrayList<>();
- long wndTimestamp = (long) Math.floor(TimeUnit.MILLISECONDS.toMinutes(message.getMessage().timestamp) / 5) * 5;
- String wndKey = String.format("%s-%d", message.getMessage().memberId, wndTimestamp);
- StatsWindowState curState = this.statsStore.get(wndKey);
- curState.newCount++;
- long curTimeMs = System.currentTimeMillis();
- if (curState.newCount > 0 && curState.timeAtLastOutput + timeoutMs < curTimeMs) {
- curState.timeAtLastOutput = curTimeMs;
- curState.lastCount += curState.newCount;
- curState.newCount = 0;
- outputStats.add(new StatsOutput(message.getMessage().memberId, wndTimestamp, curState.lastCount));
- }
- // update counter w/o generating output
- this.statsStore.put(wndKey, curState);
- return outputStats;
- }
-
- @Override
- public void init(Config config, TaskContext context) {
- this.statsStore = (KeyValueStore<String, StatsWindowState>) context.getStore("my-stats-wnd-store");
- }
- }
-
- StreamSpec input1 = new StreamSpec() {
- @Override public SystemStream getSystemStream() {
- return new SystemStream("kafka", "PageViewEvent");
- }
-
- @Override public Properties getProperties() {
- return null;
- }
- };
-
- StreamSpec output = new StreamSpec() {
- @Override public SystemStream getSystemStream() {
- return new SystemStream("kafka", "PageViewPerMember5min");
- }
-
- @Override public Properties getProperties() {
- return null;
- }
- };
-
- class PageViewEvent implements MessageEnvelope<String, PageViewEvent> {
- String pageId;
- String memberId;
- long timestamp;
-
- PageViewEvent(String pageId, String memberId, long timestamp) {
- this.pageId = pageId;
- this.memberId = memberId;
- this.timestamp = timestamp;
- }
-
- @Override
- public String getKey() {
- return this.pageId;
- }
-
- @Override
- public PageViewEvent getMessage() {
- return this;
- }
- }
-
- class StatsOutput implements MessageEnvelope<String, StatsOutput> {
- private String memberId;
- private long timestamp;
- private Integer count;
-
- StatsOutput(String key, long timestamp, Integer count) {
- this.memberId = key;
- this.timestamp = timestamp;
- this.count = count;
- }
-
- @Override
- public String getKey() {
- return this.memberId;
- }
-
- @Override
- public StatsOutput getMessage() {
- return this;
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-operator/src/test/java/org/apache/samza/example/NoContextStreamExample.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/NoContextStreamExample.java b/samza-operator/src/test/java/org/apache/samza/example/NoContextStreamExample.java
deleted file mode 100644
index c6d2e6e..0000000
--- a/samza-operator/src/test/java/org/apache/samza/example/NoContextStreamExample.java
+++ /dev/null
@@ -1,151 +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.example;
-
-import org.apache.samza.operators.*;
-import org.apache.samza.operators.StreamGraphBuilder;
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.data.InputMessageEnvelope;
-import org.apache.samza.operators.data.JsonIncomingSystemMessageEnvelope;
-import org.apache.samza.operators.data.Offset;
-import org.apache.samza.operators.functions.JoinFunction;
-import org.apache.samza.serializers.JsonSerde;
-import org.apache.samza.serializers.StringSerde;
-import org.apache.samza.system.ExecutionEnvironment;
-import org.apache.samza.system.SystemStream;
-import org.apache.samza.system.SystemStreamPartition;
-import org.apache.samza.util.CommandLine;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Properties;
-
-
-/**
- * Example {@link StreamGraphBuilder} code to test the API methods
- */
-public class NoContextStreamExample implements StreamGraphBuilder {
-
- StreamSpec input1 = new StreamSpec() {
- @Override public SystemStream getSystemStream() {
- return new SystemStream("kafka", "input1");
- }
-
- @Override public Properties getProperties() {
- return null;
- }
- };
-
- StreamSpec input2 = new StreamSpec() {
- @Override public SystemStream getSystemStream() {
- return new SystemStream("kafka", "input2");
- }
-
- @Override public Properties getProperties() {
- return null;
- }
- };
-
- StreamSpec output = new StreamSpec() {
- @Override public SystemStream getSystemStream() {
- return new SystemStream("kafka", "output");
- }
-
- @Override public Properties getProperties() {
- return null;
- }
- };
-
- 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);
- }
- }
-
- private JsonMessageEnvelope getInputMessage(InputMessageEnvelope ism) {
- return new JsonMessageEnvelope(
- ((MessageType) ism.getMessage()).joinKey,
- (MessageType) ism.getMessage(),
- ism.getOffset(),
- ism.getSystemStreamPartition());
- }
-
- class MyJoinFunction implements JoinFunction<String, JsonMessageEnvelope, JsonMessageEnvelope, JsonIncomingSystemMessageEnvelope<MessageType>> {
-
- @Override
- public JsonIncomingSystemMessageEnvelope<MessageType> apply(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);
- }
-
- @Override
- public String getFirstKey(JsonMessageEnvelope message) {
- return message.getKey();
- }
-
- @Override
- public String getSecondKey(JsonMessageEnvelope message) {
- return message.getKey();
- }
- }
-
- /**
- * used by remote execution environment to launch the job in remote program. The remote program should follow the similar
- * invoking context as in standalone:
- *
- * public static void main(String args[]) throws Exception {
- * CommandLine cmdLine = new CommandLine();
- * Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
- * ExecutionEnvironment remoteEnv = ExecutionEnvironment.fromConfig(config);
- * remoteEnv.run(new NoContextStreamExample(), config);
- * }
- *
- */
- @Override public void init(StreamGraph graph, Config config) {
- MessageStream<InputMessageEnvelope> inputSource1 = graph.<Object, Object, InputMessageEnvelope>createInStream(
- input1, null, null);
- MessageStream<InputMessageEnvelope> inputSource2 = graph.<Object, Object, InputMessageEnvelope>createInStream(
- input2, null, null);
- OutputStream<JsonIncomingSystemMessageEnvelope<MessageType>> outStream = graph.createOutStream(output,
- new StringSerde("UTF-8"), new JsonSerde<>());
-
- inputSource1.map(this::getInputMessage).
- join(inputSource2.map(this::getInputMessage), new MyJoinFunction()).
- sendTo(outStream);
-
- }
-
- // standalone local program model
- public static void main(String[] args) throws Exception {
- CommandLine cmdLine = new CommandLine();
- Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
- ExecutionEnvironment standaloneEnv = ExecutionEnvironment.getLocalEnvironment(config);
- standaloneEnv.run(new NoContextStreamExample(), config);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-operator/src/test/java/org/apache/samza/example/OrderShipmentJoinExample.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/OrderShipmentJoinExample.java b/samza-operator/src/test/java/org/apache/samza/example/OrderShipmentJoinExample.java
deleted file mode 100644
index 0477066..0000000
--- a/samza-operator/src/test/java/org/apache/samza/example/OrderShipmentJoinExample.java
+++ /dev/null
@@ -1,188 +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.example;
-
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.OutputStream;
-import org.apache.samza.operators.StreamGraphBuilder;
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.StreamGraph;
-import org.apache.samza.operators.StreamSpec;
-import org.apache.samza.operators.data.MessageEnvelope;
-import org.apache.samza.operators.functions.JoinFunction;
-import org.apache.samza.serializers.JsonSerde;
-import org.apache.samza.serializers.StringSerde;
-import org.apache.samza.system.ExecutionEnvironment;
-import org.apache.samza.system.SystemStream;
-import org.apache.samza.util.CommandLine;
-
-import java.util.Properties;
-
-
-/**
- * Simple 2-way stream-to-stream join example
- */
-public class OrderShipmentJoinExample implements StreamGraphBuilder {
-
- /**
- * used by remote execution environment to launch the job in remote program. The remote program should follow the similar
- * invoking context as in standalone:
- *
- * public static void main(String args[]) throws Exception {
- * CommandLine cmdLine = new CommandLine();
- * Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
- * ExecutionEnvironment remoteEnv = ExecutionEnvironment.getRemoteEnvironment(config);
- * UserMainExample runnableApp = new UserMainExample();
- * runnableApp.run(remoteEnv, config);
- * }
- *
- */
- @Override public void init(StreamGraph graph, Config config) {
-
- MessageStream<OrderRecord> orders = graph.createInStream(input1, new StringSerde("UTF-8"), new JsonSerde<>());
- MessageStream<ShipmentRecord> shipments = graph.createInStream(input2, new StringSerde("UTF-8"), new JsonSerde<>());
- OutputStream<FulFilledOrderRecord> fulfilledOrders = graph.createOutStream(output, new StringSerde("UTF-8"), new JsonSerde<>());
-
- orders.join(shipments, new MyJoinFunction()).sendTo(fulfilledOrders);
-
- }
-
- // standalone local program model
- public static void main(String[] args) throws Exception {
- CommandLine cmdLine = new CommandLine();
- Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
- ExecutionEnvironment standaloneEnv = ExecutionEnvironment.getLocalEnvironment(config);
- standaloneEnv.run(new OrderShipmentJoinExample(), config);
- }
-
- StreamSpec input1 = new StreamSpec() {
- @Override public SystemStream getSystemStream() {
- return new SystemStream("kafka", "Orders");
- }
-
- @Override public Properties getProperties() {
- return null;
- }
- };
-
- StreamSpec input2 = new StreamSpec() {
- @Override public SystemStream getSystemStream() {
- return new SystemStream("kafka", "Shipment");
- }
-
- @Override public Properties getProperties() {
- return null;
- }
- };
-
- StreamSpec output = new StreamSpec() {
- @Override public SystemStream getSystemStream() {
- return new SystemStream("kafka", "FulfilledOrders");
- }
-
- @Override public Properties getProperties() {
- return null;
- }
- };
-
- class OrderRecord implements MessageEnvelope<String, OrderRecord> {
- String orderId;
- long orderTimeMs;
-
- OrderRecord(String orderId, long timeMs) {
- this.orderId = orderId;
- this.orderTimeMs = timeMs;
- }
-
- @Override
- public String getKey() {
- return this.orderId;
- }
-
- @Override
- public OrderRecord getMessage() {
- return this;
- }
- }
-
- class ShipmentRecord implements MessageEnvelope<String, ShipmentRecord> {
- String orderId;
- long shipTimeMs;
-
- ShipmentRecord(String orderId, long timeMs) {
- this.orderId = orderId;
- this.shipTimeMs = timeMs;
- }
-
- @Override
- public String getKey() {
- return this.orderId;
- }
-
- @Override
- public ShipmentRecord getMessage() {
- return this;
- }
- }
-
- class FulFilledOrderRecord implements MessageEnvelope<String, FulFilledOrderRecord> {
- String orderId;
- long orderTimeMs;
- long shipTimeMs;
-
- FulFilledOrderRecord(String orderId, long orderTimeMs, long shipTimeMs) {
- this.orderId = orderId;
- this.orderTimeMs = orderTimeMs;
- this.shipTimeMs = shipTimeMs;
- }
-
-
- @Override
- public String getKey() {
- return this.orderId;
- }
-
- @Override
- public FulFilledOrderRecord getMessage() {
- return this;
- }
- }
-
- FulFilledOrderRecord myJoinResult(OrderRecord m1, ShipmentRecord m2) {
- return new FulFilledOrderRecord(m1.getKey(), m1.orderTimeMs, m2.shipTimeMs);
- }
-
- class MyJoinFunction implements JoinFunction<String, OrderRecord, ShipmentRecord, FulFilledOrderRecord> {
-
- @Override
- public FulFilledOrderRecord apply(OrderRecord message, ShipmentRecord otherMessage) {
- return OrderShipmentJoinExample.this.myJoinResult(message, otherMessage);
- }
-
- @Override
- public String getFirstKey(OrderRecord message) {
- return message.getKey();
- }
-
- @Override
- public String getSecondKey(ShipmentRecord message) {
- return message.getKey();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-operator/src/test/java/org/apache/samza/example/PageViewCounterExample.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/PageViewCounterExample.java b/samza-operator/src/test/java/org/apache/samza/example/PageViewCounterExample.java
deleted file mode 100644
index f7d8bda..0000000
--- a/samza-operator/src/test/java/org/apache/samza/example/PageViewCounterExample.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.example;
-
-import org.apache.samza.operators.*;
-import org.apache.samza.operators.StreamGraphBuilder;
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.data.MessageEnvelope;
-import org.apache.samza.operators.triggers.Triggers;
-import org.apache.samza.operators.windows.AccumulationMode;
-import org.apache.samza.operators.windows.WindowPane;
-import org.apache.samza.operators.windows.Windows;
-import org.apache.samza.serializers.JsonSerde;
-import org.apache.samza.serializers.StringSerde;
-import org.apache.samza.system.ExecutionEnvironment;
-import org.apache.samza.system.SystemStream;
-import org.apache.samza.util.CommandLine;
-
-import java.time.Duration;
-import java.util.Properties;
-
-
-/**
- * Example code to implement window-based counter
- */
-public class PageViewCounterExample implements StreamGraphBuilder {
-
- @Override public void init(StreamGraph graph, Config config) {
-
- MessageStream<PageViewEvent> pageViewEvents = graph.createInStream(input1, new StringSerde("UTF-8"), new JsonSerde<>());
- OutputStream<MyStreamOutput> pageViewPerMemberCounters = graph.createOutStream(output, new StringSerde("UTF-8"), new JsonSerde<>());
-
- pageViewEvents.
- window(Windows.<PageViewEvent, String, Integer>keyedTumblingWindow(m -> m.getMessage().memberId, Duration.ofSeconds(10), (m, c) -> c + 1).
- setEarlyTrigger(Triggers.repeat(Triggers.count(5))).
- setAccumulationMode(AccumulationMode.DISCARDING)).
- map(MyStreamOutput::new).
- sendTo(pageViewPerMemberCounters);
-
- }
-
- public static void main(String[] args) {
- CommandLine cmdLine = new CommandLine();
- Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
- ExecutionEnvironment standaloneEnv = ExecutionEnvironment.getLocalEnvironment(config);
- standaloneEnv.run(new PageViewCounterExample(), config);
- }
-
- StreamSpec input1 = new StreamSpec() {
- @Override public SystemStream getSystemStream() {
- return new SystemStream("kafka", "PageViewEvent");
- }
-
- @Override public Properties getProperties() {
- return null;
- }
- };
-
- StreamSpec output = new StreamSpec() {
- @Override public SystemStream getSystemStream() {
- return new SystemStream("kafka", "PageViewPerMember5min");
- }
-
- @Override public Properties getProperties() {
- return null;
- }
- };
-
- class PageViewEvent implements MessageEnvelope<String, PageViewEvent> {
- String pageId;
- String memberId;
- long timestamp;
-
- PageViewEvent(String pageId, String memberId, long timestamp) {
- this.pageId = pageId;
- this.memberId = memberId;
- this.timestamp = timestamp;
- }
-
- @Override
- public String getKey() {
- return this.pageId;
- }
-
- @Override
- public PageViewEvent getMessage() {
- return this;
- }
- }
-
- class MyStreamOutput implements MessageEnvelope<String, MyStreamOutput> {
- String memberId;
- long timestamp;
- int count;
-
- MyStreamOutput(WindowPane<String, Integer> m) {
- this.memberId = m.getKey().getKey();
- this.timestamp = Long.valueOf(m.getKey().getPaneId());
- this.count = m.getMessage();
- }
-
- @Override
- public String getKey() {
- return this.memberId;
- }
-
- @Override
- public MyStreamOutput getMessage() {
- return this;
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-operator/src/test/java/org/apache/samza/example/RepartitionExample.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/RepartitionExample.java b/samza-operator/src/test/java/org/apache/samza/example/RepartitionExample.java
deleted file mode 100644
index 6994ac4..0000000
--- a/samza-operator/src/test/java/org/apache/samza/example/RepartitionExample.java
+++ /dev/null
@@ -1,140 +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.example;
-
-import org.apache.samza.operators.*;
-import org.apache.samza.operators.StreamGraphBuilder;
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.data.MessageEnvelope;
-import org.apache.samza.operators.windows.WindowPane;
-import org.apache.samza.operators.windows.Windows;
-import org.apache.samza.serializers.JsonSerde;
-import org.apache.samza.serializers.StringSerde;
-import org.apache.samza.system.ExecutionEnvironment;
-import org.apache.samza.system.SystemStream;
-import org.apache.samza.util.CommandLine;
-
-import java.time.Duration;
-import java.util.*;
-
-
-/**
- * Example {@link StreamGraphBuilder} code to test the API methods with re-partition operator
- */
-public class RepartitionExample implements StreamGraphBuilder {
-
- /**
- * used by remote execution environment to launch the job in remote program. The remote program should follow the similar
- * invoking context as in standalone:
- *
- * public static void main(String args[]) throws Exception {
- * CommandLine cmdLine = new CommandLine();
- * Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
- * ExecutionEnvironment remoteEnv = ExecutionEnvironment.getRemoteEnvironment(config);
- * remoteEnv.run(new UserMainExample(), config);
- * }
- *
- */
- @Override public void init(StreamGraph graph, Config config) {
-
- MessageStream<PageViewEvent> pageViewEvents = graph.createInStream(input1, new StringSerde("UTF-8"), new JsonSerde<>());
- OutputStream<MyStreamOutput> pageViewPerMemberCounters = graph.createOutStream(output, new StringSerde("UTF-8"), new JsonSerde<>());
-
- pageViewEvents.
- partitionBy(m -> m.getMessage().memberId).
- window(Windows.<PageViewEvent, String, Integer>keyedTumblingWindow(
- msg -> msg.getMessage().memberId, Duration.ofMinutes(5), (m, c) -> c + 1)).
- map(MyStreamOutput::new).
- sendTo(pageViewPerMemberCounters);
-
- }
-
- // standalone local program model
- public static void main(String[] args) throws Exception {
- CommandLine cmdLine = new CommandLine();
- Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
- ExecutionEnvironment standaloneEnv = ExecutionEnvironment.getLocalEnvironment(config);
- standaloneEnv.run(new RepartitionExample(), config);
- }
-
- StreamSpec input1 = new StreamSpec() {
- @Override public SystemStream getSystemStream() {
- return new SystemStream("kafka", "PageViewEvent");
- }
-
- @Override public Properties getProperties() {
- return null;
- }
- };
-
- StreamSpec output = new StreamSpec() {
- @Override public SystemStream getSystemStream() {
- return new SystemStream("kafka", "PageViewPerMember5min");
- }
-
- @Override public Properties getProperties() {
- return null;
- }
- };
-
- class PageViewEvent implements MessageEnvelope<String, PageViewEvent> {
- String pageId;
- String memberId;
- long timestamp;
-
- PageViewEvent(String pageId, String memberId, long timestamp) {
- this.pageId = pageId;
- this.memberId = memberId;
- this.timestamp = timestamp;
- }
-
- @Override
- public String getKey() {
- return this.pageId;
- }
-
- @Override
- public PageViewEvent getMessage() {
- return this;
- }
- }
-
- class MyStreamOutput implements MessageEnvelope<String, MyStreamOutput> {
- String memberId;
- long timestamp;
- int count;
-
- MyStreamOutput(WindowPane<String, Integer> m) {
- this.memberId = m.getKey().getKey();
- this.timestamp = Long.valueOf(m.getKey().getPaneId());
- this.count = m.getMessage();
- }
-
- @Override
- public String getKey() {
- return this.memberId;
- }
-
- @Override
- public MyStreamOutput getMessage() {
- return this;
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-operator/src/test/java/org/apache/samza/example/TestBasicStreamGraphs.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/TestBasicStreamGraphs.java b/samza-operator/src/test/java/org/apache/samza/example/TestBasicStreamGraphs.java
deleted file mode 100644
index 8ecd44f..0000000
--- a/samza-operator/src/test/java/org/apache/samza/example/TestBasicStreamGraphs.java
+++ /dev/null
@@ -1,99 +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.example;
-
-import java.lang.reflect.Field;
-import org.apache.samza.Partition;
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.impl.OperatorGraph;
-import org.apache.samza.system.SystemStreamPartition;
-import org.apache.samza.task.StreamOperatorTask;
-import org.apache.samza.task.TaskContext;
-import org.junit.Test;
-
-import java.util.HashSet;
-import java.util.Set;
-
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
-
-
-/**
- * Unit test for {@link StreamOperatorTask}
- */
-public class TestBasicStreamGraphs {
-
- private final Set<SystemStreamPartition> inputPartitions = new HashSet<SystemStreamPartition>() { {
- for (int i = 0; i < 4; i++) {
- this.add(new SystemStreamPartition("my-system", String.format("my-topic%d", i), 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);
- TestWindowExample userTask = new TestWindowExample(this.inputPartitions);
- StreamOperatorTask adaptorTask = new StreamOperatorTask(userTask);
- Field pipelineMapFld = StreamOperatorTask.class.getDeclaredField("operatorGraph");
- pipelineMapFld.setAccessible(true);
- OperatorGraph opGraph = (OperatorGraph) pipelineMapFld.get(adaptorTask);
-
- adaptorTask.init(mockConfig, mockContext);
- this.inputPartitions.forEach(partition -> {
- assertNotNull(opGraph.get(partition.getSystemStream()));
- });
- }
-
- @Test
- public void testSplitTask() throws Exception {
- Config mockConfig = mock(Config.class);
- TaskContext mockContext = mock(TaskContext.class);
- when(mockContext.getSystemStreamPartitions()).thenReturn(this.inputPartitions);
- TestBroadcastExample splitTask = new TestBroadcastExample(this.inputPartitions);
- StreamOperatorTask adaptorTask = new StreamOperatorTask(splitTask);
- Field pipelineMapFld = StreamOperatorTask.class.getDeclaredField("operatorGraph");
- pipelineMapFld.setAccessible(true);
- OperatorGraph opGraph = (OperatorGraph) pipelineMapFld.get(adaptorTask);
-
- adaptorTask.init(mockConfig, mockContext);
- this.inputPartitions.forEach(partition -> {
- assertNotNull(opGraph.get(partition.getSystemStream()));
- });
- }
-
- @Test
- public void testJoinTask() throws Exception {
- Config mockConfig = mock(Config.class);
- TaskContext mockContext = mock(TaskContext.class);
- when(mockContext.getSystemStreamPartitions()).thenReturn(this.inputPartitions);
- TestJoinExample joinTask = new TestJoinExample(this.inputPartitions);
- StreamOperatorTask adaptorTask = new StreamOperatorTask(joinTask);
- Field pipelineMapFld = StreamOperatorTask.class.getDeclaredField("operatorGraph");
- pipelineMapFld.setAccessible(true);
- OperatorGraph opGraph = (OperatorGraph) pipelineMapFld.get(adaptorTask);
-
- adaptorTask.init(mockConfig, mockContext);
- this.inputPartitions.forEach(partition -> {
- assertNotNull(opGraph.get(partition.getSystemStream()));
- });
- }
-
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-operator/src/test/java/org/apache/samza/example/TestBroadcastExample.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/TestBroadcastExample.java b/samza-operator/src/test/java/org/apache/samza/example/TestBroadcastExample.java
deleted file mode 100644
index d22324b..0000000
--- a/samza-operator/src/test/java/org/apache/samza/example/TestBroadcastExample.java
+++ /dev/null
@@ -1,113 +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.example;
-
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.StreamGraph;
-import org.apache.samza.operators.StreamSpec;
-import org.apache.samza.operators.data.InputMessageEnvelope;
-import org.apache.samza.operators.data.JsonIncomingSystemMessageEnvelope;
-import org.apache.samza.operators.data.Offset;
-import org.apache.samza.operators.triggers.Triggers;
-import org.apache.samza.operators.windows.Windows;
-import org.apache.samza.system.SystemStream;
-import org.apache.samza.system.SystemStreamPartition;
-
-import java.time.Duration;
-import java.util.function.BiFunction;
-import java.util.Properties;
-import java.util.Set;
-
-
-/**
- * Example implementation of split stream tasks
- *
- */
-public class TestBroadcastExample extends TestExampleBase {
-
- TestBroadcastExample(Set<SystemStreamPartition> inputs) {
- super(inputs);
- }
-
- 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 init(StreamGraph graph, Config config) {
- BiFunction<JsonMessageEnvelope, Integer, Integer> sumAggregator = (m, c) -> c + 1;
- inputs.keySet().forEach(entry -> {
- MessageStream<JsonMessageEnvelope> inputStream = graph.<Object, Object, InputMessageEnvelope>createInStream(new StreamSpec() {
- @Override public SystemStream getSystemStream() {
- return entry;
- }
-
- @Override public Properties getProperties() {
- return null;
- }
- }, null, null).map(this::getInputMessage);
-
- inputStream.filter(this::myFilter1).window(Windows.tumblingWindow(Duration.ofMillis(100), sumAggregator)
- .setLateTrigger(Triggers.any(Triggers.count(30000), Triggers.timeSinceFirstMessage(Duration.ofMillis(10)))));
-
- inputStream.filter(this::myFilter2).window(Windows.tumblingWindow(Duration.ofMillis(100), sumAggregator)
- .setLateTrigger(Triggers.any(Triggers.count(30000), Triggers.timeSinceFirstMessage(Duration.ofMillis(10)))));
-
- inputStream.filter(this::myFilter3).window(Windows.tumblingWindow(Duration.ofMillis(100), sumAggregator)
- .setLateTrigger(Triggers.any(Triggers.count(30000), Triggers.timeSinceFirstMessage(Duration.ofMillis(10)))));
-
- });
- }
-
- JsonMessageEnvelope getInputMessage(InputMessageEnvelope 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/8515448a/samza-operator/src/test/java/org/apache/samza/example/TestExampleBase.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/TestExampleBase.java b/samza-operator/src/test/java/org/apache/samza/example/TestExampleBase.java
deleted file mode 100644
index c4df9d4..0000000
--- a/samza-operator/src/test/java/org/apache/samza/example/TestExampleBase.java
+++ /dev/null
@@ -1,46 +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.example;
-
-import org.apache.samza.operators.StreamGraphBuilder;
-import org.apache.samza.system.SystemStream;
-import org.apache.samza.system.SystemStreamPartition;
-
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * Base class for test examples
- *
- */
-public abstract class TestExampleBase implements StreamGraphBuilder {
-
- protected final Map<SystemStream, Set<SystemStreamPartition>> inputs;
-
- TestExampleBase(Set<SystemStreamPartition> inputs) {
- this.inputs = new HashMap<>();
- for (SystemStreamPartition input : inputs) {
- this.inputs.putIfAbsent(input.getSystemStream(), new HashSet<>());
- this.inputs.get(input.getSystemStream()).add(input);
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-operator/src/test/java/org/apache/samza/example/TestJoinExample.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/TestJoinExample.java b/samza-operator/src/test/java/org/apache/samza/example/TestJoinExample.java
deleted file mode 100644
index fe6e7e7..0000000
--- a/samza-operator/src/test/java/org/apache/samza/example/TestJoinExample.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.example;
-
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.StreamGraph;
-import org.apache.samza.operators.StreamSpec;
-import org.apache.samza.operators.data.InputMessageEnvelope;
-import org.apache.samza.operators.data.JsonIncomingSystemMessageEnvelope;
-import org.apache.samza.operators.data.Offset;
-import org.apache.samza.operators.functions.JoinFunction;
-import org.apache.samza.serializers.JsonSerde;
-import org.apache.samza.serializers.StringSerde;
-import org.apache.samza.system.SystemStream;
-import org.apache.samza.system.SystemStreamPartition;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Properties;
-import java.util.Set;
-
-
-/**
- * Example implementation of unique key-based stream-stream join tasks
- *
- */
-public class TestJoinExample extends TestExampleBase {
-
- TestJoinExample(Set<SystemStreamPartition> inputs) {
- super(inputs);
- }
-
- 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 init(StreamGraph graph, Config config) {
-
- for (SystemStream input : inputs.keySet()) {
- MessageStream<JsonMessageEnvelope> newSource = graph.<Object, Object, InputMessageEnvelope>createInStream(
- new StreamSpec() {
- @Override public SystemStream getSystemStream() {
- return input;
- }
-
- @Override public Properties getProperties() {
- return null;
- }
- }, null, null).map(this::getInputMessage);
- if (joinOutput == null) {
- joinOutput = newSource;
- } else {
- joinOutput = joinOutput.join(newSource, new MyJoinFunction());
- }
- }
-
- joinOutput.sendTo(graph.createOutStream(new StreamSpec() {
- @Override public SystemStream getSystemStream() {
- return null;
- }
-
- @Override public Properties getProperties() {
- return null;
- }
- }, new StringSerde("UTF-8"), new JsonSerde<>()));
-
- }
-
- private JsonMessageEnvelope getInputMessage(InputMessageEnvelope ism) {
- return new JsonMessageEnvelope(
- ((MessageType) ism.getMessage()).joinKey,
- (MessageType) ism.getMessage(),
- ism.getOffset(),
- ism.getSystemStreamPartition());
- }
-
- class MyJoinFunction implements JoinFunction<String, JsonMessageEnvelope, JsonMessageEnvelope, JsonMessageEnvelope> {
- 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);
- }
-
- @Override
- public JsonMessageEnvelope apply(JsonMessageEnvelope message, JsonMessageEnvelope otherMessage) {
- return this.myJoinResult(message, otherMessage);
- }
-
- @Override
- public String getFirstKey(JsonMessageEnvelope message) {
- return message.getKey();
- }
-
- @Override
- public String getSecondKey(JsonMessageEnvelope message) {
- return message.getKey();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/samza-operator/src/test/java/org/apache/samza/example/TestWindowExample.java
----------------------------------------------------------------------
diff --git a/samza-operator/src/test/java/org/apache/samza/example/TestWindowExample.java b/samza-operator/src/test/java/org/apache/samza/example/TestWindowExample.java
deleted file mode 100644
index e08ca20..0000000
--- a/samza-operator/src/test/java/org/apache/samza/example/TestWindowExample.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.example;
-
-import org.apache.samza.config.Config;
-import org.apache.samza.operators.StreamGraph;
-import org.apache.samza.operators.StreamSpec;
-import org.apache.samza.operators.data.InputMessageEnvelope;
-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.Windows;
-import org.apache.samza.system.SystemStream;
-import org.apache.samza.system.SystemStreamPartition;
-
-import java.time.Duration;
-import java.util.function.BiFunction;
-import java.util.Properties;
-import java.util.Set;
-
-
-/**
- * Example implementation of a simple user-defined tasks w/ window operators
- *
- */
-public class TestWindowExample extends TestExampleBase {
- class MessageType {
- String field1;
- String field2;
- }
-
- TestWindowExample(Set<SystemStreamPartition> inputs) {
- super(inputs);
- }
-
- class JsonMessageEnvelope extends JsonIncomingSystemMessageEnvelope<MessageType> {
-
- JsonMessageEnvelope(String key, MessageType data, Offset offset, SystemStreamPartition partition) {
- super(key, data, offset, partition);
- }
- }
-
- @Override
- public void init(StreamGraph graph, Config config) {
- BiFunction<JsonMessageEnvelope, Integer, Integer> maxAggregator = (m, c) -> c + 1;
- inputs.keySet().forEach(source -> graph.<Object, Object, InputMessageEnvelope>createInStream(new StreamSpec() {
- @Override public SystemStream getSystemStream() {
- return source;
- }
-
- @Override public Properties getProperties() {
- return null;
- }
- }, null, null).
- map(m1 -> new JsonMessageEnvelope(this.myMessageKeyFunction(m1), (MessageType) m1.getMessage(), m1.getOffset(),
- m1.getSystemStreamPartition())).window(Windows.tumblingWindow(Duration.ofMillis(200), maxAggregator)));
-
- }
-
- String myMessageKeyFunction(MessageEnvelope<Object, Object> m) {
- return m.getKey().toString();
- }
-
-}
http://git-wip-us.apache.org/repos/asf/samza/blob/8515448a/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
deleted file mode 100644
index 160a47a..0000000
--- a/samza-operator/src/test/java/org/apache/samza/operators/TestMessageStreamImpl.java
+++ /dev/null
@@ -1,204 +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.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.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 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 TestMessageStreamImpl {
-
- private StreamGraphImpl mockGraph = mock(StreamGraphImpl.class);
-
- @Test
- public void testMap() {
- MessageStreamImpl<TestMessageEnvelope> inputStream = new MessageStreamImpl<>(mockGraph);
- MapFunction<TestMessageEnvelope, TestOutputMessageEnvelope> xMap = (TestMessageEnvelope 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.getNextStream(), 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<>(mockGraph);
- 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 = (TestMessageEnvelope message) -> 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.getNextStream(), 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<>(mockGraph);
- FilterFunction<TestMessageEnvelope> xFilter = (TestMessageEnvelope 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.getNextStream(), 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<>(mockGraph);
- SinkFunction<TestMessageEnvelope> xSink = (TestMessageEnvelope m, MessageCollector mc, TaskCoordinator 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).getNextStream());
- }
-
- @Test
- public void testJoin() {
- MessageStreamImpl<TestMessageEnvelope> source1 = new MessageStreamImpl<>(mockGraph);
- MessageStreamImpl<TestMessageEnvelope> source2 = new MessageStreamImpl<>(mockGraph);
- JoinFunction<String, TestMessageEnvelope, TestMessageEnvelope, TestOutputMessageEnvelope> joiner =
- new JoinFunction<String, TestMessageEnvelope, TestMessageEnvelope, TestOutputMessageEnvelope>() {
- @Override
- public TestOutputMessageEnvelope apply(TestMessageEnvelope m1, TestMessageEnvelope m2) {
- return new TestOutputMessageEnvelope(m1.getKey(), m1.getMessage().getValue().length() + m2.getMessage().getValue().length());
- }
-
- @Override
- public String getFirstKey(TestMessageEnvelope message) {
- return message.getKey();
- }
-
- @Override
- public String getSecondKey(TestMessageEnvelope message) {
- return message.getKey();
- }
- };
-
- 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).getNextStream(), joinOutput);
- subs = source2.getRegisteredOperatorSpecs();
- assertEquals(subs.size(), 1);
- OperatorSpec<TestMessageEnvelope> joinOp2 = subs.iterator().next();
- assertTrue(joinOp2 instanceof PartialJoinOperatorSpec);
- assertEquals(((PartialJoinOperatorSpec) joinOp2).getNextStream(), 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<>(mockGraph);
- Collection<MessageStream<TestMessageEnvelope>> others = new ArrayList<MessageStream<TestMessageEnvelope>>() { {
- this.add(new MessageStreamImpl<>(mockGraph));
- this.add(new MessageStreamImpl<>(mockGraph));
- } };
- 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).getNextStream(), 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);
- }
-}