From 4fe5458f45e1f1a9da84704d7c4d7eaff3b625bb Mon Sep 17 00:00:00 2001 From: Kam Kasravi Date: Tue, 11 Oct 2016 10:32:04 +0800 Subject: [PATCH 1/9] Fixes #22 support akka-streams Gearpump Materializer Author: Kam Kasravi Closes #94 from kkasravi/GEARPUMP-22. --- .../apache/gearpump/cluster/UserConfig.scala | 21 +- .../cluster/master/AppMasterLauncher.scala | 15 +- experiments/akkastream/README.md | 4 +- .../src/main/resources/geardefault.conf | 5 +- .../scala/akka/stream/BaseMaterializer.scala | 47 -- .../main/scala/akka/stream/ModuleGraph.scala | 298 --------- .../gearpump/GearpumpMaterializer.scala | 71 --- .../akka/stream/gearpump/example/Test.scala | 69 -- .../akka/stream/gearpump/example/Test6.scala | 67 -- .../materializer/LocalMaterializer.scala | 152 ----- .../materializer/LocalMaterializerImpl.scala | 284 --------- .../materializer/RemoteMaterializerImpl.scala | 453 ------------- .../gearpump/akkastream}/GearAttributes.scala | 13 +- .../akkastream/GearpumpMaterializer.scala | 290 +++++++++ .../GearpumpMaterializerSession.scala | 152 +++++ .../gearpump/akkastream/example/Test.scala} | 48 +- .../gearpump/akkastream/example/Test10.scala | 82 +++ .../gearpump/akkastream/example/Test11.scala | 72 +++ .../gearpump/akkastream/example/Test12.scala | 81 +++ .../gearpump/akkastream/example/Test13.scala | 177 ++++++ .../gearpump/akkastream/example/Test14.scala | 73 +++ .../gearpump/akkastream/example/Test15.scala | 72 +++ .../gearpump/akkastream/example/Test16.scala | 50 ++ .../gearpump/akkastream}/example/Test2.scala | 58 +- .../gearpump/akkastream/example/Test3.scala | 70 +++ .../gearpump/akkastream}/example/Test4.scala | 48 +- .../gearpump/akkastream}/example/Test5.scala | 67 +- .../gearpump/akkastream/example/Test6.scala | 90 +++ .../gearpump/akkastream/example/Test7.scala | 56 ++ .../gearpump/akkastream/example/Test8.scala | 66 ++ .../gearpump/akkastream/example/Test9.scala | 87 +++ .../akkastream}/example/WikipediaApp.scala | 98 +-- .../akkastream/graph/GraphPartitioner.scala} | 96 +-- .../akkastream}/graph/LocalGraph.scala | 55 +- .../akkastream}/graph/RemoteGraph.scala | 53 +- .../gearpump/akkastream}/graph/SubGraph.scala | 21 +- .../materializer/LocalMaterializerImpl.scala | 319 ++++++++++ .../materializer/RemoteMaterializerImpl.scala | 594 ++++++++++++++++++ .../akkastream}/module/BridgeModule.scala | 101 +-- .../akkastream}/module/DummyModule.scala | 22 +- .../module/GearpumpTaskModule.scala | 86 +-- .../akkastream}/module/GroupByModule.scala | 37 +- .../akkastream}/module/ReduceModule.scala | 32 +- .../gearpump/akkastream}/scaladsl/Api.scala | 47 +- .../akkastream}/task/BalanceTask.scala | 11 +- .../gearpump/akkastream/task/BatchTask.scala | 50 ++ .../akkastream}/task/BroadcastTask.scala | 11 +- .../gearpump/akkastream/task/ConcatTask.scala | 38 ++ .../akkastream/task/DelayInitialTask.scala | 61 ++ .../akkastream/task/DropWithinTask.scala | 62 ++ .../akkastream/task/FlattenMergeTask.scala | 38 ++ .../gearpump/akkastream/task/FoldTask.scala | 56 ++ .../gearpump/akkastream}/task/GraphTask.scala | 35 +- .../akkastream/task/GroupedWithinTask.scala | 44 ++ .../akkastream/task/InterleaveTask.scala | 44 ++ .../akkastream/task/MapAsyncTask.scala | 53 ++ .../gearpump/akkastream/task/MergeTask.scala | 39 ++ .../akkastream/task/SingleSourceTask.scala | 43 ++ .../akkastream}/task/SinkBridgeTask.scala | 42 +- .../akkastream}/task/SourceBridgeTask.scala | 45 +- .../task/StatefulMapConcatTask.scala | 50 ++ .../akkastream/task/TakeWithinTask.scala | 62 ++ .../akkastream/task/ThrottleTask.scala | 53 ++ .../akkastream/task/TickSourceTask.scala | 56 ++ .../akkastream/task/Unzip2Task.scala} | 25 +- .../gearpump/akkastream/task/Zip2Task.scala | 57 ++ .../util/MaterializedValueOps.scala | 14 +- .../gearpump/akkastream}/AttributesSpec.scala | 5 +- project/Build.scala | 35 +- project/Pack.scala | 16 +- .../scalastyle_config.xml | 0 .../gearpump/services/AppMasterService.scala | 2 +- .../gearpump/services/MasterService.scala | 2 +- .../gearpump/services/SecurityService.scala | 4 +- .../gearpump/services/StaticService.scala | 8 +- .../gearpump/services/WorkerService.scala | 2 +- .../streaming/dsl/plan/OpTranslator.scala | 2 +- .../gearpump/streaming/task/TaskActor.scala | 13 +- 78 files changed, 3809 insertions(+), 1968 deletions(-) delete mode 100644 experiments/akkastream/src/main/scala/akka/stream/BaseMaterializer.scala delete mode 100644 experiments/akkastream/src/main/scala/akka/stream/ModuleGraph.scala delete mode 100644 experiments/akkastream/src/main/scala/akka/stream/gearpump/GearpumpMaterializer.scala delete mode 100644 experiments/akkastream/src/main/scala/akka/stream/gearpump/example/Test.scala delete mode 100644 experiments/akkastream/src/main/scala/akka/stream/gearpump/example/Test6.scala delete mode 100644 experiments/akkastream/src/main/scala/akka/stream/gearpump/materializer/LocalMaterializer.scala delete mode 100644 experiments/akkastream/src/main/scala/akka/stream/gearpump/materializer/LocalMaterializerImpl.scala delete mode 100644 experiments/akkastream/src/main/scala/akka/stream/gearpump/materializer/RemoteMaterializerImpl.scala rename experiments/akkastream/src/main/scala/{akka/stream/gearpump => org/apache/gearpump/akkastream}/GearAttributes.scala (92%) create mode 100644 experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearpumpMaterializer.scala create mode 100644 experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearpumpMaterializerSession.scala rename experiments/akkastream/src/main/scala/{akka/stream/gearpump/example/Test3.scala => org/apache/gearpump/akkastream/example/Test.scala} (51%) create mode 100644 experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test10.scala create mode 100644 experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test11.scala create mode 100644 experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test12.scala create mode 100644 experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test13.scala create mode 100644 experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test14.scala create mode 100644 experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test15.scala create mode 100644 experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test16.scala rename experiments/akkastream/src/main/scala/{akka/stream/gearpump => org/apache/gearpump/akkastream}/example/Test2.scala (52%) create mode 100644 experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test3.scala rename experiments/akkastream/src/main/scala/{akka/stream/gearpump => org/apache/gearpump/akkastream}/example/Test4.scala (51%) rename experiments/akkastream/src/main/scala/{akka/stream/gearpump => org/apache/gearpump/akkastream}/example/Test5.scala (51%) create mode 100644 experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test6.scala create mode 100644 experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test7.scala create mode 100644 experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test8.scala create mode 100644 experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test9.scala rename experiments/akkastream/src/main/scala/{akka/stream/gearpump => org/apache/gearpump/akkastream}/example/WikipediaApp.scala (61%) rename experiments/akkastream/src/main/scala/{akka/stream/gearpump/graph/GraphCutter.scala => org/apache/gearpump/akkastream/graph/GraphPartitioner.scala} (65%) rename experiments/akkastream/src/main/scala/{akka/stream/gearpump => org/apache/gearpump/akkastream}/graph/LocalGraph.scala (55%) rename experiments/akkastream/src/main/scala/{akka/stream/gearpump => org/apache/gearpump/akkastream}/graph/RemoteGraph.scala (63%) rename experiments/akkastream/src/main/scala/{akka/stream/gearpump => org/apache/gearpump/akkastream}/graph/SubGraph.scala (74%) create mode 100644 experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/materializer/LocalMaterializerImpl.scala create mode 100644 experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/materializer/RemoteMaterializerImpl.scala rename experiments/akkastream/src/main/scala/{akka/stream/gearpump => org/apache/gearpump/akkastream}/module/BridgeModule.scala (52%) rename experiments/akkastream/src/main/scala/{akka/stream/gearpump => org/apache/gearpump/akkastream}/module/DummyModule.scala (89%) rename experiments/akkastream/src/main/scala/{akka/stream/gearpump => org/apache/gearpump/akkastream}/module/GearpumpTaskModule.scala (66%) rename experiments/akkastream/src/main/scala/{akka/stream/gearpump => org/apache/gearpump/akkastream}/module/GroupByModule.scala (56%) rename experiments/akkastream/src/main/scala/{akka/stream/gearpump => org/apache/gearpump/akkastream}/module/ReduceModule.scala (59%) rename experiments/akkastream/src/main/scala/{akka/stream/gearpump => org/apache/gearpump/akkastream}/scaladsl/Api.scala (91%) rename experiments/akkastream/src/main/scala/{akka/stream/gearpump => org/apache/gearpump/akkastream}/task/BalanceTask.scala (80%) create mode 100644 experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/BatchTask.scala rename experiments/akkastream/src/main/scala/{akka/stream/gearpump => org/apache/gearpump/akkastream}/task/BroadcastTask.scala (78%) create mode 100644 experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/ConcatTask.scala create mode 100644 experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/DelayInitialTask.scala create mode 100644 experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/DropWithinTask.scala create mode 100644 experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/FlattenMergeTask.scala create mode 100644 experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/FoldTask.scala rename experiments/akkastream/src/main/scala/{akka/stream/gearpump => org/apache/gearpump/akkastream}/task/GraphTask.scala (63%) create mode 100644 experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/GroupedWithinTask.scala create mode 100644 experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/InterleaveTask.scala create mode 100644 experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/MapAsyncTask.scala create mode 100644 experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/MergeTask.scala create mode 100644 experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/SingleSourceTask.scala rename experiments/akkastream/src/main/scala/{akka/stream/gearpump => org/apache/gearpump/akkastream}/task/SinkBridgeTask.scala (77%) rename experiments/akkastream/src/main/scala/{akka/stream/gearpump => org/apache/gearpump/akkastream}/task/SourceBridgeTask.scala (70%) create mode 100644 experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/StatefulMapConcatTask.scala create mode 100644 experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/TakeWithinTask.scala create mode 100644 experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/ThrottleTask.scala create mode 100644 experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/TickSourceTask.scala rename experiments/akkastream/src/main/scala/{akka/stream/gearpump/task/UnZip2Task.scala => org/apache/gearpump/akkastream/task/Unzip2Task.scala} (61%) create mode 100644 experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/Zip2Task.scala rename experiments/akkastream/src/main/scala/{akka/stream/gearpump => org/apache/gearpump/akkastream}/util/MaterializedValueOps.scala (75%) rename experiments/akkastream/src/test/scala/{akka/stream/gearpump => org/apache/gearpump/akkastream}/AttributesSpec.scala (92%) rename scalastyle-config.xml => project/scalastyle_config.xml (100%) diff --git a/core/src/main/scala/org/apache/gearpump/cluster/UserConfig.scala b/core/src/main/scala/org/apache/gearpump/cluster/UserConfig.scala index 3f4280892..393d5f7ac 100644 --- a/core/src/main/scala/org/apache/gearpump/cluster/UserConfig.scala +++ b/core/src/main/scala/org/apache/gearpump/cluster/UserConfig.scala @@ -20,13 +20,16 @@ package org.apache.gearpump.cluster import akka.actor.{ActorSystem, ExtendedActorSystem} import akka.serialization.JavaSerializer - import org.apache.gearpump.google.common.io.BaseEncoding +import org.apache.gearpump.util.LogUtil + +import scala.util.{Failure, Success, Try} /** * Immutable configuration */ final class UserConfig(private val _config: Map[String, String]) extends Serializable { + private val LOG = LogUtil.getLogger(getClass) def withBoolean(key: String, value: Boolean): UserConfig = { new UserConfig(_config + (key -> value.toString)) @@ -135,10 +138,18 @@ final class UserConfig(private val _config: Map[String, String]) extends Seriali if (null == value) { this } else { - val serializer = new JavaSerializer(system.asInstanceOf[ExtendedActorSystem]) - val bytes = serializer.toBinary(value) - val encoded = BaseEncoding.base64().encode(bytes) - this.withString(key, encoded) + Try({ + val serializer = new JavaSerializer(system.asInstanceOf[ExtendedActorSystem]) + val bytes = serializer.toBinary(value) + BaseEncoding.base64().encode(bytes) + }) match { + case Success(enc) => + this.withString(key, enc) + case Failure(throwable) => + LOG.error(s"Could not serialize value with key $key ${throwable.getMessage}") + this + } + } } // scalastyle:on line.size.limit diff --git a/core/src/main/scala/org/apache/gearpump/cluster/master/AppMasterLauncher.scala b/core/src/main/scala/org/apache/gearpump/cluster/master/AppMasterLauncher.scala index 9305d5c35..de104b9f1 100644 --- a/core/src/main/scala/org/apache/gearpump/cluster/master/AppMasterLauncher.scala +++ b/core/src/main/scala/org/apache/gearpump/cluster/master/AppMasterLauncher.scala @@ -19,16 +19,9 @@ package org.apache.gearpump.cluster.master import java.util.concurrent.{TimeUnit, TimeoutException} -import org.apache.gearpump.cluster.worker.WorkerId - -import scala.collection.JavaConverters._ -import scala.concurrent.duration.Duration -import scala.util.{Failure, Success} import akka.actor.{Actor, ActorRef, Props, _} import com.typesafe.config.Config -import org.slf4j.Logger - import org.apache.gearpump.cluster.AppMasterToMaster.RequestResource import org.apache.gearpump.cluster.AppMasterToWorker.{LaunchExecutor, ShutdownExecutor} import org.apache.gearpump.cluster.MasterToAppMaster.ResourceAllocated @@ -36,11 +29,17 @@ import org.apache.gearpump.cluster.MasterToClient.SubmitApplicationResult import org.apache.gearpump.cluster.WorkerToAppMaster.ExecutorLaunchRejected import org.apache.gearpump.cluster.appmaster.{AppMasterRuntimeEnvironment, AppMasterRuntimeInfo, WorkerInfo} import org.apache.gearpump.cluster.scheduler.{Resource, ResourceAllocation, ResourceRequest} +import org.apache.gearpump.cluster.worker.WorkerId import org.apache.gearpump.cluster.{AppDescription, AppJar, _} import org.apache.gearpump.transport.HostPort import org.apache.gearpump.util.ActorSystemBooter._ import org.apache.gearpump.util.Constants._ import org.apache.gearpump.util.{ActorSystemBooter, ActorUtil, LogUtil, Util} +import org.slf4j.Logger + +import scala.collection.JavaConverters._ +import scala.concurrent.duration.Duration +import scala.util.{Failure, Success} /** * @@ -145,4 +144,4 @@ object AppMasterLauncher extends AppMasterLauncherFactory { trait AppMasterLauncherFactory { def props(appId: Int, executorId: Int, app: AppDescription, jar: Option[AppJar], username: String, master: ActorRef, client: Option[ActorRef]): Props -} \ No newline at end of file +} diff --git a/experiments/akkastream/README.md b/experiments/akkastream/README.md index 7c9a31654..fe045542d 100644 --- a/experiments/akkastream/README.md +++ b/experiments/akkastream/README.md @@ -1,4 +1,2 @@ Akka Stream -========= - -TODO: This directory is obsolte. Working on updating it to Akka 2.4.3. +========= \ No newline at end of file diff --git a/experiments/akkastream/src/main/resources/geardefault.conf b/experiments/akkastream/src/main/resources/geardefault.conf index 626a1dcc4..e9da531b0 100644 --- a/experiments/akkastream/src/main/resources/geardefault.conf +++ b/experiments/akkastream/src/main/resources/geardefault.conf @@ -2,4 +2,7 @@ gearpump.serializers { "akka.stream.gearpump.example.WikipediaApp$WikidataElement" = "" "scala.collection.immutable.Map$Map1" = "" "scala.collection.immutable.Map$Map2" = "" -} \ No newline at end of file +} +akka { + version = "2.4.10" +} diff --git a/experiments/akkastream/src/main/scala/akka/stream/BaseMaterializer.scala b/experiments/akkastream/src/main/scala/akka/stream/BaseMaterializer.scala deleted file mode 100644 index d2b328d89..000000000 --- a/experiments/akkastream/src/main/scala/akka/stream/BaseMaterializer.scala +++ /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 akka.stream - -import scala.concurrent.ExecutionContextExecutor - -/** - * [[BaseMaterializer]] is a extension to [[akka.stream.Materializer]]. - * - * Compared with [[akka.stream.Materializer]], the difference is that - * [[materialize]] accepts a [[ModuleGraph]] instead of a RunnableGraph. - * - * @see [[ModuleGraph]] for the difference between RunnableGraph and - * [[ModuleGraph]] - * - */ -abstract class BaseMaterializer extends akka.stream.Materializer { - - override def withNamePrefix(name: String): Materializer = throw new UnsupportedOperationException() - - override implicit def executionContext: ExecutionContextExecutor = throw new UnsupportedOperationException() - - def materialize[Mat](graph: ModuleGraph[Mat]): Mat - - override def materialize[Mat](runnableGraph: Graph[ClosedShape, Mat]): Mat = { - val graph = ModuleGraph(runnableGraph) - materialize(graph) - } - - def shutdown(): Unit -} diff --git a/experiments/akkastream/src/main/scala/akka/stream/ModuleGraph.scala b/experiments/akkastream/src/main/scala/akka/stream/ModuleGraph.scala deleted file mode 100644 index 48d06f7a2..000000000 --- a/experiments/akkastream/src/main/scala/akka/stream/ModuleGraph.scala +++ /dev/null @@ -1,298 +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. - */ - -/** - * This code is similar to MaterializerSession and will be deprecated in the next release. - */ - -package akka.stream - -import scala.collection.mutable - -import akka.stream.Attributes.Attribute -import akka.stream.ModuleGraph.Edge -import akka.stream.gearpump.util.MaterializedValueOps -import akka.stream.impl.StreamLayout._ -import akka.stream.impl._ -import akka.stream.{Graph => AkkaGraph} - -import _root_.org.apache.gearpump.util -import _root_.org.apache.gearpump.util.Graph - -/** - * - * ModuleGraph is a transformation on [[akka.stream.scaladsl.RunnableGraph]]. - * It carries all the information of [[akka.stream.scaladsl.RunnableGraph]], but - * represents it in a different way. - * - * Here is the difference: - * - * RunnableGraph - * ============================== - * [[akka.stream.scaladsl.RunnableGraph]] is represented as a [[Module]] tree: - * TopLevelModule - * | - * ------------------- - * | | - * SubModule1 SubModule2 - * | | - * ---------------- ---------- - * | | | - * AtomicModule1 AtomicModule2 AtomicModule3 - * - * ModuleGraph - * ============================== - * [[ModuleGraph]] is represented as a [[util.Graph]] of Atomic [[Module]]: - * - * AtomicModule2 -> AtomicModule4 - * /| \ - * / \ - * / \| - * AtomicModule1 AtomicModule5 - * \ /| - * \ / - * \| / - * AtomicModule3 - * - * Each vertex in the Graph is a [[Module]], each [[Edge]] in the Graph is a tuple - * ([[OutPort]], [[InPort]]). [[OutPort]] is one of upstream Atomic Module - * output ports. [[InPort]] is one of downstream Atomic Module input ports. - * - * - * Why use [[ModuleGraph]] instead of [[akka.stream.scaladsl.RunnableGraph]]? - * ========================= - * There are several good reasons:): - * 1. [[ModuleGraph]] outlines explicitly the upstream/downstream relation. - * Each [[Edge]] of [[ModuleGraph]] represent a upstream/downstream pair. - * It is easier for user to understand the overall data flow. - * - * 2. It is easier for performance optimization. - * For the above Graph, if we want to fuse AtomicModule2 and AtomicModule3 - * together, it can be done within [[ModuleGraph]]. We only need - * to substitute Pair(AtomicModule2, AtomicModule4) with a unified Module. - * - * 3. It avoids module duplication. - * In [[akka.stream.scaladsl.RunnableGraph]], composite Module can be re-used. - * It is possible that there can be duplicate Modules. - * The duplication problem causes big headache when doing materialization. - * - * [[ModuleGraph]] doesn't have thjis problem. [[ModuleGraph]] does a transformation on the Module - * Tree to make sure each Atomic Module [[ModuleGraph]] is unique. - * - * - * @param graph a Graph of Atomic modules. - * @param mat is a function of: - * input => materialized value of each Atomic module - * output => final materialized value. - * @tparam Mat - */ -class ModuleGraph[Mat](val graph: util.Graph[Module, Edge], val mat: MaterializedValueNode) { - - def resolve(materializedValues: Map[Module, Any]): Mat = { - MaterializedValueOps(mat).resolve[Mat](materializedValues) - } -} - -object ModuleGraph { - - def apply[Mat](runnableGraph: AkkaGraph[ClosedShape, Mat]): ModuleGraph[Mat] = { - val topLevel = runnableGraph.module - val factory = new ModuleGraphFactory(topLevel) - val (graph, mat) = factory.create() - new ModuleGraph(graph, mat) - } - - /** - * - * @param from outport of upstream module - * @param to inport of downstream module - */ - case class Edge(from: OutPort, to: InPort) - - private class ModuleGraphFactory(val topLevel: StreamLayout.Module) { - - private var subscribersStack: List[mutable.Map[InPort, (InPort, Module)]] = - mutable.Map.empty[InPort, (InPort, Module)].withDefaultValue(null) :: Nil - private var publishersStack: List[mutable.Map[OutPort, (OutPort, Module)]] = - mutable.Map.empty[OutPort, (OutPort, Module)].withDefaultValue(null) :: Nil - - /* - * Please note that this stack keeps track of the scoped modules wrapped in CopiedModule but not the CopiedModule - * itself. The reason is that the CopiedModule itself is only needed for the enterScope and exitScope methods but - * not elsewhere. For this reason they are just simply passed as parameters to those methods. - * - * The reason why the encapsulated (copied) modules are stored as mutable state to save subclasses of this class - * from passing the current scope around or even knowing about it. - */ - private var moduleStack: List[Module] = topLevel :: Nil - - private def subscribers: mutable.Map[InPort, (InPort, Module)] = subscribersStack.head - private def publishers: mutable.Map[OutPort, (OutPort, Module)] = publishersStack.head - private def currentLayout: Module = moduleStack.head - - private val graph = Graph.empty[Module, Edge] - - private def copyAtomicModule[T <: Module](module: T, parentAttributes: Attributes): T = { - val currentAttributes = mergeAttributes(parentAttributes, module.attributes) - module.withAttributes(currentAttributes).asInstanceOf[T] - } - - private def materializeAtomic(atomic: Module, parentAttributes: Attributes): MaterializedValueNode = { - val (inputs, outputs) = (atomic.shape.inlets, atomic.shape.outlets) - val copied = copyAtomicModule(atomic, parentAttributes) - - for ((in, id) <- inputs.zipWithIndex) { - val inPort = inPortMapping(atomic, copied)(in) - assignPort(in, (inPort, copied)) - } - - for ((out, id) <- outputs.zipWithIndex) { - val outPort = outPortMapping(atomic, copied)(out) - assignPort(out, (outPort, copied)) - } - - graph.addVertex(copied) - Atomic(copied) - } - - def create(): (util.Graph[Module, Edge], MaterializedValueNode) = { - val mat = materializeModule(topLevel, Attributes.none) - (graph, mat) - } - - private def outPortMapping(from: Module, to: Module): Map[OutPort, OutPort] = { - from.shape.outlets.iterator.zip(to.shape.outlets.iterator).toList.toMap - } - - private def inPortMapping(from: Module, to: Module): Map[InPort, InPort] = { - from.shape.inlets.iterator.zip(to.shape.inlets.iterator).toList.toMap - } - - private def materializeModule(module: Module, parentAttributes: Attributes): MaterializedValueNode = { - - val materializedValues = collection.mutable.HashMap.empty[Module, MaterializedValueNode] - val currentAttributes = mergeAttributes(parentAttributes, module.attributes) - - var materializedValueSources = List.empty[MaterializedValueSource[_]] - - for (submodule <- module.subModules) { - submodule match { - case mv: MaterializedValueSource[_] => - materializedValueSources :+= mv - case atomic if atomic.isAtomic => - materializedValues.put(atomic, materializeAtomic(atomic, currentAttributes)) - case copied: CopiedModule => - enterScope(copied) - materializedValues.put(copied, materializeModule(copied, currentAttributes)) - exitScope(copied) - case composite => - materializedValues.put(composite, materializeComposite(composite, currentAttributes)) - } - } - - val mat = resolveMaterialized(module.materializedValueComputation, materializedValues) - - materializedValueSources.foreach { module => - val matAttribute = new MaterializedValueSourceAttribute(mat) - val copied = copyAtomicModule(module, parentAttributes and Attributes(matAttribute)) - assignPort(module.shape.outlet, (copied.shape.outlet, copied)) - graph.addVertex(copied) - materializedValues.put(copied, Atomic(copied)) - } - mat - } - - private def materializeComposite(composite: Module, effectiveAttributes: Attributes): MaterializedValueNode = { - materializeModule(composite, effectiveAttributes) - } - - private def mergeAttributes(parent: Attributes, current: Attributes): Attributes = { - parent and current - } - - private def resolveMaterialized(matNode: MaterializedValueNode, materializedValues: collection.Map[Module, MaterializedValueNode]): MaterializedValueNode = matNode match { - case Atomic(m) => materializedValues(m) - case Combine(f, d1, d2) => Combine(f, resolveMaterialized(d1, materializedValues), resolveMaterialized(d2, materializedValues)) - case Transform(f, d) => Transform(f, resolveMaterialized(d, materializedValues)) - case Ignore => Ignore - } - - final protected def assignPort(in: InPort, subscriber: (InPort, Module)): Unit = { - addVertex(subscriber._2) - subscribers(in) = subscriber - // Interface (unconnected) ports of the current scope will be wired when exiting the scope - if (!currentLayout.inPorts(in)) { - val out = currentLayout.upstreams(in) - val publisher = publishers(out) - if (publisher ne null) addEdge(publisher, subscriber) - } - } - - final protected def assignPort(out: OutPort, publisher: (OutPort, Module)): Unit = { - addVertex(publisher._2) - publishers(out) = publisher - // Interface (unconnected) ports of the current scope will be wired when exiting the scope - if (!currentLayout.outPorts(out)) { - val in = currentLayout.downstreams(out) - val subscriber = subscribers(in) - if (subscriber ne null) addEdge(publisher, subscriber) - } - } - - // Enters a copied module and establishes a scope that prevents internals to leak out and interfere with copies - // of the same module. - // We don't store the enclosing CopiedModule itself as state since we don't use it anywhere else than exit and enter - private def enterScope(enclosing: CopiedModule): Unit = { - subscribersStack ::= mutable.Map.empty.withDefaultValue(null) - publishersStack ::= mutable.Map.empty.withDefaultValue(null) - moduleStack ::= enclosing.copyOf - } - - // Exits the scope of the copied module and propagates Publishers/Subscribers to the enclosing scope assigning - // them to the copied ports instead of the original ones (since there might be multiple copies of the same module - // leading to port identity collisions) - // We don't store the enclosing CopiedModule itself as state since we don't use it anywhere else than exit and enter - private def exitScope(enclosing: CopiedModule): Unit = { - val scopeSubscribers = subscribers - val scopePublishers = publishers - subscribersStack = subscribersStack.tail - publishersStack = publishersStack.tail - moduleStack = moduleStack.tail - - // When we exit the scope of a copied module, pick up the Subscribers/Publishers belonging to exposed ports of - // the original module and assign them to the copy ports in the outer scope that we will return to - enclosing.copyOf.shape.inlets.iterator.zip(enclosing.shape.inlets.iterator).foreach { - case (original, exposed) => assignPort(exposed, scopeSubscribers(original)) - } - - enclosing.copyOf.shape.outlets.iterator.zip(enclosing.shape.outlets.iterator).foreach { - case (original, exposed) => assignPort(exposed, scopePublishers(original)) - } - } - - private def addEdge(publisher: (OutPort, Module), subscriber: (InPort, Module)): Unit = { - graph.addEdge(publisher._2, Edge(publisher._1, subscriber._1), subscriber._2) - } - - private def addVertex(module: Module): Unit = { - graph.addVertex(module) - } - } - - final case class MaterializedValueSourceAttribute(mat: MaterializedValueNode) extends Attribute -} diff --git a/experiments/akkastream/src/main/scala/akka/stream/gearpump/GearpumpMaterializer.scala b/experiments/akkastream/src/main/scala/akka/stream/gearpump/GearpumpMaterializer.scala deleted file mode 100644 index a11d7cb3c..000000000 --- a/experiments/akkastream/src/main/scala/akka/stream/gearpump/GearpumpMaterializer.scala +++ /dev/null @@ -1,71 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package akka.stream.gearpump - -import akka.actor.ActorSystem -import akka.stream._ -import akka.stream.gearpump.graph.GraphCutter.Strategy -import akka.stream.gearpump.graph.LocalGraph.LocalGraphMaterializer -import akka.stream.gearpump.graph.RemoteGraph.RemoteGraphMaterializer -import akka.stream.gearpump.graph.{GraphCutter, LocalGraph, RemoteGraph, SubGraphMaterializer} -import akka.stream.impl.StreamLayout.Module - -/** - * - * [[GearpumpMaterializer]] allows you to render akka-stream DSL as a Gearpump - * streaming application. If some module cannot be rendered remotely in Gearpump - * Cluster, then it uses local Actor materializer as fallback to materialize - * the module locally. - * - * User can custom a [[Strategy]] to determinie which module should be rendered - * remotely, and which module should be rendered locally. - * - * @see [[GraphCutter]] to find out how we cut the [[ModuleGraph]] to two parts, - * and materialize them seperately. - * - * @param system - * @param strategy - * @param useLocalCluster whether to use built-in in-process local cluster - */ -class GearpumpMaterializer(system: ActorSystem, strategy: Strategy = GraphCutter.AllRemoteStrategy, - useLocalCluster: Boolean = true) - extends BaseMaterializer { - - private val subMaterializers: Map[Class[_], SubGraphMaterializer] = Map( - classOf[LocalGraph] -> new LocalGraphMaterializer(system), - classOf[RemoteGraph] -> new RemoteGraphMaterializer(useLocalCluster, system) - ) - - override def materialize[Mat](graph: ModuleGraph[Mat]): Mat = { - val subGraphs = new GraphCutter(strategy).cut(graph) - val matValues = subGraphs.foldLeft(Map.empty[Module, Any]) { (map, subGraph) => - val materializer = subMaterializers(subGraph.getClass) - map ++ materializer.materialize(subGraph, map) - } - graph.resolve(matValues) - } - - override def shutdown(): Unit = { - subMaterializers.values.foreach(_.shutdown()) - } -} - -object GearpumpMaterializer { - def apply(system: ActorSystem): GearpumpMaterializer = new GearpumpMaterializer(system) -} \ No newline at end of file diff --git a/experiments/akkastream/src/main/scala/akka/stream/gearpump/example/Test.scala b/experiments/akkastream/src/main/scala/akka/stream/gearpump/example/Test.scala deleted file mode 100644 index 7808b52b3..000000000 --- a/experiments/akkastream/src/main/scala/akka/stream/gearpump/example/Test.scala +++ /dev/null @@ -1,69 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package akka.stream.gearpump.example - -import scala.concurrent.Await -import scala.concurrent.duration.Duration - -import akka.actor.{Actor, ActorSystem, Props} -import akka.stream.gearpump.GearpumpMaterializer -import akka.stream.gearpump.graph.GraphCutter -import akka.stream.scaladsl.{Sink, Source} - -/** - * This tests how the [[GearpumpMaterializer]] materializes different partials of Graph - * to different runtime. - * - * In this test, source module and sink module are materialized locally, - * Other transformation module are materialized remotely in Gearpump - * streaming Application. - * - * Usage: output/target/pack/bin/gear app -jar experiments/akkastream/target/scala.11/akkastream-2.11.5-0.6.2-SNAPSHOT-assembly.jar - * - * - */ -object Test { - - def main(args: Array[String]): Unit = { - - println("running Test...") - - implicit val system = ActorSystem("akka-test") - implicit val materializer = new GearpumpMaterializer(system, GraphCutter.AllRemoteStrategy) - - val echo = system.actorOf(Props(new Echo())) - val sink = Sink.actorRef(echo, "COMPLETE") - val source = Source(List("red hat", "yellow sweater", "blue jack", "red apple", "green plant", - "blue sky")) - source.filter(_.startsWith("red")).fold("Items:") { (a, b) => - a + "|" + b - }.map("I want to order item: " + _).runWith(sink) - - Await.result(system.whenTerminated, Duration.Inf) - } - - class Echo extends Actor { - def receive: Receive = { - case any: AnyRef => - // scalastyle:off println - println("Confirm received: " + any) - // scalastyle:on println - } - } -} \ No newline at end of file diff --git a/experiments/akkastream/src/main/scala/akka/stream/gearpump/example/Test6.scala b/experiments/akkastream/src/main/scala/akka/stream/gearpump/example/Test6.scala deleted file mode 100644 index 0fccd30e4..000000000 --- a/experiments/akkastream/src/main/scala/akka/stream/gearpump/example/Test6.scala +++ /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 akka.stream.gearpump.example - -import scala.concurrent.Await -import scala.concurrent.duration.Duration - -import akka.actor.{Actor, ActorSystem, Props} -import akka.stream.gearpump.GearpumpMaterializer -import akka.stream.gearpump.scaladsl.GearSource -import akka.stream.scaladsl.Sink - -import org.apache.gearpump.streaming.dsl.CollectionDataSource - -/** - * WordCount example - * Test GroupBy - */ - -import akka.stream.gearpump.scaladsl.Implicits._ - -object Test6 { - - def main(args: Array[String]): Unit = { - - println("running Test...") - - implicit val system = ActorSystem("akka-test") - implicit val materializer = new GearpumpMaterializer(system) - - val echo = system.actorOf(Props(new Echo())) - val sink = Sink.actorRef(echo, "COMPLETE") - val sourceData = new CollectionDataSource(List("this is a good start", "this is a good time", "time to start", "congratulations", "green plant", "blue sky")) - val source = GearSource.from[String](sourceData) - source.mapConcat { line => - line.split(" ").toList - }.groupBy2(x => x).map(word => (word, 1)) - .reduce({ (a, b) => - (a._1, a._2 + b._2) - }).log("word-count").runWith(sink) - - Await.result(system.whenTerminated, Duration.Inf) - } - - class Echo extends Actor { - def receive: Receive = { - case any: AnyRef => - println("Confirm received: " + any) - } - } -} diff --git a/experiments/akkastream/src/main/scala/akka/stream/gearpump/materializer/LocalMaterializer.scala b/experiments/akkastream/src/main/scala/akka/stream/gearpump/materializer/LocalMaterializer.scala deleted file mode 100644 index a5c6e48da..000000000 --- a/experiments/akkastream/src/main/scala/akka/stream/gearpump/materializer/LocalMaterializer.scala +++ /dev/null @@ -1,152 +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 akka.stream.gearpump.materializer - -import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong} -import scala.concurrent.{Await, ExecutionContextExecutor} - -import akka.actor.{ActorCell, ActorRef, ActorSystem, Deploy, LocalActorRef, PoisonPill, Props, RepointableActorRef} -import akka.dispatch.Dispatchers -import akka.pattern.ask -import akka.stream.ModuleGraph.Edge -import akka.stream.impl.StreamLayout.Module -import akka.stream.impl.StreamSupervisor -import akka.stream.{ActorAttributes, ActorMaterializer, ActorMaterializerSettings, Attributes, ClosedShape, Graph => AkkaGraph, MaterializationContext, ModuleGraph} - -import org.apache.gearpump.util.Graph - -/** - * [[LocalMaterializer]] will use local actor to materialize the graph - * Use LocalMaterializer.apply to construct the LocalMaterializer. - * - * It is functional equivalent to [[akka.stream.impl.ActorMaterializerImpl]] - * - * - * @param system - * @param settings - * @param dispatchers - * @param supervisor - * @param haveShutDown - * @param flowNameCounter - * @param namePrefix - * @param optimizations - */ -abstract class LocalMaterializer( - val system: ActorSystem, - override val settings: ActorMaterializerSettings, - dispatchers: Dispatchers, - val supervisor: ActorRef, - val haveShutDown: AtomicBoolean, - flowNameCounter: AtomicLong, - namePrefix: String, - optimizations: Optimizations) extends ActorMaterializer { - - override def effectiveSettings(opAttr: Attributes): ActorMaterializerSettings = { - import ActorAttributes._ - import Attributes._ - opAttr.attributeList.foldLeft(settings) { (s, attr) => - attr match { - case InputBuffer(initial, max) => s.withInputBuffer(initial, max) - case Dispatcher(dispatcher) => s.withDispatcher(dispatcher) - case SupervisionStrategy(decider) => s.withSupervisionStrategy(decider) - case l: LogLevels => s - case Name(_) => s - case other => s - } - } - } - - override def shutdown(): Unit = - if (haveShutDown.compareAndSet(false, true)) supervisor ! PoisonPill - - override def isShutdown: Boolean = haveShutDown.get() - - private[akka] def actorOf(props: Props, name: String, dispatcher: String): ActorRef = { - supervisor match { - case ref: LocalActorRef => - ref.underlying.attachChild(props.withDispatcher(dispatcher), name, systemService = false) - case ref: RepointableActorRef => - if (ref.isStarted) { - ref.underlying.asInstanceOf[ActorCell].attachChild(props.withDispatcher(dispatcher), - name, systemService = false) - } else { - implicit val timeout = ref.system.settings.CreationTimeout - val f = (supervisor ? StreamSupervisor.Materialize(props.withDispatcher(dispatcher), - name)).mapTo[ActorRef] - Await.result(f, timeout.duration) - } - case unknown => - throw new IllegalStateException( - s"Stream supervisor must be a local actor, was [${unknown.getClass.getName}]") - } - } - - override lazy val executionContext: ExecutionContextExecutor = - dispatchers.lookup(settings.dispatcher match { - case Deploy.NoDispatcherGiven => Dispatchers.DefaultDispatcherId - case other => other - }) - - def materialize(graph: Graph[Module, Edge], inputMatValues: Map[Module, Any]): Map[Module, Any] - - override def materialize[Mat](runnableGraph: AkkaGraph[ClosedShape, Mat]): Mat = { - val graph = ModuleGraph(runnableGraph) - val matValues = materialize(graph.graph, Map.empty[Module, Any]) - graph.resolve(matValues) - } - - override def actorOf(context: MaterializationContext, props: Props): ActorRef = { - val dispatcher = - if (props.deploy.dispatcher == Deploy.NoDispatcherGiven) { - effectiveSettings(context.effectiveAttributes).dispatcher - } else { - props.dispatcher - } - actorOf(props, context.stageName, dispatcher) - } -} - -object LocalMaterializer { - - def apply(materializerSettings: Option[ActorMaterializerSettings] = None, - namePrefix: Option[String] = None, - optimizations: Optimizations = Optimizations.none)(implicit system: ActorSystem) - : LocalMaterializerImpl = { - - val settings = materializerSettings getOrElse ActorMaterializerSettings(system) - apply(settings, namePrefix.getOrElse("flow"), optimizations)(system) - } - - def apply(materializerSettings: ActorMaterializerSettings, - namePrefix: String, optimizations: Optimizations)(implicit system: ActorSystem) - : LocalMaterializerImpl = { - val haveShutDown = new AtomicBoolean(false) - - new LocalMaterializerImpl( - system, - materializerSettings, - system.dispatchers, - system.actorOf(StreamSupervisor.props(materializerSettings, - haveShutDown).withDispatcher(materializerSettings.dispatcher)), - haveShutDown, - FlowNameCounter(system).counter, - namePrefix, - optimizations) - } -} diff --git a/experiments/akkastream/src/main/scala/akka/stream/gearpump/materializer/LocalMaterializerImpl.scala b/experiments/akkastream/src/main/scala/akka/stream/gearpump/materializer/LocalMaterializerImpl.scala deleted file mode 100644 index 1ec724e8e..000000000 --- a/experiments/akkastream/src/main/scala/akka/stream/gearpump/materializer/LocalMaterializerImpl.scala +++ /dev/null @@ -1,284 +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 akka.stream.gearpump.materializer - -import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong} - -import akka.actor.{ActorRef, ActorSystem} -import akka.dispatch.Dispatchers -import akka.stream.ModuleGraph.{Edge, MaterializedValueSourceAttribute} -import akka.stream.actor.ActorSubscriber -import akka.stream.gearpump.materializer.LocalMaterializerImpl.MaterializedModule -import akka.stream.gearpump.module.ReduceModule -import akka.stream.gearpump.util.MaterializedValueOps -import akka.stream.impl.Stages.{DirectProcessor, Fold, StageModule} -import akka.stream.impl.StreamLayout.Module -import akka.stream.impl.{ActorProcessorFactory, ActorPublisher, ExposedPublisher, FanIn, FanOut, SinkModule, SourceModule, VirtualProcessor} -import akka.stream.{ActorMaterializerSettings, Attributes, Graph => AkkaGraph, InPort, MaterializationContext, Materializer, OutPort, Shape} -import org.reactivestreams.{Processor, Publisher, Subscriber} - -import org.apache.gearpump.util.Graph - -/** - * This materializer is functional equivalent to [[akka.stream.impl.ActorMaterializerImpl]] - * - * @param system - * @param settings - * @param dispatchers - * @param supervisor - * @param haveShutDown - * @param flowNameCounter - * @param namePrefix - * @param optimizations - */ -class LocalMaterializerImpl ( - system: ActorSystem, - settings: ActorMaterializerSettings, - dispatchers: Dispatchers, - supervisor: ActorRef, - haveShutDown: AtomicBoolean, - flowNameCounter: AtomicLong, - namePrefix: String, - optimizations: Optimizations) - extends LocalMaterializer( - system, settings, dispatchers, supervisor, - haveShutDown, flowNameCounter, namePrefix, optimizations) { - - override def materialize(graph: Graph[Module, Edge], inputMatValues: Map[Module, Any]): Map[Module, Any] = { - val materializedGraph = graph.mapVertex { module => - materializeAtomic(module) - } - - materializedGraph.edges.foreach { nodeEdgeNode => - val (node1, edge, node2) = nodeEdgeNode - val from = edge.from - val to = edge.to - val publisher = node1.outputs(from).asInstanceOf[Publisher[Any]] - val subscriber = node2.inputs(to).asInstanceOf[Subscriber[Any]] - publisher.subscribe(subscriber) - } - - val matValues = inputMatValues ++ materializedGraph.vertices.map { vertex => - (vertex.module, vertex.matValue) - }.toMap - - val matValueSources = materializedGraph.vertices.filter(_.module.isInstanceOf[MaterializedValueSource[_]]) - publishToMaterializedValueSource(matValueSources, matValues) - - matValues - } - - private def publishToMaterializedValueSource(modules: List[MaterializedModule], matValues: Map[Module, Any]) = { - modules.foreach { module => - val source = module.module.asInstanceOf[MaterializedValueSource[_]] - val attr = source.attributes.getAttribute(classOf[MaterializedValueSourceAttribute], null) - - Option(attr).map { attr => - val valueToPublish = MaterializedValueOps(attr.mat).resolve[Any](matValues) - module.outputs.foreach { portAndPublisher => - val (port, publisher) = portAndPublisher - publisher match { - case valuePublisher: MaterializedValuePublisher => - valuePublisher.setValue(valueToPublish) - } - } - } - } - } - - private[this] def nextFlowNameCount(): Long = flowNameCounter.incrementAndGet() - - private[this] def createFlowName(): String = s"$namePrefix-${nextFlowNameCount()}" - - val flowName = createFlowName() - var nextId = 0 - - def stageName(attr: Attributes): String = { - val name = s"$flowName-$nextId-${attr.nameOrDefault()}" - nextId += 1 - name - } - - private def materializeAtomic(atomic: Module): MaterializedModule = { - val effectiveAttributes = atomic.attributes - - def newMaterializationContext() = - new MaterializationContext(LocalMaterializerImpl.this, effectiveAttributes, stageName(effectiveAttributes)) - - atomic match { - case matValue: MaterializedValueSource[_] => - val pub = new MaterializedValuePublisher - val outputs = Map[OutPort, Publisher[_]](matValue.shape.outlet -> pub) - MaterializedModule(matValue, (), outputs = outputs) - case sink: SinkModule[_, _] => - val (sub, mat) = sink.create(newMaterializationContext()) - val inputs = Map[InPort, Subscriber[_]](sink.shape.inlet -> sub) - MaterializedModule(sink, mat, inputs) - case source: SourceModule[_, _] => - val (pub, mat) = source.create(newMaterializationContext()) - val outputs = Map[OutPort, Publisher[_]](source.shape.outlet -> pub) - MaterializedModule(source, mat, outputs = outputs) - - case reduce: ReduceModule[Any] => - //TODO: remove this after the official akka-stream API support the Reduce Module - val stage = LocalMaterializerImpl.toFoldModule(reduce) - val (processor, mat) = processorFor(stage, effectiveAttributes, effectiveSettings(effectiveAttributes)) - val inputs = Map[InPort, Subscriber[_]](stage.inPort -> processor) - val outputs = Map[OutPort, Publisher[_]](stage.outPort -> processor) - MaterializedModule(stage, mat, inputs, outputs) - - case stage: StageModule => - val (processor, mat) = processorFor(stage, effectiveAttributes, effectiveSettings(effectiveAttributes)) - val inputs = Map[InPort, Subscriber[_]](stage.inPort -> processor) - val outputs = Map[OutPort, Publisher[_]](stage.outPort -> processor) - MaterializedModule(stage, mat, inputs, outputs) - case tls: TlsModule => // TODO solve this so TlsModule doesn't need special treatment here - val es = effectiveSettings(effectiveAttributes) - val props = - SslTlsCipherActor.props(es, tls.sslContext, tls.firstSession, tracing = false, tls.role, tls.closing) - val impl = actorOf(props, stageName(effectiveAttributes), es.dispatcher) - def factory(id: Int) = new ActorPublisher[Any](impl) { - override val wakeUpMsg = FanOut.SubstreamSubscribePending(id) - } - val publishers = Vector.tabulate(2)(factory) - impl ! FanOut.ExposedPublishers(publishers) - - val inputs = Map[InPort, Subscriber[_]]( - tls.plainIn -> FanIn.SubInput[Any](impl, SslTlsCipherActor.UserIn), - tls.cipherIn -> FanIn.SubInput[Any](impl, SslTlsCipherActor.TransportIn)) - - val outputs = Map[OutPort, Publisher[_]]( - tls.plainOut -> publishers(SslTlsCipherActor.UserOut), - tls.cipherOut -> publishers(SslTlsCipherActor.TransportOut)) - MaterializedModule(tls, (), inputs, outputs) - - case junction: JunctionModule => - materializeJunction(junction, effectiveAttributes, effectiveSettings(effectiveAttributes)) - } - } - - private def processorFor(op: StageModule, - effectiveAttributes: Attributes, - effectiveSettings: ActorMaterializerSettings): (Processor[Any, Any], Any) = op match { - case DirectProcessor(processorFactory, _) => processorFactory() - case Identity(attr) => (new VirtualProcessor, ()) - case _ => - val (opprops, mat) = ActorProcessorFactory.props(LocalMaterializerImpl.this, op, effectiveAttributes) - ActorProcessorFactory[Any, Any]( - actorOf(opprops, stageName(effectiveAttributes), effectiveSettings.dispatcher)) -> mat - } - - private def materializeJunction( - op: JunctionModule, - effectiveAttributes: Attributes, - effectiveSettings: ActorMaterializerSettings): MaterializedModule = { - op match { - case fanin: FanInModule => - val (props, inputs, output) = fanin match { - - case MergeModule(shape, _) => - (FairMerge.props(effectiveSettings, shape.inSeq.size), shape.inSeq, shape.out) - - case f: FlexiMergeModule[_, Shape] => - val flexi = f.flexi(f.shape) - val shape: Shape = f.shape - (FlexiMerge.props(effectiveSettings, f.shape, flexi), shape.inlets, shape.outlets.head) - - case MergePreferredModule(shape, _) => - (UnfairMerge.props(effectiveSettings, shape.inlets.size), shape.preferred +: shape.inSeq, shape.out) - - case ConcatModule(shape, _) => - require(shape.inSeq.size == 2, "currently only supporting concatenation of exactly two inputs") // TODO - (Concat.props(effectiveSettings), shape.inSeq, shape.out) - - case zip: ZipWithModule => - (zip.props(effectiveSettings), zip.shape.inlets, zip.outPorts.head) - } - - val impl = actorOf(props, stageName(effectiveAttributes), effectiveSettings.dispatcher) - val publisher = new ActorPublisher[Any](impl) - // Resolve cyclic dependency with actor. This MUST be the first message no matter what. - impl ! ExposedPublisher(publisher) - - val inputMapping: Map[InPort, Subscriber[_]] = inputs.zipWithIndex.map { pair => - val (in, id) = pair - (in, FanIn.SubInput[Any](impl, id)) - }.toMap - - val outMapping = Map(output -> publisher) - MaterializedModule(fanin, (), inputMapping, outMapping) - - case fanout: FanOutModule => - val (props, in, outs) = fanout match { - - case r: FlexiRouteModule[t, Shape] => - val flexi = r.flexi(r.shape) - val shape: Shape = r.shape - (FlexiRoute.props(effectiveSettings, r.shape, flexi), shape.inlets.head: InPort, r.shape.outlets) - - case BroadcastModule(shape, eagerCancel, _) => - (Broadcast.props(effectiveSettings, eagerCancel, shape.outArray.size), shape.in, shape.outArray.toSeq) - - case BalanceModule(shape, waitForDownstreams, _) => - (Balance.props(effectiveSettings, shape.outArray.size, waitForDownstreams), shape.in, shape.outArray.toSeq) - - case unzip: UnzipWithModule => - (unzip.props(effectiveSettings), unzip.inPorts.head, unzip.shape.outlets) - } - val impl = actorOf(props, stageName(effectiveAttributes), effectiveSettings.dispatcher) - val size = outs.size - def factory(id: Int) = - new ActorPublisher[Any](impl) { - override val wakeUpMsg = FanOut.SubstreamSubscribePending(id) - } - val publishers = - if (outs.size < 8) Vector.tabulate(size)(factory) - else List.tabulate(size)(factory) - - impl ! FanOut.ExposedPublishers(publishers) - val outputs: Map[OutPort, Publisher[_]] = publishers.iterator.zip(outs.iterator).map { case (pub, out) => - (out, pub) - }.toMap - - val inputs: Map[InPort, Subscriber[_]] = Map(in -> ActorSubscriber[Any](impl)) - MaterializedModule(fanout, (), inputs, outputs) - } - } - - override def withNamePrefix(name: String): Materializer = { - new LocalMaterializerImpl(system, settings, dispatchers, supervisor, - haveShutDown, flowNameCounter, namePrefix = name, optimizations) - } -} - -object LocalMaterializerImpl { - case class MaterializedModule(val module: Module, val matValue: Any, inputs: Map[InPort, Subscriber[_]] = Map.empty[InPort, Subscriber[_]], outputs: Map[OutPort, Publisher[_]] = Map.empty[OutPort, Publisher[_]]) - - def toFoldModule(reduce: ReduceModule[Any]): Fold = { - val f = reduce.f - val aggregator = { (zero: Any, input: Any) => - if (zero == null) { - input - } else { - f(zero, input) - } - } - new Fold(null, aggregator) - } -} \ No newline at end of file diff --git a/experiments/akkastream/src/main/scala/akka/stream/gearpump/materializer/RemoteMaterializerImpl.scala b/experiments/akkastream/src/main/scala/akka/stream/gearpump/materializer/RemoteMaterializerImpl.scala deleted file mode 100644 index 47ed1f210..000000000 --- a/experiments/akkastream/src/main/scala/akka/stream/gearpump/materializer/RemoteMaterializerImpl.scala +++ /dev/null @@ -1,453 +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 akka.stream.gearpump.materializer - -import akka.actor.ActorSystem -import akka.stream.ModuleGraph.Edge -import akka.stream.gearpump.GearAttributes -import akka.stream.gearpump.module.{GroupByModule, ProcessorModule, ReduceModule, SinkBridgeModule, SinkTaskModule, SourceBridgeModule, SourceTaskModule} -import akka.stream.gearpump.task.{BalanceTask, BroadcastTask, GraphTask, SinkBridgeTask, SourceBridgeTask, UnZip2Task} -import akka.stream.impl.Stages -import akka.stream.impl.Stages.StageModule -import akka.stream.impl.StreamLayout.Module -import org.slf4j.LoggerFactory - -import org.apache.gearpump.cluster.UserConfig -import org.apache.gearpump.streaming.dsl.StreamApp -import org.apache.gearpump.streaming.dsl.op.{DataSinkOp, DataSourceOp, Direct, FlatMapOp, GroupByOp, MasterOp, MergeOp, Op, OpEdge, ProcessorOp, Shuffle, SlaveOp} -import org.apache.gearpump.streaming.{ProcessorId, StreamApplication} -import org.apache.gearpump.util.Graph - -/** - * [[RemoteMaterializerImpl]] will materialize the [[Graph[Module, Edge]] to a Gearpump - * Streaming Application. - * - * @param graph - * @param system - */ -class RemoteMaterializerImpl(graph: Graph[Module, Edge], system: ActorSystem) { - - import RemoteMaterializerImpl._ - - type Clue = String - private implicit val actorSystem = system - - private def uuid: String = { - java.util.UUID.randomUUID.toString - } - - /** - * @return a mapping from Module to Materialized Processor Id. - */ - def materialize: (StreamApplication, Map[Module, ProcessorId]) = { - val (opGraph, clues) = toOpGraph() - val app: StreamApplication = new StreamApp("app", system, UserConfig.empty, opGraph) - val processorIds = resolveClues(app, clues) - - val updatedApp = updateJunctionConfig(processorIds, app) - (cleanClues(updatedApp), processorIds) - } - - private def updateJunctionConfig(processorIds: Map[Module, ProcessorId], app: StreamApplication): StreamApplication = { - val config = junctionConfig(processorIds) - - val dag = app.dag.mapVertex { vertex => - val processorId = vertex.id - val newConf = vertex.taskConf.withConfig(config(processorId)) - vertex.copy(taskConf = newConf) - } - new StreamApplication(app.name, app.inputUserConfig, dag) - } - - /** - * Update junction config so that each GraphTask know its upstream and downstream. - * @param processorIds - * @return - */ - private def junctionConfig(processorIds: Map[Module, ProcessorId]): Map[ProcessorId, UserConfig] = { - val updatedConfigs = graph.vertices.map { vertex => - val processorId = processorIds(vertex) - vertex match { - case junction: JunctionModule => - val inProcessors = junction.shape.inlets.map { inlet => - val upstreamModule = graph.incomingEdgesOf(junction).find(_._2.to == inlet).map(_._1) - val upstreamProcessorId = processorIds(upstreamModule.get) - upstreamProcessorId - }.toList - - val outProcessors = junction.shape.outlets.map { outlet => - val downstreamModule = graph.outgoingEdgesOf(junction).find(_._2.from == outlet).map(_._3) - val downstreamProcessorId = downstreamModule.map(processorIds(_)) - downstreamProcessorId.get - }.toList - - (processorId, UserConfig.empty.withValue(GraphTask.OUT_PROCESSORS, outProcessors) - .withValue(GraphTask.IN_PROCESSORS, inProcessors)) - case _ => - (processorId, UserConfig.empty) - } - }.toMap - updatedConfigs - } - - private def resolveClues(app: StreamApplication, clues: Map[Module, Clue]): Map[Module, ProcessorId] = { - clues.flatMap { kv => - val (module, clue) = kv - val processorId = app.dag.vertices.find { processor => - processor.taskConf.getString(clue).isDefined - }.map(_.id) - processorId.map((module, _)) - } - } - - private def cleanClues(app: StreamApplication): StreamApplication = { - val graph = app.dag.mapVertex { processor => - val conf = cleanClue(processor.taskConf) - processor.copy(taskConf = conf) - } - new StreamApplication(app.name, app.inputUserConfig, graph) - } - - private def cleanClue(conf: UserConfig): UserConfig = { - conf.filter { kv => - kv._2 != RemoteMaterializerImpl.STAINS - } - } - - private def toOpGraph(): (Graph[Op, OpEdge], Map[Module, Clue]) = { - var matValues = Map.empty[Module, Clue] - val opGraph = graph.mapVertex{ module => - val name = uuid - val conf = UserConfig.empty.withString(name, RemoteMaterializerImpl.STAINS) - matValues += module -> name - val parallelism = GearAttributes.count(module.attributes) - val op = module match { - case source: SourceTaskModule[t] => - val updatedConf = conf.withConfig(source.conf) - new DataSourceOp[t](source.source, parallelism, updatedConf, "source") - case sink: SinkTaskModule[t] => - val updatedConf = conf.withConfig(sink.conf) - new DataSinkOp[t](sink.sink, parallelism, updatedConf, "sink") - case sourceBridge: SourceBridgeModule[_, _] => - new ProcessorOp(classOf[SourceBridgeTask], parallelism = 1, conf, "source") - case processor: ProcessorModule[_, _, _] => - val updatedConf = conf.withConfig(processor.conf) - new ProcessorOp(processor.processor, parallelism, updatedConf, "source") - case sinkBridge: SinkBridgeModule[_, _] => - new ProcessorOp(classOf[SinkBridgeTask], parallelism, conf, "sink") - case groupBy: GroupByModule[t, g] => - new GroupByOp[t, g](groupBy.groupBy, parallelism, "groupBy", conf) - case reduce: ReduceModule[Any] => - reduceOp(reduce.f, conf) - case stage: StageModule => - translateStage(stage, conf) - case fanIn: FanInModule => - translateFanIn(fanIn, graph.incomingEdgesOf(fanIn), parallelism, conf) - case fanOut: FanOutModule => - translateFanOut(fanOut, graph.outgoingEdgesOf(fanOut), parallelism, conf) - } - - if (op == null) { - throw new UnsupportedOperationException(module.getClass.toString + " is not supported with RemoteMaterializer") - } - op - }.mapEdge[OpEdge]{(n1, edge, n2) => - n2 match { - case master: MasterOp => - Shuffle - case slave: SlaveOp[_] if n1.isInstanceOf[ProcessorOp[_]] => - Shuffle - case slave: SlaveOp[_] => - Direct - } - } - (opGraph, matValues) - } - - private def translateStage(module: StageModule, conf: UserConfig): Op = { - module match { - case buffer: Stages.Buffer => - //ignore the buffering operation - identity("buffer", conf) - case collect: Stages.Collect => - collectOp(collect.pf, conf) - case concatAll: Stages.ConcatAll => - //TODO: - null - case conflat: Stages.Conflate => - conflatOp(conflat.seed, conflat.aggregate, conf) - case drop: Stages.Drop => - dropOp(drop.n, conf) - case dropWhile: Stages.DropWhile => - dropWhileOp(dropWhile.p, conf) - case expand: Stages.Expand => - //TODO - null - case filter: Stages.Filter => - filterOp(filter.p, conf) - case fold: Stages.Fold => - foldOp(fold.zero, fold.f, conf) - case groupBy: Stages.GroupBy => - //TODO - null - case grouped: Stages.Grouped => - groupedOp(grouped.n, conf) - case _: Stages.Identity => - identity("identity", conf) - case log: Stages.Log => - logOp(log.name, log.extract, conf) - case map: Stages.Map => - mapOp(map.f, conf) - case mapAsync: Stages.MapAsync => - //TODO - null - case mapAsync: Stages.MapAsyncUnordered => - //TODO - null - case flatMap: Stages.MapConcat => - flatMapOp(flatMap.f, "mapConcat", conf) - case stage: MaterializingStageFactory => - //TODO - null - case prefixAndTail: Stages.PrefixAndTail => - //TODO - null - case recover: Stages.Recover => - //TODO: we will just ignore this - identity("recover", conf) - case scan: Stages.Scan => - scanOp(scan.zero, scan.f, conf) - case split: Stages.Split => - //TODO - null - case stage: Stages.StageFactory => - //TODO - null - case take: Stages.Take => - takeOp(take.n, conf) - case takeWhile: Stages.TakeWhile => - filterOp(takeWhile.p, conf) - case time: Stages.TimerTransform => - //TODO - null - } - } - - private def translateFanIn( - fanIn: FanInModule, - edges: List[(Module, Edge, Module)], - parallelism: Int, - conf: UserConfig): Op = { - fanIn match { - case merge: MergeModule[_] => - MergeOp("merge", conf) - case mergePrefered: MergePreferredModule[_] => - //TODO, support "prefer" merge - MergeOp("mergePrefered", conf) - case zip: ZipWithModule => - //TODO: support zip module - null - case concat: ConcatModule[_] => - //TODO: support concat module - null - case flexiMerge: FlexiMergeModule[_, _] => - //TODO: Suport flexi merge module - null - } - } - - private def translateFanOut( - fanOut: FanOutModule, - edges: List[(Module, Edge, Module)], - parallelism: Int, - conf: UserConfig): Op = { - fanOut match { - case unzip2: UnzipWith2Module[Any, Any, Any] => - val updatedConf = conf.withValue(UnZip2Task.UNZIP2_FUNCTION, new UnZip2Task.UnZipFunction(unzip2.f)) - new ProcessorOp(classOf[UnZip2Task], parallelism, updatedConf, "unzip") - case broadcast: BroadcastModule[_] => - new ProcessorOp(classOf[BroadcastTask], parallelism, conf, "broadcast") - case broadcast: BalanceModule[_] => - new ProcessorOp(classOf[BalanceTask], parallelism, conf, "balance") - case flexi: FlexiRouteImpl[_, _] => - //TODO - null - } - } -} - -object RemoteMaterializerImpl { - final val NotApplied: Any => Any = _ => NotApplied - - def collectOp(collect: PartialFunction[Any, Any], conf: UserConfig): Op = { - flatMapOp({ data => - collect.applyOrElse(data, NotApplied) match { - case NotApplied => None - case result: Any => Option(result) - } - }, "collect", conf) - } - - def filterOp(filter: Any => Boolean, conf: UserConfig): Op = { - flatMapOp({ data => - if (filter(data)) Option(data) else None - }, "filter", conf) - } - - def reduceOp(reduce: (Any, Any) => Any, conf: UserConfig): Op = { - var result: Any = null - val flatMap = { elem: Any => - if (result == null) { - result = elem - } else { - result = reduce(result, elem) - } - List(result) - } - flatMapOp(flatMap, "reduce", conf) - } - - def identity(description: String, conf: UserConfig): Op = { - flatMapOp({ data => - List(data) - }, description, conf) - } - - def mapOp(map: Any => Any, conf: UserConfig): Op = { - flatMapOp({ data: Any => - List(map(data)) - }, "map", conf) - } - - def flatMapOp(flatMap: Any => Iterable[Any], conf: UserConfig): Op = { - flatMapOp(flatMap, "flatmap", conf) - } - - def flatMapOp(fun: Any => TraversableOnce[Any], description: String, conf: UserConfig): Op = { - FlatMapOp(fun, description, conf) - } - - def conflatOp(seed: Any => Any, aggregate: (Any, Any) => Any, conf: UserConfig): Op = { - var agg: Any = null - val flatMap = { elem: Any => - agg = if (agg == null) { - seed(elem) - } else { - aggregate(agg, elem) - } - List(agg) - } - - flatMapOp(flatMap, "map", conf) - } - - def foldOp(zero: Any, fold: (Any, Any) => Any, conf: UserConfig): Op = { - var aggregator: Any = zero - val map = { elem: Any => - aggregator = fold(aggregator, elem) - List(aggregator) - } - flatMapOp(map, "fold", conf) - } - - def groupedOp(count: Int, conf: UserConfig): Op = { - var left = count - val buf = { - val b = Vector.newBuilder[Any] - b.sizeHint(count) - b - } - - val flatMap: Any => Iterable[Any] = { input: Any => - buf += input - left -= 1 - if (left == 0) { - val emit = buf.result() - buf.clear() - left = count - Some(emit) - } else { - None - } - } - flatMapOp(flatMap, conf: UserConfig) - } - - def dropOp(number: Long, conf: UserConfig): Op = { - var left = number - val flatMap: Any => Iterable[Any] = { input: Any => - if (left > 0) { - left -= 1 - None - } else { - Some(input) - } - } - flatMapOp(flatMap, "drop", conf) - } - - def dropWhileOp(drop: Any => Boolean, conf: UserConfig): Op = { - flatMapOp({ data => - if (drop(data)) None else Option(data) - }, "dropWhile", conf) - } - - def logOp(name: String, extract: Any => Any, conf: UserConfig): Op = { - val flatMap = { elem: Any => - LoggerFactory.getLogger(name).info(s"Element: {${extract(elem)}}") - List(elem) - } - flatMapOp(flatMap, "log", conf) - } - - def scanOp(zero: Any, f: (Any, Any) => Any, conf: UserConfig): Op = { - var aggregator = zero - var pushedZero = false - - val flatMap = { elem: Any => - aggregator = f(aggregator, elem) - - if (pushedZero) { - pushedZero = true - List(zero, aggregator) - } else { - List(aggregator) - } - } - flatMapOp(flatMap, "scan", conf) - } - - def takeOp(count: Long, conf: UserConfig): Op = { - var left: Long = count - - val filter: Any => Iterable[Any] = { elem: Any => - left -= 1 - if (left > 0) Some(elem) - else if (left == 0) Some(elem) - else None - } - flatMapOp(filter, "take", conf) - } - - /** - * We use stains to track how module maps to Processor - * - */ - val STAINS = "track how module is fused to processor" -} \ No newline at end of file diff --git a/experiments/akkastream/src/main/scala/akka/stream/gearpump/GearAttributes.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearAttributes.scala similarity index 92% rename from experiments/akkastream/src/main/scala/akka/stream/gearpump/GearAttributes.scala rename to experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearAttributes.scala index 50c44501c..016a7b279 100644 --- a/experiments/akkastream/src/main/scala/akka/stream/gearpump/GearAttributes.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearAttributes.scala @@ -7,7 +7,7 @@ * "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 + * 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, @@ -16,16 +16,15 @@ * limitations under the License. */ -package akka.stream.gearpump +package org.apache.gearpump.akkastream import akka.stream.Attributes import akka.stream.Attributes.Attribute object GearAttributes { - /** * Define how many parallel instance we want to use to run this module - * @param count + * @param count Int * @return */ def count(count: Int): Attributes = Attributes(ParallismAttribute(count)) @@ -46,7 +45,7 @@ object GearAttributes { * Get the effective location settings if child override the parent * setttings. * - * @param attrs + * @param attrs Attributes * @return */ def location(attrs: Attributes): Location = { @@ -60,7 +59,7 @@ object GearAttributes { /** * get effective parallelism settings if child override parent. - * @param attrs + * @param attrs Attributes * @return */ def count(attrs: Attributes): Int = { @@ -84,7 +83,7 @@ object GearAttributes { /** * How many parallel instance we want to use for this module. * - * @param parallelism + * @param parallelism Int */ final case class ParallismAttribute(parallelism: Int) extends Attribute } diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearpumpMaterializer.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearpumpMaterializer.scala new file mode 100644 index 000000000..75dc95a2c --- /dev/null +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearpumpMaterializer.scala @@ -0,0 +1,290 @@ +/* + * 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.gearpump.akkastream + +import java.util.concurrent.atomic.AtomicBoolean + +import akka.NotUsed +import akka.actor.{ActorContext, ActorRef, ActorRefFactory, ActorSystem, Cancellable, ExtendedActorSystem} +import akka.event.{Logging, LoggingAdapter} +import akka.stream.Attributes.Attribute +import akka.stream._ +import akka.stream.impl.Stages.SymbolicGraphStage +import akka.stream.impl.StreamLayout._ +import akka.stream.impl._ +import akka.stream.impl.fusing.{GraphInterpreterShell, GraphStageModule} +import akka.stream.scaladsl.ModuleExtractor +import akka.stream.stage.GraphStage +import org.apache.gearpump.akkastream.GearpumpMaterializer.Edge +import org.apache.gearpump.akkastream.graph.GraphPartitioner.Strategy +import org.apache.gearpump.akkastream.graph.LocalGraph.LocalGraphMaterializer +import org.apache.gearpump.akkastream.graph.RemoteGraph.RemoteGraphMaterializer +import org.apache.gearpump.akkastream.graph._ +import org.apache.gearpump.akkastream.util.MaterializedValueOps + +import scala.collection.mutable +import scala.concurrent.{ExecutionContextExecutor, Promise} +import scala.concurrent.duration.FiniteDuration + +object GearpumpMaterializer { + + final val Debug = true + + final case class Edge(from: OutPort, to: InPort) + + final case class MaterializedValueSourceAttribute(mat: MaterializedValueNode) extends Attribute + + implicit def boolToAtomic(bool: Boolean): AtomicBoolean = new AtomicBoolean(bool) + + def apply(strategy: Strategy)(implicit context: ActorRefFactory): + ExtendedActorMaterializer = { + val system = actorSystemOf(context) + + apply(ActorMaterializerSettings( + system).withAutoFusing(false), strategy, useLocalCluster = false, "flow")(context) + } + + def apply(materializerSettings: Option[ActorMaterializerSettings] = None, + strategy: Strategy = GraphPartitioner.AllRemoteStrategy, + useLocalCluster: Boolean = true, + namePrefix: Option[String] = None)(implicit context: ActorRefFactory): + ExtendedActorMaterializer = { + val system = actorSystemOf(context) + + val settings = materializerSettings getOrElse + ActorMaterializerSettings(system).withAutoFusing(false) + apply(settings, strategy, useLocalCluster, namePrefix.getOrElse("flow"))(context) + } + + def apply(materializerSettings: ActorMaterializerSettings, + strategy: Strategy, + useLocalCluster: Boolean, + namePrefix: String)(implicit context: ActorRefFactory): + ExtendedActorMaterializer = { + val system = actorSystemOf(context) + + new GearpumpMaterializer( + system, + materializerSettings, + context.actorOf( + StreamSupervisor.props(materializerSettings, false).withDispatcher( + materializerSettings.dispatcher), StreamSupervisor.nextName())) + } + + + private def actorSystemOf(context: ActorRefFactory): ActorSystem = { + val system = context match { + case s: ExtendedActorSystem => s + case c: ActorContext => c.system + case null => throw new IllegalArgumentException("ActorRefFactory context must be defined") + case _ => + throw new IllegalArgumentException( + s""" + | context must be a ActorSystem or ActorContext, got [${context.getClass.getName}] + """.stripMargin + ) + } + system + } + +} + +/** + * + * [[GearpumpMaterializer]] allows you to render akka-stream DSL as a Gearpump + * streaming application. If some module cannot be rendered remotely in Gearpump + * Cluster, then it will use local Actor materializer as fallback to materialize + * the module locally. + * + * User can customize a [[org.apache.gearpump.akkastream.graph.GraphPartitioner.Strategy]] + * to determine which module should be rendered + * remotely, and which module should be rendered locally. + * + * @see [[org.apache.gearpump.akkastream.graph.GraphPartitioner]] + * to find out how we cut the runnableGraph to two parts, + * and materialize them separately. + * @param system ActorSystem + * @param strategy Strategy + * @param useLocalCluster whether to use built-in in-process local cluster + */ +class GearpumpMaterializer(override val system: ActorSystem, + override val settings: ActorMaterializerSettings, + override val supervisor: ActorRef, + strategy: Strategy = GraphPartitioner.AllRemoteStrategy, + useLocalCluster: Boolean = true, namePrefix: Option[String] = None) + extends ExtendedActorMaterializer { + + private val subMaterializers: Map[Class[_], SubGraphMaterializer] = Map( + classOf[LocalGraph] -> new LocalGraphMaterializer(system), + classOf[RemoteGraph] -> new RemoteGraphMaterializer(useLocalCluster, system) + ) + + override def logger: LoggingAdapter = Logging.getLogger(system, this) + + override def isShutdown: Boolean = system.isTerminated + + override def effectiveSettings(opAttr: Attributes): ActorMaterializerSettings = { + import ActorAttributes._ + import Attributes._ + opAttr.attributeList.foldLeft(settings) { (s, attr) => + attr match { + case InputBuffer(initial, max) => s.withInputBuffer(initial, max) + case Dispatcher(dispatcher) => s.withDispatcher(dispatcher) + case SupervisionStrategy(decider) => s.withSupervisionStrategy(decider) + case _ => s + } + } + } + + override def withNamePrefix(name: String): ExtendedActorMaterializer = + throw new UnsupportedOperationException() + + override implicit def executionContext: ExecutionContextExecutor = + throw new UnsupportedOperationException() + + override def schedulePeriodically(initialDelay: FiniteDuration, + interval: FiniteDuration, + task: Runnable): Cancellable = + system.scheduler.schedule(initialDelay, interval, task)(executionContext) + + override def scheduleOnce(delay: FiniteDuration, task: Runnable): Cancellable = + system.scheduler.scheduleOnce(delay, task)(executionContext) + + override def materialize[Mat](runnableGraph: Graph[ClosedShape, Mat]): Mat = { + val initialAttributes = Attributes( + Attributes.InputBuffer( + settings.initialInputBufferSize, + settings.maxInputBufferSize + ) :: + ActorAttributes.Dispatcher(settings.dispatcher) :: + ActorAttributes.SupervisionStrategy(settings.supervisionDecider) :: + Nil) + + val info = Fusing.aggressive(runnableGraph).module.info + import _root_.org.apache.gearpump.util.{Graph => GGraph} + val graph = GGraph.empty[Module, Edge] + + info.allModules.foreach(module => { + if (module.isCopied) { + val original = module.asInstanceOf[CopiedModule].copyOf + graph.addVertex(original) + module.shape.outlets.zip(original.shape.outlets).foreach(out => { + val (cout, oout) = out + val cin = info.downstreams(cout) + val downStreamModule = info.inOwners(cin) + if(downStreamModule.isCopied) { + val downStreamOriginal = downStreamModule.asInstanceOf[CopiedModule].copyOf + downStreamModule.shape.inlets.zip(downStreamOriginal.shape.inlets).foreach(in => { + in._1 == cin match { + case true => + val oin = in._2 + graph.addEdge(original, Edge(oout, oin), downStreamOriginal) + case false => + } + }) + } + }) + } + }) + + if(Debug) { + val iterator = graph.topologicalOrderIterator + while (iterator.hasNext) { + val module = iterator.next() + // scalastyle:off println + module match { + case graphStageModule: GraphStageModule => + graphStageModule.stage match { + case symbolicGraphStage: SymbolicGraphStage[_, _, _] => + val symbolicName = symbolicGraphStage.symbolicStage.getClass.getSimpleName + println( + s"${module.getClass.getSimpleName}(${symbolicName})" + ) + case graphStage: GraphStage[_] => + val name = graphStage.getClass.getSimpleName + println( + s"${module.getClass.getSimpleName}(${name})" + ) + case other => + println( + s"${module.getClass.getSimpleName}(${other.getClass.getSimpleName})" + ) + } + case _ => + println(module.getClass.getSimpleName) + } + // scalastyle:on println + } + } + + val subGraphs = GraphPartitioner(strategy).partition(graph) + val matValues = subGraphs.foldLeft(mutable.Map.empty[Module, Any]) { (map, subGraph) => + val materializer = subMaterializers(subGraph.getClass) + map ++ materializer.materialize(subGraph, map) + } + val mat = matValues.flatMap(pair => { + val (module, any) = pair + any match { + case notUsed: NotUsed => + None + case others => + val rt = module.shape match { + case sink: SinkShape[_] => + Some(any) + case _ => + None + } + rt + } + }).toList + val matModule = subGraphs.last.graph.topologicalOrderIterator.toList.last + val mat2 = resolveMaterialized(matModule.materializedValueComputation, matValues) + val rt = Some(mat).flatMap(any => { + any match { + case promise: Promise[_] => + Some(promise.future) + case other => + Some(other) + } + }) + rt.getOrElse(null).asInstanceOf[Mat] + } + + override def materialize[Mat](runnableGraph: Graph[ClosedShape, Mat], + subflowFuser: GraphInterpreterShell => ActorRef): Mat = { + materialize(runnableGraph) + } + + def shutdown: Unit = { + subMaterializers.values.foreach(_.shutdown) + } + + private def resolveMaterialized(mat: MaterializedValueNode, + materializedValues: mutable.Map[Module, Any]): Any = mat match { + case Atomic(m) => + materializedValues.getOrElse(m, ()) + case Combine(f, d1, d2) => + f(resolveMaterialized(d1, materializedValues), resolveMaterialized(d2, materializedValues)) + case Transform(f, d) => + f(resolveMaterialized(d, materializedValues)) + case Ignore => + () + } +} + diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearpumpMaterializerSession.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearpumpMaterializerSession.scala new file mode 100644 index 000000000..871dcf817 --- /dev/null +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearpumpMaterializerSession.scala @@ -0,0 +1,152 @@ +/* + * 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.gearpump.akkastream + +import java.{util => ju} + +import _root_.org.apache.gearpump.util.{Graph => GGraph} +import akka.actor.ActorSystem +import akka.stream._ +import org.apache.gearpump.akkastream.GearpumpMaterializer.{Edge, MaterializedValueSourceAttribute} +import akka.stream.impl.StreamLayout._ +import akka.stream.impl._ +import akka.stream.impl.fusing.GraphStages.MaterializedValueSource + +class GearpumpMaterializerSession(system: ActorSystem, topLevel: Module, + initialAttributes: Attributes, namePrefix: Option[String] = None) + extends MaterializerSession(topLevel, initialAttributes) { + + private[this] def createFlowName(): String = + FlowNames(system).name.copy(namePrefix.getOrElse("flow")).next() + + private val flowName = createFlowName() + private var nextId = 0 + + private def stageName(attr: Attributes): String = { + val name = s"$flowName-$nextId-${attr.nameOrDefault()}" + nextId += 1 + name + } + + val graph = GGraph.empty[Module, Edge] + + def addEdge(publisher: (OutPort, Module), subscriber: (InPort, Module)): Unit = { + graph.addEdge(publisher._2, Edge(publisher._1, subscriber._1), subscriber._2) + } + + def addVertex(module: Module): Unit = { + graph.addVertex(module) + } + + override def materializeModule(module: Module, parentAttributes: Attributes): Any = { + + val materializedValues: ju.Map[Module, Any] = new ju.HashMap + val currentAttributes = mergeAttributes(parentAttributes, module.attributes) + + val materializedValueSources = List.empty[MaterializedValueSource[_]] + + for (submodule <- module.subModules) { + submodule match { + case atomic: AtomicModule => + materializeAtomic(atomic, currentAttributes, materializedValues) + case copied: CopiedModule => + enterScope(copied) + materializedValues.put(copied, materializeModule(copied, currentAttributes)) + exitScope(copied) + case composite => + materializedValues.put(composite, materializeComposite(composite, currentAttributes)) + case EmptyModule => + } + } + + val mat = resolveMaterialized(module.materializedValueComputation, materializedValues) + + materializedValueSources.foreach { module => + val matAttribute = + new MaterializedValueSourceAttribute(mat.asInstanceOf[MaterializedValueNode]) + val copied = copyAtomicModule(module.module, parentAttributes + and Attributes(matAttribute)) + // TODO + // assignPort(module.shape.out, (copied.shape.outlets.head, copied)) + addVertex(copied) + materializedValues.put(copied, Atomic(copied)) + } + mat + + } + + override protected def materializeComposite(composite: Module, + effectiveAttributes: Attributes): Any = { + materializeModule(composite, effectiveAttributes) + } + + protected def materializeAtomic(atomic: AtomicModule, + parentAttributes: Attributes, + matVal: ju.Map[Module, Any]): Unit = { + + val (inputs, outputs) = (atomic.shape.inlets, atomic.shape.outlets) + val copied = copyAtomicModule(atomic, parentAttributes) + + for ((in, id) <- inputs.zipWithIndex) { + val inPort = inPortMapping(atomic, copied)(in) + // assignPort(in, (inPort, copied)) + } + + for ((out, id) <- outputs.zipWithIndex) { + val outPort = outPortMapping(atomic, copied)(out) + // TODO + // assignPort(out, (outPort, copied)) + } + + addVertex(copied) + matVal.put(atomic, Atomic(copied)) + } + + private def copyAtomicModule[T <: Module](module: T, parentAttributes: Attributes): T = { + val currentAttributes = mergeAttributes(parentAttributes, module.attributes) + module.withAttributes(currentAttributes).asInstanceOf[T] + } + + private def outPortMapping(from: Module, to: Module): Map[OutPort, OutPort] = { + from.shape.outlets.iterator.zip(to.shape.outlets.iterator).toList.toMap + } + + private def inPortMapping(from: Module, to: Module): Map[InPort, InPort] = { + from.shape.inlets.iterator.zip(to.shape.inlets.iterator).toList.toMap + } + + protected def resolveMaterialized(matNode: MaterializedValueNode, + materializedValues: ju.Map[Module, Any]): + Any = + matNode match { + case Atomic(m) => materializedValues.get(m) + case Combine(f, d1, d2) => f(resolveMaterialized(d1, materializedValues), + resolveMaterialized(d2, materializedValues)) + case Transform(f, d) => f(resolveMaterialized(d, materializedValues)) + case Ignore => Ignore + } +} + +object GearpumpMaterializerSession { + def apply(system: ActorSystem, topLevel: Module, + initialAttributes: Attributes, namePrefix: Option[String] = None): + GearpumpMaterializerSession = { + new GearpumpMaterializerSession(system, topLevel, initialAttributes, namePrefix) + } +} diff --git a/experiments/akkastream/src/main/scala/akka/stream/gearpump/example/Test3.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test.scala similarity index 51% rename from experiments/akkastream/src/main/scala/akka/stream/gearpump/example/Test3.scala rename to experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test.scala index 976b1e685..2ce4e1905 100644 --- a/experiments/akkastream/src/main/scala/akka/stream/gearpump/example/Test3.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test.scala @@ -7,7 +7,7 @@ * "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 + * 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, @@ -16,38 +16,39 @@ * limitations under the License. */ -package akka.stream.gearpump.example - -import scala.concurrent.Await -import scala.concurrent.duration.Duration +package org.apache.gearpump.akkastream.example import akka.actor.{Actor, ActorSystem, Props} -import akka.stream.gearpump.GearpumpMaterializer -import akka.stream.gearpump.scaladsl.GearSource -import akka.stream.scaladsl.Sink +import akka.stream.scaladsl.{Sink, Source} +import org.apache.gearpump.akkastream.GearpumpMaterializer +import org.apache.gearpump.cluster.main.ArgumentsParser +import org.apache.gearpump.util.AkkaApp + +import scala.concurrent.Await +import scala.concurrent.duration._ -import org.apache.gearpump.streaming.dsl.CollectionDataSource /** - * read from remote and write to local - * Usage: output/target/pack/bin/gear app -jar experiments/akkastream/target/scala.11/akkastream-2.11.5-0.6.2-SNAPSHOT-assembly.jar + * Source and Sink are materialized locally. + * Remaining GraphStages are materialized remotely: + * statefulMap, filter, fold, flatMap */ -object Test3 { - - def main(args: Array[String]): Unit = { - - println("running Test...") - - implicit val system = ActorSystem("akka-test") - implicit val materializer = new GearpumpMaterializer(system) +object Test extends AkkaApp with ArgumentsParser { + // scalastyle:off println + override def main(akkaConf: Config, args: Array[String]): Unit = { + implicit val system = ActorSystem("Test", akkaConf) + implicit val materializer = GearpumpMaterializer() val echo = system.actorOf(Props(new Echo())) val sink = Sink.actorRef(echo, "COMPLETE") - val sourceData = new CollectionDataSource(List("red hat", "yellow sweater", "blue jack", "red apple", "green plant", "blue sky")) - val source = GearSource.from[String](sourceData) - source.filter(_.startsWith("red")).map("I want to order item: " + _).runWith(sink) - Await.result(system.whenTerminated, Duration.Inf) + Source( + List("red hat", "yellow sweater", "blue jack", "red apple", "green plant", "blue sky") + ).filter(_.startsWith("red")).fold("Items:") {(a, b) => + a + "|" + b + }.map("I want to order item: " + _).runWith(sink) + + Await.result(system.whenTerminated, 60.minutes) } class Echo extends Actor { @@ -56,4 +57,5 @@ object Test3 { println("Confirm received: " + any) } } + // scalastyle:on println } diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test10.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test10.scala new file mode 100644 index 000000000..71678c3a7 --- /dev/null +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test10.scala @@ -0,0 +1,82 @@ +/* + * 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.gearpump.akkastream.example + +import akka.NotUsed +import akka.stream.{ClosedShape, ThrottleMode} +import org.apache.gearpump.akkastream.GearpumpMaterializer +import org.apache.gearpump.cluster.main.ArgumentsParser +import org.apache.gearpump.util.AkkaApp + +import scala.concurrent.Await +import scala.concurrent.duration._ + +/** + * Stream example showing Conflate, Throttle + */ +object Test10 extends AkkaApp with ArgumentsParser { + + // scalastyle:off println + override def main(akkaConf: Config, args: Array[String]): Unit = { + import akka.actor.ActorSystem + import akka.stream.scaladsl._ + + implicit val system = ActorSystem("Test10", akkaConfig) + implicit val materializer = GearpumpMaterializer() + implicit val ec = system.dispatcher + + // Conflate[A] - (2 inputs, 1 output) concatenates two streams + // (first consumes one, then the second one) + def stream(x: String) = Stream.continually(x) + + val sourceA = Source(stream("A")) + val sourceB = Source(stream("B")) + + val throttler: Flow[String, String, NotUsed] = + Flow[String].throttle(1, 1.second, 1, ThrottleMode.Shaping) + val conflateFlow: Flow[String, String, NotUsed] = + Flow[String].conflate((x: String, y: String) => x: String) + ((acc: String, x: String) => s"$acc::$x") + + val printFlow: Flow[(String, String), String, NotUsed] = + Flow[(String, String)].map { + x => + println(s" lengths are : ${x._1.length} and ${x._2.length} ; ${x._1} zip ${x._2}") + x.toString + } + + val graph = RunnableGraph.fromGraph(GraphDSL.create() { implicit b => + import GraphDSL.Implicits._ + + val zipping = b.add(Zip[String, String]()) + + sourceA ~> throttler ~> zipping.in0 + sourceB ~> conflateFlow ~> zipping.in1 + + zipping.out ~> printFlow ~> Sink.ignore + + ClosedShape + }) + + graph.run() + + Await.result(system.whenTerminated, 60.minutes) + } + // scalastyle:on println +} diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test11.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test11.scala new file mode 100644 index 000000000..b80398c0c --- /dev/null +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test11.scala @@ -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.gearpump.akkastream.example + +import akka.NotUsed +import akka.stream.ClosedShape +import org.apache.gearpump.akkastream.GearpumpMaterializer +import org.apache.gearpump.cluster.main.ArgumentsParser +import org.apache.gearpump.util.AkkaApp + +import scala.concurrent.Await +import scala.concurrent.duration._ + +/** + * Stream example showing Broadcast and Merge + */ +object Test11 extends AkkaApp with ArgumentsParser { + // scalastyle:off println + override def main(akkaConf: Config, args: Array[String]): Unit = { + import akka.actor.ActorSystem + import akka.stream.scaladsl._ + + implicit val system = ActorSystem("Test11", akkaConfig) + implicit val materializer = GearpumpMaterializer() +// implicit val materializer = +// ActorMaterializer(ActorMaterializerSettings(system).withAutoFusing(false)) + implicit val ec = system.dispatcher + + val g = RunnableGraph.fromGraph(GraphDSL.create() { + implicit builder: GraphDSL.Builder[NotUsed] => + + import GraphDSL.Implicits._ + val in = Source(1 to 10) + val output: (Any) => Unit = any => { + val s = s"**** $any" + println(s) + } + val out = Sink.foreach(output) + + val broadcast = builder.add(Broadcast[Int](2)) + val merge = builder.add(Merge[Int](2)) + + val f1, f2, f3, f4 = Flow[Int].map(_ + 10) + + in ~> f1 ~> broadcast ~> f2 ~> merge ~> f3 ~> out + broadcast ~> f4 ~> merge + + ClosedShape + }) + + g.run() + + Await.result(system.whenTerminated, 60.minutes) + } + // scalastyle:on println +} diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test12.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test12.scala new file mode 100644 index 000000000..a9e8b0817 --- /dev/null +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test12.scala @@ -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.gearpump.akkastream.example + +import akka.stream.{ClosedShape, UniformFanInShape} +import org.apache.gearpump.akkastream.GearpumpMaterializer +import org.apache.gearpump.cluster.main.ArgumentsParser +import org.apache.gearpump.util.AkkaApp + +import scala.concurrent.{Await, Future} + +/** + * Partial source, sink example + */ +object Test12 extends AkkaApp with ArgumentsParser{ + // scalastyle:off println + override def main(akkaConf: Config, args: Array[String]): Unit = { + import akka.actor.ActorSystem + import akka.stream.scaladsl._ + + import scala.concurrent.duration._ + + implicit val system = ActorSystem("Test12", akkaConfig) +// implicit val materializer = ActorMaterializer( +// ActorMaterializerSettings(system).withAutoFusing(false) +// ) + implicit val materializer = GearpumpMaterializer() + implicit val ec = system.dispatcher + + val pickMaxOfThree = GraphDSL.create() { implicit b => + import GraphDSL.Implicits._ + + val zip1 = b.add(ZipWith[Int, Int, Int](math.max)) + val zip2 = b.add(ZipWith[Int, Int, Int](math.max)) + + zip1.out ~> zip2.in0 + + UniformFanInShape(zip2.out, zip1.in0, zip1.in1, zip2.in1) + } + + val resultSink = Sink.head[Int] + + val g = RunnableGraph.fromGraph(GraphDSL.create(resultSink) { implicit b => + sink => + import GraphDSL.Implicits._ + + // Importing the partial shape will return its shape (inlets & outlets) + val pm3 = b.add(pickMaxOfThree) + + Source.single(1) ~> pm3.in(0) + Source.single(2) ~> pm3.in(1) + Source.single(3) ~> pm3.in(2) + + pm3.out ~> sink.in + + ClosedShape + }) + + val max: Future[Int] = g.run() + max.map(x => println(s"maximum of three numbers : $x")) + + Await.result(system.whenTerminated, 60.minutes) + } + // scalastyle:on println +} diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test13.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test13.scala new file mode 100644 index 000000000..984c861a6 --- /dev/null +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test13.scala @@ -0,0 +1,177 @@ +/* + * 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.gearpump.akkastream.example + +import java.time._ + +import akka.actor.ActorSystem +import akka.stream.scaladsl.Source +import org.apache.gearpump.akkastream.GearpumpMaterializer +import org.apache.gearpump.akkastream.scaladsl.Implicits._ +import org.apache.gearpump.cluster.main.ArgumentsParser +import org.apache.gearpump.util.AkkaApp + +import scala.collection.mutable +import scala.concurrent.Await +import scala.concurrent.duration._ +import scala.util.Random + +/** + * GroupBy example + */ + +/* +// Original example +val f = Source + .tick(0.seconds, 1.second, "") + .map { _ => + val now = System.currentTimeMillis() + val delay = random.nextInt(8) + MyEvent(now - delay * 1000L) + } + .statefulMapConcat { () => + val generator = new CommandGenerator() + ev => generator.forEvent(ev) + } + .groupBy(64, command => command.w) + .takeWhile(!_.isInstanceOf[CloseWindow]) + .fold(AggregateEventData((0L, 0L), 0))({ + case (agg, OpenWindow(window)) => agg.copy(w = window) + // always filtered out by takeWhile + case (agg, CloseWindow(_)) => agg + case (agg, AddToWindow(ev, _)) => agg.copy(eventCount = agg.eventCount + 1) + }) + .async + .mergeSubstreams + .runForeach { agg => + println(agg.toString) + } + */ +object Test13 extends AkkaApp with ArgumentsParser { + // scalastyle:off println + + override def main(akkaConf: Config, args: Array[String]): Unit = { + + implicit val system = ActorSystem("Test13", akkaConfig) + implicit val materializer = GearpumpMaterializer() + + val random = new Random() + + val result = Source + .tick(0.seconds, 1.second, "tick data") + .map { _ => + val now = System.currentTimeMillis() + val delay = random.nextInt(8) + MyEvent(now - delay * 1000L) + } + .statefulMapConcat { () => + val generator = new CommandGenerator() + ev => generator.forEvent(ev) + } + .groupBy2(command => command.w) + .takeWhile(!_.isInstanceOf[CloseWindow]) + .fold(AggregateEventData((0L, 0L), 0))({ + case (agg, OpenWindow(window)) => agg.copy(w = window) + // always filtered out by takeWhile + case (agg, CloseWindow(_)) => agg + case (agg, AddToWindow(ev, _)) => agg.copy(eventCount = agg.eventCount + 1) + }) + .runForeach(agg => + println(agg.toString) + ) + + Await.result(system.whenTerminated, 60.minutes) + } + + case class MyEvent(timestamp: Long) + + type Window = (Long, Long) + + object Window { + val WindowLength = 10.seconds.toMillis + val WindowStep = 1.second.toMillis + val WindowsPerEvent = (WindowLength / WindowStep).toInt + + def windowsFor(ts: Long): Set[Window] = { + val firstWindowStart = ts - ts % WindowStep - WindowLength + WindowStep + (for (i <- 0 until WindowsPerEvent) yield + (firstWindowStart + i * WindowStep, + firstWindowStart + i * WindowStep + WindowLength) + ).toSet + } + } + + sealed trait WindowCommand { + def w: Window + } + + case class OpenWindow(w: Window) extends WindowCommand + + case class CloseWindow(w: Window) extends WindowCommand + + case class AddToWindow(ev: MyEvent, w: Window) extends WindowCommand + + class CommandGenerator { + private val MaxDelay = 5.seconds.toMillis + private var watermark = 0L + private val openWindows = mutable.Set[Window]() + + def forEvent(ev: MyEvent): List[WindowCommand] = { + watermark = math.max(watermark, ev.timestamp - MaxDelay) + if (ev.timestamp < watermark) { + println(s"Dropping event with timestamp: ${tsToString(ev.timestamp)}") + Nil + } else { + val eventWindows = Window.windowsFor(ev.timestamp) + + val closeCommands = openWindows.flatMap { ow => + if (!eventWindows.contains(ow) && ow._2 < watermark) { + openWindows.remove(ow) + Some(CloseWindow(ow)) + } else None + } + + val openCommands = eventWindows.flatMap { w => + if (!openWindows.contains(w)) { + openWindows.add(w) + Some(OpenWindow(w)) + } else None + } + + val addCommands = eventWindows.map(w => AddToWindow(ev, w)) + + openCommands.toList ++ closeCommands.toList ++ addCommands.toList + } + } + } + + case class AggregateEventData(w: Window, eventCount: Int) { + override def toString: String = + s"Between ${tsToString(w._1)} and ${tsToString(w._2)}, there were $eventCount events." + } + + def tsToString(ts: Long): String = OffsetDateTime + .ofInstant(Instant.ofEpochMilli(ts), ZoneId.systemDefault()) + .toLocalTime + .toString + // scalastyle:on println + +} + + diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test14.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test14.scala new file mode 100644 index 000000000..0542f43f5 --- /dev/null +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test14.scala @@ -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.gearpump.akkastream.example + +import java.io.File + +import akka.NotUsed +import akka.actor.ActorSystem +import akka.stream._ +import akka.stream.scaladsl._ +import akka.util.ByteString +import org.apache.gearpump.akkastream.GearpumpMaterializer +import org.apache.gearpump.cluster.main.ArgumentsParser +import org.apache.gearpump.util.AkkaApp + +import scala.concurrent._ +import scala.concurrent.duration._ + +object Test14 extends AkkaApp with ArgumentsParser { + // scalastyle:off println + override def main(akkaConf: Config, args: Array[String]): Unit = { + implicit val system = ActorSystem("Test14", akkaConf) + implicit val materializer = GearpumpMaterializer() + + def lineSink(filename: String): Sink[String, Future[IOResult]] = { + Flow[String] + .alsoTo(Sink.foreach(s => println(s"$filename: $s"))) + .map(s => ByteString(s + "\n")) + .toMat(FileIO.toPath(new File(filename).toPath))(Keep.right) + } + + val source: Source[Int, NotUsed] = Source(1 to 100) + val factorials: Source[BigInt, NotUsed] = source.scan(BigInt(1))((acc, next) => acc * next) + val sink1 = lineSink("factorial1.txt") + val sink2 = lineSink("factorial2.txt") + val slowSink2 = Flow[String].via( + Flow[String].throttle(1, 1.second, 1, ThrottleMode.shaping) + ).toMat(sink2)(Keep.right) + val bufferedSink2 = Flow[String].buffer(50, OverflowStrategy.backpressure).via( + Flow[String].throttle(1, 1.second, 1, ThrottleMode.shaping) + ).toMat(sink2)(Keep.right) + + val g = RunnableGraph.fromGraph(GraphDSL.create() { implicit b => + import GraphDSL.Implicits._ + val bcast = b.add(Broadcast[String](2)) + factorials.map(_.toString) ~> bcast.in + bcast.out(0) ~> sink1 + bcast.out(1) ~> bufferedSink2 + ClosedShape + }) + + g.run() + + Await.result(system.whenTerminated, 60.minutes) + } + // scalastyle:on println +} diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test15.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test15.scala new file mode 100644 index 000000000..c2f8d5f6c --- /dev/null +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test15.scala @@ -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.gearpump.akkastream.example + +import akka.actor.ActorSystem +import akka.stream._ +import akka.stream.scaladsl.{Balance, Broadcast, Flow, GraphDSL, Merge, RunnableGraph, Sink, Source} +import org.apache.gearpump.akkastream.GearpumpMaterializer +import org.apache.gearpump.cluster.main.{ArgumentsParser, CLIOption} +import org.apache.gearpump.util.AkkaApp + +import scala.concurrent.Await +import scala.concurrent.duration._ + + +object Test15 extends AkkaApp with ArgumentsParser { + // scalastyle:off println + override val options: Array[(String, CLIOption[Any])] = Array( + "gearpump" -> CLIOption[Boolean]("", required = false, defaultValue = Some(false)) + ) + + override def main(akkaConf: Config, args: Array[String]): Unit = { + val config = parse(args) + implicit val system = ActorSystem("Test15", akkaConf) + implicit val materializer: ActorMaterializer = config.getBoolean("gearpump") match { + case true => + GearpumpMaterializer() + case false => + ActorMaterializer( + ActorMaterializerSettings(system).withAutoFusing(false) + ) + } + import akka.stream.scaladsl.GraphDSL.Implicits._ + RunnableGraph.fromGraph(GraphDSL.create() { implicit builder => + val A = builder.add(Source.single(0)).out + val B = builder.add(Broadcast[Int](2)) + val C = builder.add(Merge[Int](2).named("C")) + val D = builder.add(Flow[Int].map(_ + 1).named("D")) + val E = builder.add(Balance[Int](2).named("E")) + val F = builder.add(Merge[Int](2).named("F")) + val G = builder.add(Sink.foreach(println).named("G")).in + + C <~ F + A ~> B ~> C ~> F + B ~> D ~> E ~> F + E ~> G + + ClosedShape + }).run() + + Await.result(system.whenTerminated, 60.minutes) + } + // scalastyle:on println +} + + diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test16.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test16.scala new file mode 100644 index 000000000..eb0b5c7c0 --- /dev/null +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test16.scala @@ -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.gearpump.akkastream.example + +import akka.actor.{Actor, ActorSystem, Props} +import org.apache.gearpump.akkastream.GearpumpMaterializer +import org.apache.gearpump.akkastream.scaladsl.{GearSink, GearSource} +import akka.stream.scaladsl.Sink +import org.apache.gearpump.cluster.main.ArgumentsParser +import org.apache.gearpump.streaming.dsl.{CollectionDataSource, LoggerSink} +import org.apache.gearpump.util.AkkaApp + +import scala.concurrent.Await +import scala.concurrent.duration._ + +/** + * All remote + */ +object Test16 extends AkkaApp with ArgumentsParser { + // scalastyle:off println + override def main(akkaConf: Config, args: Array[String]): Unit = { + implicit val system = ActorSystem("Test16", akkaConf) + implicit val materializer = GearpumpMaterializer() + + val sink = GearSink.to(new LoggerSink[String]) + val sourceData = new CollectionDataSource( + List("red hat", "yellow sweater", "blue jack", "red apple", "green plant", "blue sky")) + val source = GearSource.from[String](sourceData) + source.filter(_.startsWith("red")).map("I want to order item: " + _).runWith(sink) + + Await.result(system.whenTerminated, 60.minutes) + } + // scalastyle:on println +} diff --git a/experiments/akkastream/src/main/scala/akka/stream/gearpump/example/Test2.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test2.scala similarity index 52% rename from experiments/akkastream/src/main/scala/akka/stream/gearpump/example/Test2.scala rename to experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test2.scala index 2426f5f34..21f1b8c4a 100644 --- a/experiments/akkastream/src/main/scala/akka/stream/gearpump/example/Test2.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test2.scala @@ -7,7 +7,7 @@ * "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 + * 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, @@ -16,50 +16,55 @@ * limitations under the License. */ -package akka.stream.gearpump.example - -import scala.concurrent.Await -import scala.concurrent.duration.Duration +package org.apache.gearpump.akkastream.example import akka.actor.{Actor, ActorSystem, Props} -import akka.stream.ActorMaterializer -import akka.stream.gearpump.GearpumpMaterializer -import akka.stream.gearpump.scaladsl.{GearSink, GearSource} -import akka.stream.scaladsl.{Flow, Sink, Source} +import akka.stream.scaladsl._ +import akka.stream.{ActorMaterializer, ClosedShape} +import org.apache.gearpump.akkastream.GearpumpMaterializer +import org.apache.gearpump.akkastream.scaladsl.{GearSink, GearSource} +import org.apache.gearpump.cluster.main.ArgumentsParser +import org.apache.gearpump.util.AkkaApp + +import scala.concurrent.Await +import scala.concurrent.duration._ /** * * This tests how different Materializers can be used together in an explicit way. - * Usage: output/target/pack/bin/gear app -jar experiments/akkastream/target/scala.11/akkastream-2.11.5-0.6.2-SNAPSHOT-assembly.jar * */ -object Test2 { - - def main(args: Array[String]): Unit = { - - println("running Test2...") - implicit val system = ActorSystem("akka-test") - val materializer = new GearpumpMaterializer(system) +object Test2 extends AkkaApp with ArgumentsParser { + // scalastyle:off println + override def main(akkaConf: Config, args: Array[String]): Unit = { + val config = parse(args) + implicit val system = ActorSystem("Test2", akkaConf) + val gearpumpMaterializer = GearpumpMaterializer() val echo = system.actorOf(Props(new Echo())) val source = GearSource.bridge[String, String] val sink = GearSink.bridge[String, String] val flow = Flow[String].filter(_.startsWith("red")).map("I want to order item: " + _) - val (entry, exit) = flow.runWith(source, sink)(materializer) + val (entry, exit) = flow.runWith(source, sink)(gearpumpMaterializer) val actorMaterializer = ActorMaterializer() - val externalSource = Source(List("red hat", "yellow sweater", "blue jack", "red apple", "green plant", "blue sky")) + val externalSource = Source( + List("red hat", "yellow sweater", "blue jack", "red apple", "green plant", "blue sky") + ) val externalSink = Sink.actorRef(echo, "COMPLETE") - val graph = FlowGraph.closed() { implicit b => - externalSource ~> Sink(entry) - Source(exit) ~> externalSink - } - graph.run()(actorMaterializer) + RunnableGraph.fromGraph( + GraphDSL.create() { implicit b => + import GraphDSL.Implicits._ + externalSource ~> Sink.fromSubscriber(entry) + Source.fromPublisher(exit) ~> externalSink + ClosedShape + } + ).run()(actorMaterializer) - Await.result(system.whenTerminated, Duration.Inf) + Await.result(system.whenTerminated, 60.minutes) } class Echo extends Actor { @@ -68,4 +73,5 @@ object Test2 { println("Confirm received: " + any) } } -} \ No newline at end of file + // scalastyle:on println +} diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test3.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test3.scala new file mode 100644 index 000000000..0a51078c8 --- /dev/null +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test3.scala @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gearpump.akkastream.example + +import akka.actor.{Actor, ActorSystem, Props} +import akka.stream.{ActorMaterializer, ActorMaterializerSettings} +import org.apache.gearpump.akkastream.GearpumpMaterializer +import org.apache.gearpump.akkastream.scaladsl.GearSource +import akka.stream.scaladsl.Sink +import org.apache.gearpump.cluster.main.{ArgumentsParser, CLIOption} +import org.apache.gearpump.streaming.dsl.CollectionDataSource +import org.apache.gearpump.util.AkkaApp + +import scala.concurrent.Await +import scala.concurrent.duration._ + +/** + * read from remote and write to local + */ +object Test3 extends AkkaApp with ArgumentsParser { + // scalastyle:off println + override val options: Array[(String, CLIOption[Any])] = Array( + "gearpump" -> CLIOption[Boolean]("", required = false, defaultValue = Some(false)) + ) + + override def main(akkaConf: Config, args: Array[String]): Unit = { + val config = parse(args) + implicit val system = ActorSystem("Test3", akkaConf) + implicit val materializer: ActorMaterializer = config.getBoolean("gearpump") match { + case true => + GearpumpMaterializer() + case false => + ActorMaterializer( + ActorMaterializerSettings(system).withAutoFusing(false) + ) + } + val echo = system.actorOf(Props(new Echo())) + val sink = Sink.actorRef(echo, "COMPLETE") + val sourceData = new CollectionDataSource( + List("red hat", "yellow sweater", "blue jack", "red apple", "green plant", "blue sky")) + val source = GearSource.from[String](sourceData) + source.filter(_.startsWith("red")).map("I want to order item: " + _).runWith(sink) + + Await.result(system.whenTerminated, 60.minutes) + } + + class Echo extends Actor { + def receive: Receive = { + case any: AnyRef => + println("Confirm received: " + any) + } + } + // scalastyle:on println +} diff --git a/experiments/akkastream/src/main/scala/akka/stream/gearpump/example/Test4.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test4.scala similarity index 51% rename from experiments/akkastream/src/main/scala/akka/stream/gearpump/example/Test4.scala rename to experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test4.scala index 7b80b7b7c..3cb69ced2 100644 --- a/experiments/akkastream/src/main/scala/akka/stream/gearpump/example/Test4.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test4.scala @@ -7,7 +7,7 @@ * "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 + * 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, @@ -16,35 +16,35 @@ * limitations under the License. */ -package akka.stream.gearpump.example - -import scala.concurrent.Await -import scala.concurrent.duration.Duration +package org.apache.gearpump.akkastream.example import akka.actor.ActorSystem -import akka.stream.gearpump.GearpumpMaterializer -import akka.stream.gearpump.scaladsl.GearSink import akka.stream.scaladsl.Source - +import org.apache.gearpump.akkastream.GearpumpMaterializer +import org.apache.gearpump.akkastream.scaladsl.GearSink +import org.apache.gearpump.cluster.main.ArgumentsParser import org.apache.gearpump.streaming.dsl.LoggerSink +import org.apache.gearpump.util.AkkaApp + +import scala.concurrent.Await +import scala.concurrent.duration._ /** * read from local and write to remote - * Usage: output/target/pack/bin/gear app -jar experiments/akkastream/target/scala.11/akkastream-2.11.5-0.6.2-SNAPSHOT-assembly.jar */ -object Test4 { - - def main(args: Array[String]): Unit = { - - println("running Test...") - - implicit val system = ActorSystem("akka-test") - implicit val materializer = new GearpumpMaterializer(system) - - val sink = GearSink.to(new LoggerSink[String]) - val source = Source(List("red hat", "yellow sweater", "blue jack", "red apple", "green plant", "blue sky")) - source.filter(_.startsWith("red")).map("I want to order item: " + _).runWith(sink) - - Await.result(system.whenTerminated, Duration.Inf) +object Test4 extends AkkaApp with ArgumentsParser { + // scalastyle:off println + override def main(akkaConf: Config, args: Array[String]): Unit = { + implicit val system = ActorSystem("Test4", akkaConf) + implicit val materializer = GearpumpMaterializer() + + Source( + List("red hat", "yellow sweater", "blue jack", "red apple", "green plant", "blue sky") + ).filter(_.startsWith("red")). + map("I want to order item: " + _). + runWith(GearSink.to(new LoggerSink[String])) + + Await.result(system.whenTerminated, 60.minutes) } -} \ No newline at end of file + // scalastyle:on println +} diff --git a/experiments/akkastream/src/main/scala/akka/stream/gearpump/example/Test5.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test5.scala similarity index 51% rename from experiments/akkastream/src/main/scala/akka/stream/gearpump/example/Test5.scala rename to experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test5.scala index 052c0184b..72e21c77f 100644 --- a/experiments/akkastream/src/main/scala/akka/stream/gearpump/example/Test5.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test5.scala @@ -7,7 +7,7 @@ * "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 + * 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, @@ -16,47 +16,45 @@ * limitations under the License. */ -package akka.stream.gearpump.example - -import scala.concurrent.Await -import scala.concurrent.duration.Duration +package org.apache.gearpump.akkastream.example import akka.actor.{Actor, ActorSystem, Props} -import akka.stream.gearpump.GearpumpMaterializer -import akka.stream.gearpump.graph.GraphCutter -import akka.stream.scaladsl.{Sink, Source, Unzip} - -/** -test fanout - */ -object Test5 { +import akka.stream.ClosedShape +import akka.stream.scaladsl._ +import org.apache.gearpump.akkastream.GearpumpMaterializer +import org.apache.gearpump.cluster.main.ArgumentsParser +import org.apache.gearpump.util.AkkaApp - def main(args: Array[String]): Unit = { - - println("running Test...") +import scala.concurrent.Await +import scala.concurrent.duration._ - implicit val system = ActorSystem("akka-test") - implicit val materializer = new GearpumpMaterializer(system, GraphCutter.AllRemoteStrategy) +/** + * test fanout + */ +object Test5 extends AkkaApp with ArgumentsParser { + // scalastyle:off println + override def main(akkaConf: Config, args: Array[String]): Unit = { + implicit val system = ActorSystem("Test5", akkaConf) + implicit val materializer = GearpumpMaterializer() val echo = system.actorOf(Props(new Echo())) - val sink = Sink.actorRef(echo, "COMPLETE") - val source = Source(List(("male", "24"), ("female", "23"))) + val sink = Sink.actorRef(echo, "COMPLETE") - val graph = FlowGraph.closed() { implicit b => - val unzip = b.add(Unzip[String, String]()) - - val sink1 = Sink.actorRef(echo, "COMPLETE") - val sink2 = Sink.actorRef(echo, "COMPLETE") - - source ~> unzip.in - unzip.out0 ~> sink1 - unzip.out1 ~> sink1 - } - - graph.run() + RunnableGraph.fromGraph( + GraphDSL.create() { implicit b => + import GraphDSL.Implicits._ + val unzip = b.add(Unzip[String, String]()) + val sink1 = Sink.actorRef(echo, "COMPLETE") + val sink2 = Sink.actorRef(echo, "COMPLETE") + source ~> unzip.in + unzip.out0 ~> sink1 + unzip.out1 ~> sink1 + ClosedShape + } + ).run() - Await.result(system.whenTerminated, Duration.Inf) + Await.result(system.whenTerminated, 60.minutes) } class Echo extends Actor { @@ -65,4 +63,5 @@ object Test5 { println("Confirm received: " + any) } } -} \ No newline at end of file + // scalastyle:on println +} diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test6.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test6.scala new file mode 100644 index 000000000..6f5493358 --- /dev/null +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test6.scala @@ -0,0 +1,90 @@ +/* + * 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.gearpump.akkastream.example + +import akka.actor.{Actor, ActorSystem, Props} +import akka.stream.scaladsl.Sink +import akka.stream.{ActorMaterializer, ActorMaterializerSettings} +import org.apache.gearpump.akkastream.GearpumpMaterializer +import org.apache.gearpump.akkastream.scaladsl.GearSource +import org.apache.gearpump.cluster.main.{ArgumentsParser, CLIOption} +import org.apache.gearpump.streaming.dsl.CollectionDataSource +import org.apache.gearpump.util.AkkaApp + +import scala.concurrent.Await +import scala.concurrent.duration._ + + +/** + * WordCount example + * Test GroupBy2 (groupBy which uses SubFlow is not implemented yet) + */ + +import org.apache.gearpump.akkastream.scaladsl.Implicits._ + +object Test6 extends AkkaApp with ArgumentsParser { + // scalastyle:off println + override val options: Array[(String, CLIOption[Any])] = Array( + "gearpump" -> CLIOption[Boolean]("", required = false, defaultValue = Some(false)) + ) + + override def main(akkaConf: Config, args: Array[String]): Unit = { + val config = parse(args) + implicit val system = ActorSystem("Test6", akkaConf) + implicit val materializer: ActorMaterializer = config.getBoolean("gearpump") match { + case true => + GearpumpMaterializer() + case false => + ActorMaterializer( + ActorMaterializerSettings(system).withAutoFusing(false) + ) + } + val echo = system.actorOf(Props(Echo())) + val sink = Sink.actorRef(echo, "COMPLETE") + val sourceData = new CollectionDataSource( + List( + "this is a good start", + "this is a good time", + "time to start", + "congratulations", + "green plant", + "blue sky") + ) + val source = GearSource.from[String](sourceData) + source.mapConcat({line => + line.split(" ").toList + }).groupBy2(x => x) + .map(word => (word, 1)) + .reduce({(a, b) => + (a._1, a._2 + b._2) + }) + .log("word-count") + .runWith(sink) + + Await.result(system.whenTerminated, 60.minutes) + } + + case class Echo() extends Actor { + def receive: Receive = { + case any: AnyRef => + println("Confirm received: " + any) + } + } + // scalastyle:on println +} diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test7.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test7.scala new file mode 100644 index 000000000..be916103e --- /dev/null +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test7.scala @@ -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.gearpump.akkastream.example + +import akka.actor.ActorSystem +import akka.stream.scaladsl.{Broadcast, Merge, Sink, Source} +import org.apache.gearpump.akkastream.GearpumpMaterializer +import org.apache.gearpump.cluster.main.ArgumentsParser +import org.apache.gearpump.util.AkkaApp + +import scala.concurrent.Await +import scala.concurrent.duration._ + + +/** + * This is a simplified API you can use to combine sources and sinks + * with junctions like: Broadcast[T], Balance[T], Merge[In] and Concat[A] + * without the need for using the Graph DSL + */ + +object Test7 extends AkkaApp with ArgumentsParser { + // scalastyle:off println + override def main(akkaConf: Config, args: Array[String]): Unit = { + implicit val system = ActorSystem("Test7", akkaConf) + implicit val materializer = GearpumpMaterializer() + implicit val ec = system.dispatcher + + val sourceA = Source(List(1)) + val sourceB = Source(List(2)) + val mergedSource = Source.combine(sourceA, sourceB)(Merge(_)) + + val sinkA = Sink.foreach[Int](x => println(s"In SinkA : $x")) + val sinkB = Sink.foreach[Int](x => println(s"In SinkB : $x")) + val sink = Sink.combine(sinkA, sinkB)(Broadcast[Int](_)) + mergedSource.runWith(sink) + + Await.result(system.whenTerminated, 60.minutes) + } + // scalastyle:on println +} diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test8.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test8.scala new file mode 100644 index 000000000..434aa3353 --- /dev/null +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test8.scala @@ -0,0 +1,66 @@ +/* + * 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.gearpump.akkastream.example + +import akka.NotUsed +import akka.actor.ActorSystem +import akka.stream.{ActorMaterializer, ActorMaterializerSettings, Materializer} +import akka.stream.scaladsl._ +import org.apache.gearpump.akkastream.GearpumpMaterializer +import org.apache.gearpump.cluster.main.{ArgumentsParser, CLIOption} +import org.apache.gearpump.util.AkkaApp + +import scala.concurrent.{Await, Future} +import scala.concurrent.duration._ + +/** + * Stream example to find sum of elements + */ +object Test8 extends AkkaApp with ArgumentsParser { + // scalastyle:off println + override val options: Array[(String, CLIOption[Any])] = Array( + "gearpump" -> CLIOption[Boolean]("", required = false, defaultValue = Some(false)) + ) + + override def main(akkaConf: Config, args: Array[String]): Unit = { + val config = parse(args) + implicit val system = ActorSystem("Test8", akkaConf) + implicit val materializer: ActorMaterializer = config.getBoolean("gearpump") match { + case true => + GearpumpMaterializer() + case false => + ActorMaterializer( + ActorMaterializerSettings(system).withAutoFusing(false) + ) + } + implicit val ec = system.dispatcher + + // Source gives 1 to 100 elements + val source: Source[Int, NotUsed] = Source(Stream.from(1).take(100)) + val sink: Sink[Int, Future[Int]] = Sink.fold[Int, Int](0)(_ + _) + + val result: Future[Int] = source.runWith(sink) + result.map(sum => { + println(s"Sum of stream elements => $sum") + }) + + Await.result(system.whenTerminated, 60.minutes) + } + // scalastyle:on println +} diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test9.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test9.scala new file mode 100644 index 000000000..63f9e2dd0 --- /dev/null +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test9.scala @@ -0,0 +1,87 @@ +/* + * 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.gearpump.akkastream.example + +import akka.NotUsed +import akka.actor.{Actor, ActorSystem, Props} +import akka.stream.{ActorMaterializer, ActorMaterializerSettings, ClosedShape} +import akka.stream.scaladsl._ +import org.apache.gearpump.akkastream.GearpumpMaterializer +import org.apache.gearpump.cluster.main.{ArgumentsParser, CLIOption} +import org.apache.gearpump.util.AkkaApp + +import scala.concurrent.{Await, Future} +import scala.concurrent.duration._ + +/** + * Stream example showing Broadcast + */ +object Test9 extends AkkaApp with ArgumentsParser { + // scalastyle:off println + override val options: Array[(String, CLIOption[Any])] = Array( + "gearpump" -> CLIOption[Boolean]("", required = false, defaultValue = Some(false)) + ) + + override def main(akkaConf: Config, args: Array[String]): Unit = { + val config = parse(args) + implicit val system = ActorSystem("Test9", akkaConf) + implicit val materializer: ActorMaterializer = config.getBoolean("gearpump") match { + case true => + GearpumpMaterializer() + case false => + ActorMaterializer( + ActorMaterializerSettings(system).withAutoFusing(false) + ) + } + implicit val ec = system.dispatcher + + val sinkActor = system.actorOf(Props(new SinkActor())) + val source = Source((1 to 5)) + val sink = Sink.actorRef(sinkActor, "COMPLETE") + val flowA: Flow[Int, Int, NotUsed] = Flow[Int].map { + x => println(s"processing broadcasted element : $x in flowA"); x + } + val flowB: Flow[Int, Int, NotUsed] = Flow[Int].map { + x => println(s"processing broadcasted element : $x in flowB"); x + } + + val graph = RunnableGraph.fromGraph(GraphDSL.create() { implicit b => + import GraphDSL.Implicits._ + val broadcast = b.add(Broadcast[Int](2)) + val merge = b.add(Merge[Int](2)) + source ~> broadcast + broadcast ~> flowA ~> merge + broadcast ~> flowB ~> merge + merge ~> sink + ClosedShape + }) + + graph.run() + + Await.result(system.whenTerminated, 60.minutes) + } + + class SinkActor extends Actor { + def receive: Receive = { + case any: AnyRef => + println("Confirm received: " + any) + } + } + // scalastyle:on println +} diff --git a/experiments/akkastream/src/main/scala/akka/stream/gearpump/example/WikipediaApp.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/WikipediaApp.scala similarity index 61% rename from experiments/akkastream/src/main/scala/akka/stream/gearpump/example/WikipediaApp.scala rename to experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/WikipediaApp.scala index 56b89bcd6..7e2211d42 100644 --- a/experiments/akkastream/src/main/scala/akka/stream/gearpump/example/WikipediaApp.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/WikipediaApp.scala @@ -7,7 +7,7 @@ * "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 + * 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, @@ -16,30 +16,34 @@ * limitations under the License. */ -package akka.stream.gearpump.example +package org.apache.gearpump.akkastream.example import java.io.{File, FileInputStream} import java.util.zip.GZIPInputStream -import scala.concurrent.duration.Duration -import scala.concurrent.{Await, ExecutionContext, Future} -import scala.util.{Failure, Success, Try} +import akka.NotUsed import akka.actor.ActorSystem -import akka.stream.gearpump.graph.GraphCutter -import akka.stream.gearpump.{GearAttributes, GearpumpMaterializer} import akka.stream.scaladsl._ +import akka.stream.{ClosedShape, IOResult} import akka.util.ByteString -import org.json4s.JsonAST.JString - +import org.apache.gearpump.akkastream.graph.GraphPartitioner +import org.apache.gearpump.akkastream.{GearAttributes, GearpumpMaterializer} import org.apache.gearpump.cluster.main.{ArgumentsParser, CLIOption} import org.apache.gearpump.util.AkkaApp +import org.json4s.JsonAST.JString +import org.json4s.jackson.JsonMethods + +import scala.concurrent.duration._ +import scala.concurrent.{Await, ExecutionContext, Future} +import scala.util.{Failure, Success, Try} /** * this example is ported from http://engineering.intenthq.com/2015/06/wikidata-akka-streams/ * which showcases running Akka Streams DSL across JVMs on Gearpump * - * Usage: output/target/pack/bin/gear app -jar experiments/akkastream/target/scala_2.11/akkastream-${VERSION}-SNAPSHOT-assembly.jar - * -input wikidata-${DATE}-all.json.gz -languages en,de + * Usage: output/target/pack/bin/gear app + * -jar experiments/akkastream/target/scala_2.11/akkastream-${VERSION}-SNAPSHOT-assembly.jar + * -input wikidata-${DATE}-all.json.gz -languages en,de * * (Note: Wikipedia data can be downloaded from https://dumps.wikimedia.org/wikidatawiki/entities/) * @@ -58,54 +62,60 @@ object WikipediaApp extends ArgumentsParser with AkkaApp { val input = new File(parsed.getString("input")) val langs = parsed.getString("languages").split(",") - implicit val system = ActorSystem("wikidata-poc", akkaConf) - implicit val materializer = new GearpumpMaterializer(system, GraphCutter.TagAttributeStrategy, useLocalCluster = false) + implicit val system = ActorSystem("WikipediaApp", akkaConf) + implicit val materializer = + GearpumpMaterializer(GraphPartitioner.TagAttributeStrategy) import system.dispatcher val elements = source(input).via(parseJson(langs)) - val g = FlowGraph.closed(count) { implicit b => - sinkCount => { - - val broadcast = b.add(Broadcast[WikidataElement](2)) - elements ~> broadcast ~> logEveryNSink(1000) - broadcast ~> checkSameTitles(langs.toSet) ~> sinkCount + val g = RunnableGraph.fromGraph( + GraphDSL.create(count) { implicit b => + sinkCount => { + import GraphDSL.Implicits._ + val broadcast = b.add(Broadcast[WikidataElement](2)) + elements ~> broadcast ~> logEveryNSink(1000) + broadcast ~> checkSameTitles(langs.toSet) ~> sinkCount + ClosedShape + } } - } + ) g.run().onComplete { x => x match { case Success((t, f)) => printResults(t, f) + // scalastyle:off println case Failure(tr) => println("Something went wrong") + // scalastyle:on println } - system.terminate() } - Await.result(system.whenTerminated, Duration.Inf) + Await.result(system.whenTerminated, 60.minutes) } - def source(file: File): Source[String, Future[Long]] = { + def source(file: File): Source[String, Future[IOResult]] = { val compressed = new GZIPInputStream(new FileInputStream(file), 65536) - InputStreamSource(() => compressed) + StreamConverters.fromInputStream(() => compressed) .via(Framing.delimiter(ByteString("\n"), Int.MaxValue)) .map(x => x.decodeString("utf-8")) } - def parseJson(langs: Seq[String])(implicit ec: ExecutionContext): Flow[String, WikidataElement, Unit] = - Flow[String].mapAsyncUnordered(8)(line => Future(parseItem(langs, line))).collect { + def parseJson(langs: Seq[String])(implicit ec: ExecutionContext): + Flow[String, WikidataElement, NotUsed] = + Flow[String].mapAsyncUnordered(8)(line => Future(parseItem(langs, line))).collect({ case Some(v) => v - } + }) def parseItem(langs: Seq[String], line: String): Option[WikidataElement] = { - import org.json4s.jackson.JsonMethods Try(JsonMethods.parse(line)).toOption.flatMap { json => json \ "id" match { case JString(itemId) => - val sites = for { + + val sites: Seq[(String, String)] = for { lang <- langs JString(title) <- json \ "sitelinks" \ s"${lang}wiki" \ "title" } yield lang -> title - if (sites.isEmpty) None + if(sites.isEmpty) None else Some(WikidataElement(id = itemId, sites = sites.toMap)) case _ => None @@ -113,17 +123,21 @@ object WikipediaApp extends ArgumentsParser with AkkaApp { } } - def logEveryNSink[T](n: Int) = Sink.fold(0) { (x, y: T) => - if (x % n == 0) + def logEveryNSink[T](n: Int): Sink[T, Future[Int]] = Sink.fold(0) { (x, y: T) => + if (x % n == 0) { + // scalastyle:off println println(s"Processing element $x: $y") + // scalastyle:on println + } x + 1 } - def checkSameTitles(langs: Set[String]): Flow[WikidataElement, Boolean, Unit] = Flow[WikidataElement] + def checkSameTitles(langs: Set[String]): + Flow[WikidataElement, Boolean, NotUsed] = Flow[WikidataElement] .filter(_.sites.keySet == langs) .map { x => val titles = x.sites.values - titles.forall(_ == titles.head) + titles.forall( _ == titles.head) }.withAttributes(GearAttributes.remote) def count: Sink[Boolean, Future[(Int, Int)]] = Sink.fold((0, 0)) { @@ -131,13 +145,15 @@ object WikipediaApp extends ArgumentsParser with AkkaApp { case ((t, f), false) => (t, f + 1) } - def printResults(t: Int, f: Int) = { - val message = - s""" - | Number of items with the same title: $t - | Number of items with the different title: $f - | Ratios: ${t.toDouble / (t + f)} / ${f.toDouble / (t + f)} - """.stripMargin + def printResults(t: Int, f: Int): Unit = { + val message = s""" + | Number of items with the same title: $t + | Number of items with the different title: $f + | Ratios: ${t.toDouble / (t + f)} / ${f.toDouble / (t + f)} + """.stripMargin + // scalastyle:off println println(message) + // scalastyle:on println } + } diff --git a/experiments/akkastream/src/main/scala/akka/stream/gearpump/graph/GraphCutter.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/GraphPartitioner.scala similarity index 65% rename from experiments/akkastream/src/main/scala/akka/stream/gearpump/graph/GraphCutter.scala rename to experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/GraphPartitioner.scala index 19083f6f9..c1e95bb8e 100644 --- a/experiments/akkastream/src/main/scala/akka/stream/gearpump/graph/GraphCutter.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/GraphPartitioner.scala @@ -7,7 +7,7 @@ * "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 + * 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, @@ -16,23 +16,23 @@ * limitations under the License. */ -package akka.stream.gearpump.graph - -import akka.stream.ModuleGraph -import akka.stream.ModuleGraph.Edge -import akka.stream.gearpump.GearAttributes -import akka.stream.gearpump.GearAttributes.{Local, Location, Remote} -import akka.stream.gearpump.graph.GraphCutter.Strategy -import akka.stream.gearpump.module.{BridgeModule, DummyModule, GearpumpTaskModule, GroupByModule, SinkBridgeModule, SourceBridgeModule} -import akka.stream.impl.Stages.DirectProcessor -import akka.stream.impl.StreamLayout.{MaterializedValueNode, Module} +package org.apache.gearpump.akkastream.graph + +import akka.stream.{Shape, SinkShape, SourceShape} +import org.apache.gearpump.akkastream.GearAttributes +import org.apache.gearpump.akkastream.GearAttributes.{Local, Location, Remote} +import org.apache.gearpump.akkastream.GearpumpMaterializer.Edge +import org.apache.gearpump.akkastream.graph.GraphPartitioner.Strategy +import org.apache.gearpump.akkastream.module._ +import akka.stream.impl.StreamLayout.Module +import akka.stream.impl.fusing.GraphStageModule +import akka.stream.impl.fusing.GraphStages.{MaterializedValueSource, SimpleLinearGraphStage, SingleSource} import akka.stream.impl.{SinkModule, SourceModule} - import org.apache.gearpump.util.Graph /** * - * GraphCutter is used to decide which part is rendered locally + * GraphPartitioner is used to decide which part will be rendered locally * and which part should be rendered remotely. * * We will cut the graph based on the [[Strategy]] provided. @@ -54,23 +54,22 @@ import org.apache.gearpump.util.Graph * \| / * AtomicModule3 * - * - * @see [[ModuleGraph]] for more information of how Graph is organized. + * @see [[akka.stream.impl.MaterializerSession]] for more information of how Graph is organized. * */ -class GraphCutter(strategy: Strategy) { - def cut(moduleGraph: ModuleGraph[_]): List[SubGraph] = { - val graph = removeDummyModule(moduleGraph.graph) +class GraphPartitioner(strategy: Strategy) { + def partition(moduleGraph: Graph[Module, Edge]): List[SubGraph] = { + val graph = removeDummyModule(moduleGraph) val tags = tag(graph, strategy) - doCut(graph, tags, moduleGraph.mat) + doPartition(graph, tags) } - private def doCut(graph: Graph[Module, Edge], tags: Map[Module, Location], - mat: MaterializedValueNode): List[SubGraph] = { + private def doPartition(graph: Graph[Module, Edge], tags: Map[Module, Location]): + List[SubGraph] = { val local = Graph.empty[Module, Edge] val remote = Graph.empty[Module, Edge] - graph.vertices.foreach { module => + graph.vertices.foreach{ module => if (tags(module) == Local) { local.addVertex(module) } else { @@ -78,7 +77,7 @@ class GraphCutter(strategy: Strategy) { } } - graph.edges.foreach { nodeEdgeNode => + graph.edges.foreach{ nodeEdgeNode => val (node1, edge, node2) = nodeEdgeNode (tags(node1), tags(node2)) match { case (Local, Local) => @@ -90,7 +89,7 @@ class GraphCutter(strategy: Strategy) { case bridge: BridgeModule[_, _, _] => local.addEdge(node1, edge, node2) case _ => - // Creates a bridge module in between + // create a bridge module in between val bridge = new SourceBridgeModule[AnyRef, AnyRef]() val remoteEdge = Edge(bridge.outPort, edge.to) remote.addEdge(bridge, remoteEdge, node2) @@ -102,7 +101,7 @@ class GraphCutter(strategy: Strategy) { case bridge: BridgeModule[_, _, _] => local.addEdge(node1, edge, node2) case _ => - // Creates a bridge module in between + // create a bridge module in between val bridge = new SinkBridgeModule[AnyRef, AnyRef]() val remoteEdge = Edge(edge.from, bridge.inPort) remote.addEdge(node1, remoteEdge, bridge) @@ -116,14 +115,14 @@ class GraphCutter(strategy: Strategy) { } private def tag(graph: Graph[Module, Edge], strategy: Strategy): Map[Module, Location] = { - graph.vertices.map { vertex => + graph.vertices.map{vertex => vertex -> strategy.apply(vertex) }.toMap } private def removeDummyModule(inputGraph: Graph[Module, Edge]): Graph[Module, Edge] = { val graph = inputGraph.copy - val dummies = graph.vertices.filter { module => + val dummies = graph.vertices.filter {module => module match { case dummy: DummyModule => true @@ -136,7 +135,7 @@ class GraphCutter(strategy: Strategy) { } } -object GraphCutter { +object GraphPartitioner { type Strategy = PartialFunction[Module, Location] @@ -146,22 +145,33 @@ object GraphCutter { case task: GearpumpTaskModule => Remote case groupBy: GroupByModule[_, _] => - // TODO: groupBy is not supported by local materializer yet + // TODO: groupBy is not supported by local materializer Remote case source: SourceModule[_, _] => Local case sink: SinkModule[_, _] => Local - case matValueSource: MaterializedValueSource[_] => - Local - case direct: DirectProcessor => - Local - case time: TimerTransform => - // Renders to local as it requires a timer. - Local + case remaining: Module => + remaining.shape match { + case sourceShape: SourceShape[_] => + Local + case sinkShape: SinkShape[_] => + Local + case otherShapes: Shape => + Remote + } } val AllRemoteStrategy: Strategy = BaseStrategy orElse { + case graphStageModule: GraphStageModule => + graphStageModule.stage match { + case matValueSource: MaterializedValueSource[_] => + Local + case singleSource: SingleSource[_] => + Local + case _ => + Remote + } case _ => Remote } @@ -177,7 +187,19 @@ object GraphCutter { } val AllLocalStrategy: Strategy = BaseStrategy orElse { + case graphStageModule: GraphStageModule => + // TODO kasravi review + graphStageModule.stage match { + case matValueSource: MaterializedValueSource[_] => + Local + case _ => + Local + } case _ => Local } -} \ No newline at end of file + + def apply(strategy: Strategy): GraphPartitioner = { + new GraphPartitioner(strategy) + } +} diff --git a/experiments/akkastream/src/main/scala/akka/stream/gearpump/graph/LocalGraph.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/LocalGraph.scala similarity index 55% rename from experiments/akkastream/src/main/scala/akka/stream/gearpump/graph/LocalGraph.scala rename to experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/LocalGraph.scala index 6ef8598ba..c03fce284 100644 --- a/experiments/akkastream/src/main/scala/akka/stream/gearpump/graph/LocalGraph.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/LocalGraph.scala @@ -7,7 +7,7 @@ * "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 + * 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, @@ -16,26 +16,25 @@ * limitations under the License. */ -package akka.stream.gearpump.graph +package org.apache.gearpump.akkastream.graph import akka.actor.ActorSystem -import akka.stream.ModuleGraph.Edge -import akka.stream.gearpump.materializer.LocalMaterializer -import akka.stream.gearpump.module.{SinkBridgeModule, SourceBridgeModule} import akka.stream.impl.Stages.DefaultAttributes import akka.stream.impl.StreamLayout.Module import akka.stream.impl.{PublisherSource, SubscriberSink} -import akka.stream.{Outlet, SinkShape, SourceShape} -import org.reactivestreams.{Publisher, Subscriber} - +import akka.stream.{SinkShape, SourceShape} +import org.apache.gearpump.akkastream.GearpumpMaterializer.Edge +import org.apache.gearpump.akkastream.materializer.LocalMaterializerImpl +import org.apache.gearpump.akkastream.module.{SinkBridgeModule, SourceBridgeModule} import org.apache.gearpump.util.Graph +import org.reactivestreams.{Publisher, Subscriber} /** * * [[LocalGraph]] is a [[SubGraph]] of the application DSL Graph, which only - * contain module that can be materialized in local JVM. + * contain module that can be materialized in local JVM. * - * @param graph + * @param graph Graph[Module, Edge] */ class LocalGraph(override val graph: Graph[Module, Edge]) extends SubGraph @@ -43,37 +42,37 @@ object LocalGraph { /** * materialize LocalGraph in local JVM - * @param system + * @param system ActorSystem */ class LocalGraphMaterializer(system: ActorSystem) extends SubGraphMaterializer { - // Creates a local materializer - val materializer = LocalMaterializer()(system) + // create a local materializer + val materializer = LocalMaterializerImpl()(system) /** * * @param matValues Materialized Values for each module before materialization * @return Materialized Values for each Module after the materialization. */ - override def materialize(graph: SubGraph, matValues: Map[Module, Any]): Map[Module, Any] = { - val newGraph: Graph[Module, Edge] = graph.graph.mapVertex { module => - module match { - case source: SourceBridgeModule[AnyRef, AnyRef] => - val subscriber = matValues(source).asInstanceOf[Subscriber[AnyRef]] - val shape = SinkShape(source.inPort) - new SubscriberSink(subscriber, DefaultAttributes.subscriberSink, shape) - case sink: SinkBridgeModule[AnyRef, AnyRef] => - val publisher = matValues(sink).asInstanceOf[Publisher[AnyRef]] - val shape = SourceShape(sink.outPort.asInstanceOf[Outlet[AnyRef]]) - new PublisherSource(publisher, DefaultAttributes.publisherSource, shape) - case other => - other - } + override def materialize(graph: SubGraph, + matValues: scala.collection.mutable.Map[Module, Any]): + scala.collection.mutable.Map[Module, Any] = { + val newGraph: Graph[Module, Edge] = graph.graph.mapVertex { + case source: SourceBridgeModule[in, out] => + val subscriber = matValues(source).asInstanceOf[Subscriber[in]] + val shape: SinkShape[in] = SinkShape(source.inPort) + new SubscriberSink(subscriber, DefaultAttributes.subscriberSink, shape) + case sink: SinkBridgeModule[in, out] => + val publisher = matValues(sink).asInstanceOf[Publisher[out]] + val shape: SourceShape[out] = SourceShape(sink.outPort) + new PublisherSource(publisher, DefaultAttributes.publisherSource, shape) + case other => + other } materializer.materialize(newGraph, matValues) } - override def shutdown(): Unit = { + override def shutdown: Unit = { materializer.shutdown() } } diff --git a/experiments/akkastream/src/main/scala/akka/stream/gearpump/graph/RemoteGraph.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/RemoteGraph.scala similarity index 63% rename from experiments/akkastream/src/main/scala/akka/stream/gearpump/graph/RemoteGraph.scala rename to experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/RemoteGraph.scala index 3cea78aeb..8fbe78531 100644 --- a/experiments/akkastream/src/main/scala/akka/stream/gearpump/graph/RemoteGraph.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/RemoteGraph.scala @@ -7,7 +7,7 @@ * "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 + * 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, @@ -16,16 +16,15 @@ * limitations under the License. */ -package akka.stream.gearpump.graph +package org.apache.gearpump.akkastream.graph import akka.actor.ActorSystem -import akka.stream.ModuleGraph.Edge -import akka.stream.gearpump.materializer.RemoteMaterializerImpl -import akka.stream.gearpump.module.{SinkBridgeModule, SourceBridgeModule} -import akka.stream.gearpump.task.SinkBridgeTask.SinkBridgeTaskClient -import akka.stream.gearpump.task.SourceBridgeTask.SourceBridgeTaskClient +import org.apache.gearpump.akkastream.GearpumpMaterializer.Edge +import org.apache.gearpump.akkastream.materializer.RemoteMaterializerImpl +import org.apache.gearpump.akkastream.module.{SinkBridgeModule, SourceBridgeModule} +import org.apache.gearpump.akkastream.task.SinkBridgeTask.SinkBridgeTaskClient +import org.apache.gearpump.akkastream.task.SourceBridgeTask.SourceBridgeTaskClient import akka.stream.impl.StreamLayout.Module - import org.apache.gearpump.cluster.client.ClientContext import org.apache.gearpump.cluster.embedded.EmbeddedCluster import org.apache.gearpump.streaming.ProcessorId @@ -34,9 +33,9 @@ import org.apache.gearpump.util.Graph /** * * [[RemoteGraph]] is a [[SubGraph]] of the application DSL Graph, which only - * contain modules that can be materialized in remote Gearpump cluster. + * contain modules that can be materialized in remote Gearpump cluster. * - * @param graph + * @param graph Graph */ class RemoteGraph(override val graph: Graph[Module, Edge]) extends SubGraph @@ -44,10 +43,11 @@ object RemoteGraph { /** * * materialize LocalGraph in remote gearpump cluster - * @param useInProcessCluster - * @param system + * @param useInProcessCluster Boolean + * @param system ActorSystem */ - class RemoteGraphMaterializer(useInProcessCluster: Boolean, system: ActorSystem) extends SubGraphMaterializer { + class RemoteGraphMaterializer(useInProcessCluster: Boolean, system: ActorSystem) + extends SubGraphMaterializer { private val local = if (useInProcessCluster) { val cluster = EmbeddedCluster() cluster.start() @@ -57,13 +57,15 @@ object RemoteGraph { } private val context: ClientContext = local match { - case Some(local) => local.newClientContext + case Some(l) => l.newClientContext case None => ClientContext(system) } - override def materialize(subGraph: SubGraph, inputMatValues: Map[Module, Any]): Map[Module, Any] = { + override def materialize(subGraph: SubGraph, + inputMatValues: scala.collection.mutable.Map[Module, Any]): + scala.collection.mutable.Map[Module, Any] = { val graph = subGraph.graph - + if (graph.isEmpty) { inputMatValues } else { @@ -71,22 +73,27 @@ object RemoteGraph { } } - private def doMaterialize(graph: Graph[Module, Edge], inputMatValues: Map[Module, Any]): Map[Module, Any] = { + private def doMaterialize(graph: Graph[Module, Edge], + inputMatValues: scala.collection.mutable.Map[Module, Any]): + scala.collection.mutable.Map[Module, Any] = { val materializer = new RemoteMaterializerImpl(graph, system) val (app, matValues) = materializer.materialize val appId = context.submit(app) - println("sleep 5 second until the applicaiton is ready on cluster") + // scalastyle:off println + println("sleep 5 second until the application is ready on cluster") + // scalastyle:on println Thread.sleep(5000) def resolve(matValues: Map[Module, ProcessorId]): Map[Module, Any] = { matValues.toList.flatMap { kv => val (module, processorId) = kv module match { - case source: SourceBridgeModule[AnyRef, AnyRef] => - val bridge = new SourceBridgeTaskClient[AnyRef](system.dispatcher, context, appId, processorId) + case source: SourceBridgeModule[_, _] => + val bridge = new SourceBridgeTaskClient[AnyRef](system.dispatcher, + context, appId, processorId) Some((module, bridge)) - case sink: SinkBridgeModule[AnyRef, AnyRef] => + case sink: SinkBridgeModule[_, _] => val bridge = new SinkBridgeTaskClient(system, context, appId, processorId) Some((module, bridge)) case other => @@ -98,9 +105,9 @@ object RemoteGraph { inputMatValues ++ resolve(matValues) } - override def shutdown(): Unit = { + override def shutdown: Unit = { context.close() - local.map(_.stop()) + local.foreach(_.stop()) } } } diff --git a/experiments/akkastream/src/main/scala/akka/stream/gearpump/graph/SubGraph.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/SubGraph.scala similarity index 74% rename from experiments/akkastream/src/main/scala/akka/stream/gearpump/graph/SubGraph.scala rename to experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/SubGraph.scala index 564b6c70c..a0395de7a 100644 --- a/experiments/akkastream/src/main/scala/akka/stream/gearpump/graph/SubGraph.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/SubGraph.scala @@ -7,7 +7,7 @@ * "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 + * 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, @@ -16,17 +16,17 @@ * limitations under the License. */ -package akka.stream.gearpump.graph +package org.apache.gearpump.akkastream.graph -import akka.stream.ModuleGraph.Edge +import akka.actor.ActorSystem +import org.apache.gearpump.akkastream.GearpumpMaterializer.Edge import akka.stream.impl.StreamLayout.Module - import org.apache.gearpump.util.Graph /** - * [[SubGraph]] is a partial part of [[akka.stream.ModuleGraph]] + * [[SubGraph]] is a partial DAG * - * The idea is that by dividing [[akka.stream.ModuleGraph]] to several + * The idea is that by dividing [[Graph]] to several * [[SubGraph]], we can materialize each [[SubGraph]] with different * materializer. */ @@ -40,6 +40,7 @@ trait SubGraph { def graph: Graph[Module, Edge] } + /** * Materializer for Sub-Graph type */ @@ -50,7 +51,9 @@ trait SubGraphMaterializer { * @return Materialized Values for each Module after the materialization. */ - def materialize(graph: SubGraph, matValues: Map[Module, Any]): Map[Module, Any] + def materialize(graph: SubGraph, + matValues: scala.collection.mutable.Map[Module, Any]): + scala.collection.mutable.Map[Module, Any] - def shutdown(): Unit -} \ No newline at end of file + def shutdown: Unit +} diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/materializer/LocalMaterializerImpl.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/materializer/LocalMaterializerImpl.scala new file mode 100644 index 000000000..cbafcf56b --- /dev/null +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/materializer/LocalMaterializerImpl.scala @@ -0,0 +1,319 @@ +/* + * 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.gearpump.akkastream.materializer + +import java.util.concurrent.atomic.AtomicBoolean +import java.{util => ju} + +import _root_.org.apache.gearpump.util.{Graph => GGraph} +import akka.NotUsed +import akka.actor.{ActorRef, ActorSystem, Cancellable, Deploy, PoisonPill} +import akka.dispatch.Dispatchers +import akka.event.{Logging, LoggingAdapter} +import akka.stream.impl.StreamLayout._ +import akka.stream.impl._ +import akka.stream.impl.fusing.GraphInterpreter.GraphAssembly +import akka.stream.impl.fusing.GraphStages.MaterializedValueSource +import akka.stream.impl.fusing.{Map => _, _} +import akka.stream.impl.io.{TLSActor, TlsModule} +import akka.stream.scaladsl.{GraphDSL, Keep, ModuleExtractor, RunnableGraph} +import akka.stream.{ClosedShape, Graph => AkkaGraph, _} +import org.apache.gearpump.akkastream.GearpumpMaterializer.Edge +import org.apache.gearpump.akkastream.module.ReduceModule +import org.apache.gearpump.akkastream.util.MaterializedValueOps +import org.reactivestreams.{Publisher, Subscriber} + +import scala.concurrent.ExecutionContextExecutor +import scala.concurrent.duration.FiniteDuration + +/** + * This materializer is functional equivalent to [[akka.stream.impl.ActorMaterializerImpl]] + * + * @param system System + * @param settings ActorMaterializerSettings + * @param dispatchers Dispatchers + * @param supervisor ActorRef + * @param haveShutDown AtomicBoolean + * @param flowNames SeqActorName + */ +case class LocalMaterializerImpl ( + override val system: ActorSystem, + override val settings: ActorMaterializerSettings, + dispatchers: Dispatchers, + override val supervisor: ActorRef, + haveShutDown: AtomicBoolean, + flowNames: SeqActorName) + extends ExtendedActorMaterializer { + + override def logger: LoggingAdapter = Logging.getLogger(system, this) + + override def schedulePeriodically(initialDelay: FiniteDuration, interval: FiniteDuration, + task: Runnable): Cancellable = + system.scheduler.schedule(initialDelay, interval, task)(executionContext) + + override def scheduleOnce(delay: FiniteDuration, task: Runnable): Cancellable = + system.scheduler.scheduleOnce(delay, task)(executionContext) + + override def effectiveSettings(opAttr: Attributes): ActorMaterializerSettings = { + import ActorAttributes._ + import Attributes._ + opAttr.attributeList.foldLeft(settings) { (s, attr) => + attr match { + case InputBuffer(initial, max) => s.withInputBuffer(initial, max) + case Dispatcher(dispatcher) => s.withDispatcher(dispatcher) + case SupervisionStrategy(decider) => s.withSupervisionStrategy(decider) + case l: LogLevels => s + case Name(_) => s + case other => s + } + } + } + + override def shutdown(): Unit = + if (haveShutDown.compareAndSet(false, true)) supervisor ! PoisonPill + + override def isShutdown: Boolean = haveShutDown.get() + + override lazy val executionContext: ExecutionContextExecutor = + dispatchers.lookup(settings.dispatcher match { + case Deploy.NoDispatcherGiven => Dispatchers.DefaultDispatcherId + case other => other + }) + + + case class LocalMaterializerSession(module: Module, iAttributes: Attributes, + subflowFuser: GraphInterpreterShell => ActorRef = null) + extends MaterializerSession(module, iAttributes) { + + override def materializeAtomic(atomic: AtomicModule, + effectiveAttributes: Attributes, matVal: ju.Map[Module, Any]): Unit = { + + def newMaterializationContext() = + new MaterializationContext(LocalMaterializerImpl.this, effectiveAttributes, + stageName(effectiveAttributes)) + atomic match { + case sink: SinkModule[_, _] => + val (sub, mat) = sink.create(newMaterializationContext()) + assignPort(sink.shape.in, sub.asInstanceOf[Subscriber[Any]]) + matVal.put(atomic, mat) + case source: SourceModule[_, _] => + val (pub, mat) = source.create(newMaterializationContext()) + assignPort(source.shape.out, pub.asInstanceOf[Publisher[Any]]) + matVal.put(atomic, mat) + case stage: ProcessorModule[_, _, _] => + val (processor, mat) = stage.createProcessor() + assignPort(stage.inPort, processor) + assignPort(stage.outPort, processor.asInstanceOf[Publisher[Any]]) + matVal.put(atomic, mat) + case tls: TlsModule => // TODO solve this so TlsModule doesn't need special treatment here + val es = effectiveSettings(effectiveAttributes) + val props = + TLSActor.props(es, tls.sslContext, tls.sslConfig, + tls.firstSession, tls.role, tls.closing, tls.hostInfo) + val impl = actorOf(props, stageName(effectiveAttributes), es.dispatcher) + def factory(id: Int) = new ActorPublisher[Any](impl) { + override val wakeUpMsg = FanOut.SubstreamSubscribePending(id) + } + val publishers = Vector.tabulate(2)(factory) + impl ! FanOut.ExposedPublishers(publishers) + + assignPort(tls.plainOut, publishers(TLSActor.UserOut)) + assignPort(tls.cipherOut, publishers(TLSActor.TransportOut)) + + assignPort(tls.plainIn, FanIn.SubInput[Any](impl, TLSActor.UserIn)) + assignPort(tls.cipherIn, FanIn.SubInput[Any](impl, TLSActor.TransportIn)) + + matVal.put(atomic, NotUsed) + case graph: GraphModule => + matGraph(graph, effectiveAttributes, matVal) + case stage: GraphStageModule => + val graph = + GraphModule(GraphAssembly(stage.shape.inlets, stage.shape.outlets, stage.stage), + stage.shape, stage.attributes, Array(stage)) + matGraph(graph, effectiveAttributes, matVal) + } + } + + private def matGraph(graph: GraphModule, effectiveAttributes: Attributes, + matVal: ju.Map[Module, Any]): Unit = { + val calculatedSettings = effectiveSettings(effectiveAttributes) + val (handlers, logics) = + graph.assembly.materialize(effectiveAttributes, graph.matValIDs, matVal, registerSrc) + + val shell = new GraphInterpreterShell(graph.assembly, handlers, + logics, graph.shape, calculatedSettings, LocalMaterializerImpl.this) + + val impl = + if (subflowFuser != null && !effectiveAttributes.contains(Attributes.AsyncBoundary)) { + subflowFuser(shell) + } else { + val props = ActorGraphInterpreter.props(shell) + actorOf(props, stageName(effectiveAttributes), calculatedSettings.dispatcher) + } + + for ((inlet, i) <- graph.shape.inlets.iterator.zipWithIndex) { + val subscriber = new ActorGraphInterpreter.BoundarySubscriber(impl, shell, i) + assignPort(inlet, subscriber) + } + for ((outlet, i) <- graph.shape.outlets.iterator.zipWithIndex) { + val publisher = new ActorGraphInterpreter.BoundaryPublisher(impl, shell, i) + impl ! ActorGraphInterpreter.ExposedPublisher(shell, i, publisher) + assignPort(outlet, publisher) + } + } + } + + override def materialize[Mat](runnableGraph: AkkaGraph[ClosedShape, Mat]): Mat = { + + LocalMaterializerSession(ModuleExtractor.unapply(runnableGraph).get, + null, null).materialize().asInstanceOf[Mat] + + } + + override def materialize[Mat](runnableGraph: AkkaGraph[ClosedShape, Mat], + subflowFuser: GraphInterpreterShell => ActorRef): Mat = { + + LocalMaterializerSession(ModuleExtractor.unapply(runnableGraph).get, + null, null).materialize().asInstanceOf[Mat] + + } + + def buildToplevelModule(graph: GGraph[Module, Edge]): Module = { + var moduleInProgress: Module = EmptyModule + graph.vertices.foreach(module => { + moduleInProgress = moduleInProgress.compose(module) + }) + graph.edges.foreach(value => { + val (node1, edge, node2) = value + moduleInProgress = moduleInProgress.wire(edge.from, edge.to) + }) + + moduleInProgress + } + + def materialize(graph: GGraph[Module, Edge], + inputMatValues: scala.collection.mutable.Map[Module, Any]): + scala.collection.mutable.Map[Module, Any] = { + val topLevelModule = buildToplevelModule(graph) + val session = LocalMaterializerSession(topLevelModule, null, null) + import scala.collection.JavaConverters._ + val matV = inputMatValues.asJava + val materializedGraph = graph.mapVertex { module => + session.materializeAtomic(module.asInstanceOf[AtomicModule], module.attributes, matV) + matV.get(module) + } + materializedGraph.edges.foreach { nodeEdgeNode => + val (node1, edge, node2) = nodeEdgeNode + val from = edge.from + val to = edge.to + node1 match { + case module1: Module => + node2 match { + case module2: Module => + val publisher = module1.downstreams(from).asInstanceOf[Publisher[Any]] + val subscriber = module2.upstreams(to).asInstanceOf[Subscriber[Any]] + publisher.subscribe(subscriber) + case _ => + } + case _ => + } + } + val matValSources = graph.vertices.flatMap(module => { + val rt: Option[MaterializedValueSource[_]] = module match { + case graphStage: GraphStageModule => + graphStage.stage match { + case materializedValueSource: MaterializedValueSource[_] => + Some(materializedValueSource) + case _ => + None + } + case _ => + None + } + rt + }) + publishToMaterializedValueSource(matValSources, inputMatValues) + inputMatValues + } + + private def publishToMaterializedValueSource(modules: List[MaterializedValueSource[_]], + matValues: scala.collection.mutable.Map[Module, Any]): Unit = { + modules.foreach { source => + Option(source.computation).map { attr => + val valueToPublish = MaterializedValueOps(attr).resolve(matValues) + source.setValue(valueToPublish) + } + } + } + + private[this] def createFlowName(): String = flowNames.next() + + val flowName = createFlowName() + var nextId = 0 + + def stageName(attr: Attributes): String = { + val name = s"$flowName-$nextId-${attr.nameOrDefault()}" + nextId += 1 + name + } + + override def withNamePrefix(name: String): LocalMaterializerImpl = + this.copy(flowNames = flowNames.copy(name)) + +} + +object LocalMaterializerImpl { + case class MaterializedModule(module: Module, matValue: Any, + inputs: Map[InPort, Subscriber[_]] = Map.empty[InPort, Subscriber[_]], + outputs: Map[OutPort, Publisher[_]] = Map.empty[OutPort, Publisher[_]]) + + def apply(materializerSettings: Option[ActorMaterializerSettings] = None, + namePrefix: Option[String] = None)(implicit system: ActorSystem): + LocalMaterializerImpl = { + + val settings = materializerSettings getOrElse ActorMaterializerSettings(system) + apply(settings, namePrefix.getOrElse("flow"))(system) + } + + def apply(materializerSettings: ActorMaterializerSettings, + namePrefix: String)(implicit system: ActorSystem): LocalMaterializerImpl = { + val haveShutDown = new AtomicBoolean(false) + + new LocalMaterializerImpl( + system, + materializerSettings, + system.dispatchers, + system.actorOf(StreamSupervisor.props(materializerSettings, + haveShutDown).withDispatcher(materializerSettings.dispatcher)), + haveShutDown, + FlowNames(system).name.copy(namePrefix)) + } + + def toFoldModule(reduce: ReduceModule[Any]): Fold[Any, Any] = { + val f = reduce.f + val aggregator = {(zero: Any, input: Any) => + if (zero == null) { + input + } else { + f(zero, input) + } + } + Fold(null, aggregator) + } +} diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/materializer/RemoteMaterializerImpl.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/materializer/RemoteMaterializerImpl.scala new file mode 100644 index 000000000..f3f80947a --- /dev/null +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/materializer/RemoteMaterializerImpl.scala @@ -0,0 +1,594 @@ +/* + * 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.gearpump.akkastream.materializer + +import akka.actor.ActorSystem +import akka.stream.impl.StreamLayout.Module +import akka.stream.impl.Timers._ +import akka.stream.impl.fusing.GraphStages.{MaterializedValueSource, SimpleLinearGraphStage, SingleSource, TickSource} +import akka.stream.impl.fusing.{Map => FMap, _} +import akka.stream.impl.io.IncomingConnectionStage +import akka.stream.impl.{HeadOptionStage, Stages, Throttle} +import akka.stream.scaladsl._ +import akka.stream.stage.AbstractStage.PushPullGraphStageWithMaterializedValue +import akka.stream.stage.GraphStage +import akka.stream.{FanInShape, FanOutShape} +import org.apache.gearpump.akkastream.GearAttributes +import org.apache.gearpump.akkastream.GearpumpMaterializer.Edge +import org.apache.gearpump.akkastream.module._ +import org.apache.gearpump.akkastream.task._ +import org.apache.gearpump.cluster.UserConfig +import org.apache.gearpump.streaming.dsl.StreamApp +import org.apache.gearpump.streaming.dsl.op._ +import org.apache.gearpump.streaming.{ProcessorId, StreamApplication} +import org.apache.gearpump.util.Graph +import org.slf4j.LoggerFactory + +import scala.concurrent.Promise +import scala.concurrent.duration.FiniteDuration + +/** + * [[RemoteMaterializerImpl]] will materialize the [[Graph[Module, Edge]] to a Gearpump + * Streaming Application. + * + * @param graph Graph + * @param system ActorSystem + */ +class RemoteMaterializerImpl(graph: Graph[Module, Edge], system: ActorSystem) { + + import RemoteMaterializerImpl._ + + type ID = String + private implicit val actorSystem = system + + private def uuid: String = { + java.util.UUID.randomUUID.toString + } + + def materialize: (StreamApplication, Map[Module, ProcessorId]) = { + val (opGraph, ids) = toOpGraph + val app: StreamApplication = new StreamApp("app", system, UserConfig.empty, opGraph) + val processorIds = resolveIds(app, ids) + + val updatedApp = updateJunctionConfig(processorIds, app) + (removeIds(updatedApp), processorIds) + } + + private def updateJunctionConfig(processorIds: Map[Module, ProcessorId], + app: StreamApplication): StreamApplication = { + val config = junctionConfig(processorIds) + + val dag = app.dag.mapVertex { vertex => + val processorId = vertex.id + val newConf = vertex.taskConf.withConfig(config(processorId)) + vertex.copy(taskConf = newConf) + } + new StreamApplication(app.name, app.inputUserConfig, dag) + } + + private def junctionConfig(processorIds: Map[Module, ProcessorId]): + Map[ProcessorId, UserConfig] = { + val updatedConfigs = graph.vertices.flatMap { vertex => + buildShape(vertex, processorIds) + }.toMap + updatedConfigs + } + + private def buildShape(vertex: Module, processorIds: Map[Module, ProcessorId]): + Option[(ProcessorId, UserConfig)] = { + def inProcessors(vertex: Module): List[ProcessorId] = { + vertex.shape.inlets.flatMap { inlet => + graph.incomingEdgesOf(vertex).find( + _._2.to == inlet).map(_._1 + ).flatMap(processorIds.get(_)) + }.toList + } + def outProcessors(vertex: Module): List[ProcessorId] = { + vertex.shape.outlets.flatMap { outlet => + graph.outgoingEdgesOf(vertex).find( + _._2.from == outlet).map(_._3 + ).flatMap(processorIds.get(_)) + }.toList + } + processorIds.get(vertex).map(processorId => { + (processorId, UserConfig.empty. + withValue(GraphTask.OUT_PROCESSORS, outProcessors(vertex)). + withValue(GraphTask.IN_PROCESSORS, inProcessors(vertex))) + }) + } + + private def resolveIds(app: StreamApplication, ids: Map[Module, ID]): + Map[Module, ProcessorId] = { + ids.flatMap { kv => + val (module, id) = kv + val processorId = app.dag.vertices.find { processor => + processor.taskConf.getString(id).isDefined + }.map(_.id) + processorId.map((module, _)) + } + } + + private def removeIds(app: StreamApplication): StreamApplication = { + val graph = app.dag.mapVertex { processor => + val conf = removeId(processor.taskConf) + processor.copy(taskConf = conf) + } + new StreamApplication(app.name, app.inputUserConfig, graph) + } + + private def removeId(conf: UserConfig): UserConfig = { + conf.filter { kv => + kv._2 != RemoteMaterializerImpl.TRACKABLE + } + } + + private def toOpGraph: (Graph[Op, OpEdge], Map[Module, ID]) = { + var matValues = collection.mutable.Map.empty[Module, ID] + val opGraph = graph.mapVertex[Op] { module => + val name = uuid + val conf = UserConfig.empty.withString(name, RemoteMaterializerImpl.TRACKABLE) + matValues += module -> name + val parallelism = GearAttributes.count(module.attributes) + val op = module match { + case source: SourceTaskModule[_] => + val updatedConf = conf.withConfig(source.conf) + DataSourceOp(source.source, parallelism, updatedConf, "source") + case sink: SinkTaskModule[_] => + val updatedConf = conf.withConfig(sink.conf) + DataSinkOp(sink.sink, parallelism, updatedConf, "sink") + case sourceBridge: SourceBridgeModule[_, _] => + ProcessorOp(classOf[SourceBridgeTask], parallelism = 1, conf, "source") + case processor: ProcessorModule[_, _, _] => + val updatedConf = conf.withConfig(processor.conf) + ProcessorOp(processor.processor, parallelism, updatedConf, "source") + case sinkBridge: SinkBridgeModule[_, _] => + ProcessorOp(classOf[SinkBridgeTask], parallelism, conf, "sink") + case groupBy: GroupByModule[_, _] => + GroupByOp(groupBy.groupBy, parallelism, "groupBy", conf) + case reduce: ReduceModule[_] => + reduceOp(reduce.f, conf) + case graphStage: GraphStageModule => + translateGraphStageWithMaterializedValue(graphStage, parallelism, conf) + } + if (op == null) { + throw new UnsupportedOperationException( + module.getClass.toString + " is not supported with RemoteMaterializer" + ) + } + op + }.mapEdge[OpEdge] { (n1, edge, n2) => + n2 match { + case master: MasterOp => + Shuffle + case slave: SlaveOp[_] if n1.isInstanceOf[ProcessorOp[_]] => + Shuffle + case slave: SlaveOp[_] => + Direct + } + } + (opGraph, matValues.toMap) + } + + private def translateGraphStageWithMaterializedValue(module: GraphStageModule, + parallelism: Int, conf: UserConfig): Op = { + module.stage match { + case tickSource: TickSource[_] => + import TickSourceTask._ + val tick: AnyRef = tickSource.tick.asInstanceOf[AnyRef] + val tiConf = conf.withValue[FiniteDuration](INITIAL_DELAY, tickSource.initialDelay). + withValue[FiniteDuration](INTERVAL, tickSource.interval). + withValue(TICK, tick) + ProcessorOp(classOf[TickSourceTask[_]], parallelism, tiConf, "tickSource") + case graphStage: GraphStage[_] => + translateGraphStage(module, parallelism, conf) + case headOptionStage: HeadOptionStage[_] => + headOptionOp(headOptionStage, conf) + case pushPullGraphStageWithMaterializedValue: + PushPullGraphStageWithMaterializedValue[_, _, _, _] => + translateSymbolic(pushPullGraphStageWithMaterializedValue, conf) + } + } + + private def translateGraphStage(module: GraphStageModule, + parallelism: Int, conf: UserConfig): Op = { + module.stage match { + case balance: Balance[_] => + ProcessorOp(classOf[BalanceTask], parallelism, conf, "balance") + case batch: Batch[_, _] => + val batchConf = conf.withValue[_ => Long](BatchTask.COST, batch.costFn). + withLong(BatchTask.MAX, batch.max). + withValue[(_, _) => _](BatchTask.AGGREGATE, batch.aggregate). + withValue[_ => _](BatchTask.SEED, batch.seed) + ProcessorOp(classOf[BatchTask[_, _]], + parallelism, batchConf, "batch") + case broadcast: Broadcast[_] => + val name = ModuleExtractor.unapply(broadcast).map(_.attributes.nameOrDefault()).get + ProcessorOp(classOf[BroadcastTask], parallelism, conf, name) + case collect: Collect[_, _] => + collectOp(collect.pf, conf) + case concat: Concat[_] => + ProcessorOp(classOf[ConcatTask], parallelism, conf, "concat") + case delayInitial: DelayInitial[_] => + val dIConf = conf.withValue[FiniteDuration]( + DelayInitialTask.DELAY_INITIAL, delayInitial.delay) + ProcessorOp(classOf[DelayInitialTask[_]], parallelism, dIConf, "delayInitial") + case dropWhile: DropWhile[_] => + dropWhileOp(dropWhile.p, conf) + case flattenMerge: FlattenMerge[_, _] => + ProcessorOp(classOf[FlattenMergeTask], parallelism, conf, "flattenMerge") + case fold: Fold[_, _] => + val foldConf = conf.withValue(FoldTask.ZERO, fold.zero.asInstanceOf[AnyRef]). + withValue(FoldTask.AGGREGATOR, fold.f) + ProcessorOp(classOf[FoldTask[_, _]], parallelism, foldConf, "fold") + case groupBy: GroupBy[_, _] => + GroupByOp(groupBy.keyFor, groupBy.maxSubstreams, "groupBy", conf) + case groupedWithin: GroupedWithin[_] => + val diConf = conf.withValue[FiniteDuration](GroupedWithinTask.TIME_WINDOW, groupedWithin.d). + withInt(GroupedWithinTask.BATCH_SIZE, groupedWithin.n) + ProcessorOp(classOf[GroupedWithinTask[_]], parallelism, diConf, "groupedWithin") + case idleInject: IdleInject[_, _] => + // TODO + null + case idleTimeoutBidi: IdleTimeoutBidi[_, _] => + // TODO + null + case incomingConnectionStage: IncomingConnectionStage => + // TODO + null + case interleave: Interleave[_] => + val ilConf = conf.withInt(InterleaveTask.INPUT_PORTS, interleave.inputPorts). + withInt(InterleaveTask.SEGMENT_SIZE, interleave.segmentSize) + ProcessorOp(classOf[InterleaveTask], parallelism, ilConf, "interleave") + null + case intersperse: Intersperse[_] => + // TODO + null + case limitWeighted: LimitWeighted[_] => + // TODO + null + case map: FMap[_, _] => + mapOp(map.f, conf) + case mapAsync: MapAsync[_, _] => + ProcessorOp(classOf[MapAsyncTask[_, _]], + mapAsync.parallelism, conf.withValue(MapAsyncTask.MAPASYNC_FUNC, mapAsync.f), "mapAsync") + case mapAsyncUnordered: MapAsyncUnordered[_, _] => + ProcessorOp(classOf[MapAsyncTask[_, _]], + mapAsyncUnordered.parallelism, + conf.withValue(MapAsyncTask.MAPASYNC_FUNC, mapAsyncUnordered.f), "mapAsyncUnordered") + case materializedValueSource: MaterializedValueSource[_] => + // TODO + null + case merge: Merge[_] => + val mergeConf = conf.withBoolean(MergeTask.EAGER_COMPLETE, merge.eagerComplete). + withInt(MergeTask.INPUT_PORTS, merge.inputPorts) + ProcessorOp(classOf[MergeTask], parallelism, mergeConf, "merge") + case mergePreferred: MergePreferred[_] => + MergeOp("mergePreferred", conf) + case mergeSorted: MergeSorted[_] => + MergeOp("mergeSorted", conf) + case prefixAndTail: PrefixAndTail[_] => + // TODO + null + case recover: Recover[_] => + // TODO + null + case scan: Scan[_, _] => + scanOp(scan.zero, scan.f, conf) + case simpleLinearGraphStage: SimpleLinearGraphStage[_] => + translateSimpleLinearGraph(simpleLinearGraphStage, parallelism, conf) + case singleSource: SingleSource[_] => + val singleSourceConf = conf.withValue[AnyRef](SingleSourceTask.ELEMENT, + singleSource.elem.asInstanceOf[AnyRef]) + ProcessorOp(classOf[SingleSourceTask[_]], parallelism, singleSourceConf, "singleSource") + case split: Split[_] => + // TODO + null + case statefulMapConcat: StatefulMapConcat[_, _] => + val func = statefulMapConcat.f + val statefulMapConf = + conf.withValue[() => _ => Iterable[_]](StatefulMapConcatTask.FUNC, func) + ProcessorOp(classOf[StatefulMapConcatTask[_, _]], parallelism, + statefulMapConf, "statefulMapConcat") + case subSink: SubSink[_] => + // TODO + null + case subSource: SubSource[_] => + // TODO + null + case unfold: Unfold[_, _] => + // TODO + null + case unfoldAsync: UnfoldAsync[_, _] => + // TODO + null + case unzip: Unzip[_, _] => + ProcessorOp(classOf[Unzip2Task[_, _, _]], + parallelism, + conf.withValue( + Unzip2Task.UNZIP2_FUNCTION, Unzip2Task.UnZipFunction(unzip.unzipper) + ), "unzip") + case zip: Zip[_, _] => + zipWithOp(zip.zipper, conf) + case zipWith2: ZipWith2[_, _, _] => + ProcessorOp(classOf[Zip2Task[_, _, _]], + parallelism, + conf.withValue( + Zip2Task.ZIP2_FUNCTION, Zip2Task.ZipFunction(zipWith2.zipper) + ), "zipWith2") + } + } + + private def translateSimpleLinearGraph(stage: SimpleLinearGraphStage[_], + parallelism: Int, conf: UserConfig): Op = { + stage match { + case completion: Completion[_] => + // TODO + null + case delay: Delay[_] => + // TODO + null + case drop: Drop[_] => + dropOp(drop.count, conf) + case dropWithin: DropWithin[_] => + val dropWithinConf = + conf.withValue[FiniteDuration](DropWithinTask.TIMEOUT, dropWithin.timeout) + ProcessorOp(classOf[DropWithinTask[_]], + parallelism, dropWithinConf, "dropWithin") + case filter: Filter[_] => + filterOp(filter.p, conf) + case idle: Idle[_] => + // TODO + null + case initial: Initial[_] => + // TODO + null + case log: Log[_] => + logOp(log.name, log.extract, conf) + case reduce: Reduce[_] => + reduceOp(reduce.f, conf) + case take: Take[_] => + takeOp(take.count, conf) + case takeWhile: TakeWhile[_] => + filterOp(takeWhile.p, conf) + case takeWithin: TakeWithin[_] => + val takeWithinConf = + conf.withValue[FiniteDuration](TakeWithinTask.TIMEOUT, takeWithin.timeout) + ProcessorOp(classOf[TakeWithinTask[_]], + parallelism, takeWithinConf, "takeWithin") + case throttle: Throttle[_] => + val throttleConf = conf.withInt(ThrottleTask.COST, throttle.cost). + withInt(ThrottleTask.MAX_BURST, throttle.maximumBurst). + withValue[_ => Int](ThrottleTask.COST_CALC, throttle.costCalculation). + withValue[FiniteDuration](ThrottleTask.TIME_PERIOD, throttle.per) + ProcessorOp(classOf[ThrottleTask[_]], + parallelism, throttleConf, "throttle") + } + } + + private def translateSymbolic(stage: PushPullGraphStageWithMaterializedValue[_, _, _, _], + conf: UserConfig): Op = { + stage match { + case symbolicGraphStage: Stages.SymbolicGraphStage[_, _, _] => + symbolicGraphStage.symbolicStage match { + case buffer: Stages.Buffer[_] => + // ignore the buffering operation + identity("buffer", conf) + } + } + } + +} + +object RemoteMaterializerImpl { + final val NotApplied: Any => Any = _ => NotApplied + + def collectOp[In, Out](collect: PartialFunction[In, Out], conf: UserConfig): Op = { + flatMapOp({ data: In => + collect.applyOrElse(data, NotApplied) match { + case NotApplied => None + case result: Any => Option(result) + } + }, "collect", conf) + } + + def filterOp[In](filter: In => Boolean, conf: UserConfig): Op = { + flatMapOp({ data: In => + if (filter(data)) Option(data) else None + }, "filter", conf) + } + + def headOptionOp[T](headOptionStage: HeadOptionStage[T], conf: UserConfig): Op = { + val promise: Promise[Option[T]] = Promise() + flatMapOp({ data: T => + data match { + case None => + Some(promise.future.failed) + case Some(d) => + promise.future.value + } + }, "headOption", conf) + } + + def reduceOp[T](reduce: (T, T) => T, conf: UserConfig): Op = { + var result: Option[T] = None + val flatMap = { elem: T => + result match { + case None => + result = Some(elem) + case Some(r) => + result = Some(reduce(r, elem)) + } + List(result) + } + flatMapOp(flatMap, "reduce", conf) + } + + def zipWithOp[In1, In2](zipWith: (In1, In2) => (In1, In2), conf: UserConfig): Op = { + val flatMap = { elem: (In1, In2) => + val (e1, e2) = elem + val result: (In1, In2) = zipWith(e1, e2) + List(result) + } + flatMapOp(flatMap, "zipWith", conf) + } + + def zipWithOp2[In1, In2, Out](zipWith: (In1, In2) => Out, conf: UserConfig): Op = { + val flatMap = { elem: (In1, In2) => + val (e1, e2) = elem + val result: Out = zipWith(e1, e2) + List(result) + } + flatMapOp(flatMap, "zipWith", conf) + } + + def identity(description: String, conf: UserConfig): Op = { + flatMapOp({ data: Any => + List(data) + }, description, conf) + } + + def mapOp[In, Out](map: In => Out, conf: UserConfig): Op = { + val flatMap = (data: In) => List(map(data)) + flatMapOp (flatMap, conf) + } + + def flatMapOp[In, Out](flatMap: In => Iterable[Out], conf: UserConfig): Op = { + flatMapOp(flatMap, "flatmap", conf) + } + + def flatMapOp[In, Out](fun: In => TraversableOnce[Out], description: String, + conf: UserConfig): Op = { + FlatMapOp(fun, description, conf) + } + + def conflatOp[In, Out](seed: In => Out, aggregate: (Out, In) => Out, + conf: UserConfig): Op = { + var agg = None: Option[Out] + val flatMap = {elem: In => + agg = agg match { + case None => + Some(seed(elem)) + case Some(value) => + Some(aggregate(value, elem)) + } + List(agg.get) + } + flatMapOp (flatMap, "conflat", conf) + } + + def foldOp[In, Out](zero: Out, fold: (Out, In) => Out, conf: UserConfig): Op = { + var aggregator: Out = zero + val map = { elem: In => + aggregator = fold(aggregator, elem) + List(aggregator) + } + flatMapOp(map, "fold", conf) + } + + def groupedOp(count: Int, conf: UserConfig): Op = { + var left = count + val buf = { + val b = Vector.newBuilder[Any] + b.sizeHint(count) + b + } + + val flatMap: Any => Iterable[Any] = {input: Any => + buf += input + left -= 1 + if (left == 0) { + val emit = buf.result() + buf.clear() + left = count + Some(emit) + } else { + None + } + } + flatMapOp(flatMap, conf: UserConfig) + } + + def dropOp[T](number: Long, conf: UserConfig): Op = { + var left = number + val flatMap: T => Iterable[T] = {input: T => + if (left > 0) { + left -= 1 + None + } else { + Some(input) + } + } + flatMapOp(flatMap, "drop", conf) + } + + def dropWhileOp[In](drop: In => Boolean, conf: UserConfig): Op = { + flatMapOp({ data: In => + if (drop(data)) None else Option(data) + }, "dropWhile", conf) + } + + def logOp[T](name: String, extract: T => Any, conf: UserConfig): Op = { + val flatMap = {elem: T => + LoggerFactory.getLogger(name).info(s"Element: {${extract(elem)}}") + List(elem) + } + flatMapOp(flatMap, "log", conf) + } + + def scanOp[In, Out](zero: Out, f: (Out, In) => Out, conf: UserConfig): Op = { + var aggregator = zero + var pushedZero = false + + val flatMap = {elem: In => + aggregator = f(aggregator, elem) + + if (pushedZero) { + pushedZero = true + List(zero, aggregator) + } else { + List(aggregator) + } + } + flatMapOp(flatMap, "scan", conf) + } + + def statefulMapOp[In, Out](f: In => Iterable[Out], conf: UserConfig): Op = { + flatMapOp ({ data: In => + f(data) + }, conf) + } + + def takeOp(count: Long, conf: UserConfig): Op = { + var left: Long = count + + val filter: Any => Iterable[Any] = {elem: Any => + left -= 1 + if (left > 0) Some(elem) + else if (left == 0) Some(elem) + else None + } + flatMapOp(filter, "take", conf) + } + + /** + * We use this attribute to track module to Processor + * + */ + val TRACKABLE = "track how module is fused to processor" +} diff --git a/experiments/akkastream/src/main/scala/akka/stream/gearpump/module/BridgeModule.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/BridgeModule.scala similarity index 52% rename from experiments/akkastream/src/main/scala/akka/stream/gearpump/module/BridgeModule.scala rename to experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/BridgeModule.scala index c5dfc9a14..35d0e88dc 100644 --- a/experiments/akkastream/src/main/scala/akka/stream/gearpump/module/BridgeModule.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/BridgeModule.scala @@ -7,7 +7,7 @@ * "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 + * 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, @@ -16,20 +16,19 @@ * limitations under the License. */ -package akka.stream.gearpump.module +package org.apache.gearpump.akkastream.module -import akka.stream.Attributes -import akka.stream.impl.FlowModule -import akka.stream.impl.StreamLayout.Module +import akka.stream._ +import akka.stream.impl.StreamLayout.{AtomicModule, Module} import org.reactivestreams.{Publisher, Subscriber} /** * * - * [[IN]] -> [[BridgeModule]] -> [[OUT]] - * / - * / - * out of band data input or output channel [[MAT]] + * [[IN]] -> [[BridgeModule]] -> [[OUT]] + * / + * / + * out of band data input or output channel [[MAT]] * * * [[BridgeModule]] is used as a bridge between different materializers. @@ -38,18 +37,18 @@ import org.reactivestreams.{Publisher, Subscriber} * * For example: * - * Remote Materializer - * ----------------------------- - * | | - * | BridgeModule -> RemoteSink | - * | / | - * --/---------------------------- - * Local Materializer / out of band channel. - * ----------------------/---- - * | Local / | - * | Source -> BridgeModule | - * | | - * --------------------------- + * Remote Materializer + * ----------------------------- + * | | + * | BridgeModule -> RemoteSink | + * | / | + * --/---------------------------- + * Local Materializer / out of band channel. + * ----------------------/---- + * | Local / | + * | Source -> BridgeModule | + * | | + * --------------------------- * * * Typically [[BridgeModule]] is created implicitly as a temporary intermediate @@ -59,12 +58,21 @@ import org.reactivestreams.{Publisher, Subscriber} * boundary Source or Sink module which accept out of band channel inputs or * outputs. * - * - * @tparam IN - * @tparam OUT - * @tparam MAT + * @tparam IN input + * @tparam OUT output + * @tparam MAT materializer */ -abstract class BridgeModule[IN, OUT, MAT] extends FlowModule[IN, OUT, MAT] { +abstract class BridgeModule[IN, OUT, MAT] extends AtomicModule { + val inPort = Inlet[IN]("BridgeModule.in") + val outPort = Outlet[OUT]("BridgeModule.out") + override val shape = new FlowShape(inPort, outPort) + + override def replaceShape(s: Shape): Module = if (s != shape) { + throw new UnsupportedOperationException("cannot replace the shape of a FlowModule") + } else { + this + } + def attributes: Attributes def withAttributes(attributes: Attributes): BridgeModule[IN, OUT, MAT] @@ -72,28 +80,30 @@ abstract class BridgeModule[IN, OUT, MAT] extends FlowModule[IN, OUT, MAT] { override def carbonCopy: Module = newInstance } + /** * * Bridge module which accept out of band channel Input * [[org.reactivestreams.Publisher]][IN]. * * - * [[SourceBridgeModule]] -> [[OUT]] - * /| - * / - * out of band data input [[org.reactivestreams.Publisher]][IN] + * [[SourceBridgeModule]] -> [[OUT]] + * /| + * / + * out of band data input [[org.reactivestreams.Publisher]][IN] * * @see [[BridgeModule]] - * - * @param attributes + * @param attributes Attributes * @tparam IN, input data type from out of band [[org.reactivestreams.Publisher]] * @tparam OUT out put data type to next module. */ -class SourceBridgeModule[IN, OUT](val attributes: Attributes = Attributes.name("sourceBridgeModule")) extends BridgeModule[IN, OUT, Subscriber[IN]] { - override protected def newInstance: BridgeModule[IN, OUT, Subscriber[IN]] = new SourceBridgeModule[IN, OUT](attributes) +class SourceBridgeModule[IN, OUT](val attributes: Attributes = + Attributes.name("sourceBridgeModule")) extends BridgeModule[IN, OUT, Subscriber[IN]] { + override protected def newInstance: BridgeModule[IN, OUT, Subscriber[IN]] = + new SourceBridgeModule[IN, OUT](attributes) override def withAttributes(attributes: Attributes): BridgeModule[IN, OUT, Subscriber[IN]] = { - new SourceBridgeModule(attributes) + new SourceBridgeModule( attributes) } } @@ -103,22 +113,23 @@ class SourceBridgeModule[IN, OUT](val attributes: Attributes = Attributes.name(" * [[org.reactivestreams.Subscriber]][OUT]. * * - * [[IN]] -> [[BridgeModule]] - * \ - * \ - * \| - * out of band data output [[org.reactivestreams.Subscriber]][OUT] + * [[IN]] -> [[BridgeModule]] + * \ + * \ + * \| + * out of band data output [[org.reactivestreams.Subscriber]][OUT] * * @see [[BridgeModule]] - * - * @param attributes + * @param attributes Attributes * @tparam IN, input data type from previous module * @tparam OUT out put data type to out of band subscriber */ -class SinkBridgeModule[IN, OUT](val attributes: Attributes = Attributes.name("sourceBridgeModule")) extends BridgeModule[IN, OUT, Publisher[OUT]] { - override protected def newInstance: BridgeModule[IN, OUT, Publisher[OUT]] = new SinkBridgeModule[IN, OUT](attributes) +class SinkBridgeModule[IN, OUT](val attributes: Attributes = + Attributes.name("sinkBridgeModule")) extends BridgeModule[IN, OUT, Publisher[OUT]] { + override protected def newInstance: BridgeModule[IN, OUT, Publisher[OUT]] = + new SinkBridgeModule[IN, OUT](attributes) override def withAttributes(attributes: Attributes): BridgeModule[IN, OUT, Publisher[OUT]] = { new SinkBridgeModule[IN, OUT](attributes) } -} \ No newline at end of file +} diff --git a/experiments/akkastream/src/main/scala/akka/stream/gearpump/module/DummyModule.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/DummyModule.scala similarity index 89% rename from experiments/akkastream/src/main/scala/akka/stream/gearpump/module/DummyModule.scala rename to experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/DummyModule.scala index bc744f932..2c430d5b8 100644 --- a/experiments/akkastream/src/main/scala/akka/stream/gearpump/module/DummyModule.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/DummyModule.scala @@ -7,7 +7,7 @@ * "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 + * 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, @@ -16,9 +16,9 @@ * limitations under the License. */ -package akka.stream.gearpump.module +package org.apache.gearpump.akkastream.module -import akka.stream.impl.StreamLayout.Module +import akka.stream.impl.StreamLayout.{AtomicModule, Module} import akka.stream.impl.{SinkModule, SourceModule} import akka.stream.{Attributes, MaterializationContext, SinkShape, SourceShape} import org.reactivestreams.{Publisher, Subscriber} @@ -47,7 +47,8 @@ import org.reactivestreams.{Publisher, Subscriber} * * */ -trait DummyModule extends Module +trait DummyModule extends AtomicModule + /** * @@ -56,9 +57,9 @@ trait DummyModule extends Module * / * out of band input message Source * - * @param attributes - * @param shape - * @tparam Out + * @param attributes Attributes + * @param shape SourceShape[Out] + * @tparam Out Output */ class DummySource[Out](val attributes: Attributes, shape: SourceShape[Out]) extends SourceModule[Out, Unit](shape) with DummyModule { @@ -76,6 +77,7 @@ class DummySource[Out](val attributes: Attributes, shape: SourceShape[Out]) } } + /** * * Source-> [[BridgeModule]] -> [[DummySink]] @@ -84,8 +86,8 @@ class DummySource[Out](val attributes: Attributes, shape: SourceShape[Out]) * \| * out of band output message [[Subscriber]] * - * @param attributes - * @param shape + * @param attributes Attributes + * @param shape SinkShape[IN] */ class DummySink[IN](val attributes: Attributes, shape: SinkShape[IN]) extends SinkModule[IN, Unit](shape) with DummyModule { @@ -100,4 +102,4 @@ class DummySink[IN](val attributes: Attributes, shape: SinkShape[IN]) override def withAttributes(attr: Attributes): Module = { new DummySink[IN](attr, amendShape(attr)) } -} \ No newline at end of file +} diff --git a/experiments/akkastream/src/main/scala/akka/stream/gearpump/module/GearpumpTaskModule.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/GearpumpTaskModule.scala similarity index 66% rename from experiments/akkastream/src/main/scala/akka/stream/gearpump/module/GearpumpTaskModule.scala rename to experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/GearpumpTaskModule.scala index c4c78cc40..755524429 100644 --- a/experiments/akkastream/src/main/scala/akka/stream/gearpump/module/GearpumpTaskModule.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/GearpumpTaskModule.scala @@ -7,7 +7,7 @@ * "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 + * 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, @@ -16,12 +16,10 @@ * limitations under the License. */ -package akka.stream.gearpump.module - -import akka.stream.impl.FlowModule -import akka.stream.impl.StreamLayout.Module -import akka.stream.{Attributes, Inlet, Outlet, Shape, SinkShape, SourceShape} +package org.apache.gearpump.akkastream.module +import akka.stream.impl.StreamLayout.{AtomicModule, Module} +import akka.stream._ import org.apache.gearpump.cluster.UserConfig import org.apache.gearpump.streaming.sink.DataSink import org.apache.gearpump.streaming.source.DataSource @@ -32,17 +30,17 @@ import org.apache.gearpump.streaming.task.Task * * This is specially designed for Gearpump runtime. It is not supposed to be used * for local materializer. - * + * */ -trait GearpumpTaskModule extends Module +trait GearpumpTaskModule extends AtomicModule /** * This is used to represent the Gearpump Data Source - * @param source - * @param conf - * @param shape - * @param attributes - * @tparam T + * @param source DataSource + * @param conf UserConfig + * @param shape SourceShape[T} + * @param attributes Attributes + * @tparam T type */ final case class SourceTaskModule[T]( source: DataSource, @@ -51,13 +49,10 @@ final case class SourceTaskModule[T]( attributes: Attributes = Attributes.name("SourceTaskModule")) extends GearpumpTaskModule { - override def subModules: Set[Module] = Set.empty - override def withAttributes(attr: Attributes): Module = { + override def withAttributes(attr: Attributes): Module = this.copy(shape = amendShape(attr), attributes = attr) - } - override def carbonCopy: Module = { - this.copy(shape = SourceShape(Outlet[T]("SourceTaskModule.out"))) - } + override def carbonCopy: Module = + this.copy(shape = SourceShape( Outlet[T]("SourceTaskModule.out"))) override def replaceShape(s: Shape): Module = if (s == shape) this @@ -68,17 +63,17 @@ final case class SourceTaskModule[T]( val thatN = attr.nameOrDefault(null) if ((thatN eq null) || thisN == thatN) shape - else shape.copy(outlet = Outlet(thatN + ".out")) + else shape.copy(out = Outlet(thatN + ".out")) } } /** * This is used to represent the Gearpump Data Sink - * @param sink - * @param conf - * @param shape - * @param attributes - * @tparam IN + * @param sink DataSink + * @param conf UserConfig + * @param shape SinkShape[IN] + * @param attributes Attributes + * @tparam IN type */ final case class SinkTaskModule[IN]( sink: DataSink, @@ -87,11 +82,10 @@ final case class SinkTaskModule[IN]( attributes: Attributes = Attributes.name("SinkTaskModule")) extends GearpumpTaskModule { - override def subModules: Set[Module] = Set.empty - override def withAttributes(attr: Attributes): Module = { + override def withAttributes(attr: Attributes): Module = this.copy(shape = amendShape(attr), attributes = attr) - } - override def carbonCopy: Module = this.copy(shape = SinkShape(Inlet[IN]("SinkTaskModule.in"))) + override def carbonCopy: Module = + this.copy(shape = SinkShape(Inlet[IN]("SinkTaskModule.in"))) override def replaceShape(s: Shape): Module = if (s == shape) this @@ -102,32 +96,40 @@ final case class SinkTaskModule[IN]( val thatN = attr.nameOrDefault(null) if ((thatN eq null) || thisN == thatN) shape - else shape.copy(inlet = Inlet(thatN + ".out")) + else shape.copy(in = Inlet(thatN + ".out")) } } /** * This is to represent the Gearpump Processor which has exact one input and one output - * @param processor - * @param conf - * @param attributes - * @tparam IN - * @tparam OUT - * @tparam Unit + * @param processor Class[_ <: Task] + * @param conf UserConfig + * @param attributes Attributes + * @tparam IN type + * @tparam OUT type + * @tparam Unit void */ case class ProcessorModule[IN, OUT, Unit]( processor: Class[_ <: Task], conf: UserConfig, - val attributes: Attributes = Attributes.name("processorModule")) - extends FlowModule[IN, OUT, Unit] with GearpumpTaskModule { - + attributes: Attributes = Attributes.name("processorModule")) + extends AtomicModule with GearpumpTaskModule { + val inPort = Inlet[IN]("ProcessorModule.in") + val outPort = Outlet[IN]("ProcessorModule.out") + override val shape = new FlowShape(inPort, outPort) + + override def replaceShape(s: Shape): Module = if (s != shape) { + throw new UnsupportedOperationException("cannot replace the shape of a FlowModule") + } else { + this + } + override def carbonCopy: Module = newInstance - protected def newInstance: ProcessorModule[IN, OUT, Unit] = { + protected def newInstance: ProcessorModule[IN, OUT, Unit] = new ProcessorModule[IN, OUT, Unit](processor, conf, attributes) - } override def withAttributes(attributes: Attributes): ProcessorModule[IN, OUT, Unit] = { new ProcessorModule[IN, OUT, Unit](processor, conf, attributes) } -} \ No newline at end of file +} diff --git a/experiments/akkastream/src/main/scala/akka/stream/gearpump/module/GroupByModule.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/GroupByModule.scala similarity index 56% rename from experiments/akkastream/src/main/scala/akka/stream/gearpump/module/GroupByModule.scala rename to experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/GroupByModule.scala index e57a6f687..4465886db 100644 --- a/experiments/akkastream/src/main/scala/akka/stream/gearpump/module/GroupByModule.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/GroupByModule.scala @@ -7,7 +7,7 @@ * "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 + * 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, @@ -16,31 +16,40 @@ * limitations under the License. */ -package akka.stream.gearpump.module +package org.apache.gearpump.akkastream.module + +import akka.stream._ +import akka.stream.impl.StreamLayout.{AtomicModule, Module} -import akka.stream.Attributes -import akka.stream.impl.FlowModule -import akka.stream.impl.StreamLayout.Module /** * * Group the T value groupBy function * - * @param f - * @param attributes - * @tparam T - * @tparam Group + * @param groupBy T => Group + * @param attributes Attributes + * @tparam T type + * @tparam Group type */ -case class GroupByModule[T, Group](val groupBy: T => Group, - val attributes: Attributes = Attributes.name("groupByModule")) extends FlowModule[T, T, Unit] { +case class GroupByModule[T, Group](groupBy: T => Group, + attributes: Attributes = Attributes.name("groupByModule")) + extends AtomicModule { + val inPort = Inlet[T]("GroupByModule.in") + val outPort = Outlet[T]("GroupByModule.out") + override val shape = new FlowShape(inPort, outPort) + + override def replaceShape(s: Shape): Module = if (s != shape) { + throw new UnsupportedOperationException("cannot replace the shape of a FlowModule") + } else { + this + } override def carbonCopy: Module = newInstance - protected def newInstance: GroupByModule[T, Group] = { + protected def newInstance: GroupByModule[T, Group] = new GroupByModule[T, Group](groupBy, attributes) - } override def withAttributes(attributes: Attributes): GroupByModule[T, Group] = { new GroupByModule[T, Group](groupBy, attributes) } -} \ No newline at end of file +} diff --git a/experiments/akkastream/src/main/scala/akka/stream/gearpump/module/ReduceModule.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/ReduceModule.scala similarity index 59% rename from experiments/akkastream/src/main/scala/akka/stream/gearpump/module/ReduceModule.scala rename to experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/ReduceModule.scala index 926feb6fb..295556f50 100644 --- a/experiments/akkastream/src/main/scala/akka/stream/gearpump/module/ReduceModule.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/ReduceModule.scala @@ -7,7 +7,7 @@ * "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 + * 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, @@ -16,23 +16,31 @@ * limitations under the License. */ -package akka.stream.gearpump.module +package org.apache.gearpump.akkastream.module + +import akka.stream._ +import akka.stream.impl.StreamLayout.{AtomicModule, Module} -import akka.stream.Attributes -import akka.stream.impl.FlowModule -import akka.stream.impl.StreamLayout.Module /** * * Reduce Module * - * @param f - * @param attributes - * @tparam T + * @param f (T,T) => T + * @param attributes Attributes + * @tparam T type */ -case class ReduceModule[T]( - val f: (T, T) => T, val attributes: Attributes = Attributes.name("reduceModule")) - extends FlowModule[T, T, Unit] { +case class ReduceModule[T](f: (T, T) => T, attributes: Attributes = +Attributes.name("reduceModule")) extends AtomicModule { + val inPort = Inlet[T]("GroupByModule.in") + val outPort = Outlet[T]("GroupByModule.out") + override val shape = new FlowShape(inPort, outPort) + + override def replaceShape(s: Shape): Module = if (s != shape) { + throw new UnsupportedOperationException("cannot replace the shape of a FlowModule") + } else { + this + } override def carbonCopy: Module = newInstance @@ -41,4 +49,4 @@ case class ReduceModule[T]( override def withAttributes(attributes: Attributes): ReduceModule[T] = { new ReduceModule[T](f, attributes) } -} \ No newline at end of file +} diff --git a/experiments/akkastream/src/main/scala/akka/stream/gearpump/scaladsl/Api.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/scaladsl/Api.scala similarity index 91% rename from experiments/akkastream/src/main/scala/akka/stream/gearpump/scaladsl/Api.scala rename to experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/scaladsl/Api.scala index 9cc46c9a4..85b1d5e7a 100644 --- a/experiments/akkastream/src/main/scala/akka/stream/gearpump/scaladsl/Api.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/scaladsl/Api.scala @@ -7,7 +7,7 @@ * "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 + * 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, @@ -16,19 +16,19 @@ * limitations under the License. */ -package akka.stream.gearpump.scaladsl +package org.apache.gearpump.akkastream.scaladsl import akka.stream.Attributes -import akka.stream.gearpump.module.{DummySink, DummySource, GroupByModule, ProcessorModule, ReduceModule, SinkBridgeModule, SinkTaskModule, SourceBridgeModule, SourceTaskModule} +import org.apache.gearpump.akkastream.module._ import akka.stream.scaladsl.{Flow, Keep, Sink, Source} -import org.reactivestreams.{Publisher, Subscriber} - import org.apache.gearpump.cluster.UserConfig import org.apache.gearpump.streaming.sink.DataSink import org.apache.gearpump.streaming.source.DataSource import org.apache.gearpump.streaming.task.Task +import org.reactivestreams.{Publisher, Subscriber} + -object GearSource { +object GearSource{ /** * Construct a Source which accepts out of band input messages. @@ -52,7 +52,7 @@ object GearSource { /** * Construct a Source from Gearpump [[DataSource]]. * - * [[SourceTaskModule]] -> downstream Sink + * [[SourceTaskModule]] -> downstream Sink * */ def from[OUT](source: DataSource): Source[OUT, Unit] = { @@ -63,7 +63,7 @@ object GearSource { /** * Construct a Source from Gearpump [[org.apache.gearpump.streaming.Processor]]. * - * [[ProcessorModule]] -> downstream Sink + * [[ProcessorModule]] -> downstream Sink * */ def from[OUT](processor: Class[_ <: Task], conf: UserConfig): Source[OUT, Unit] = { @@ -97,7 +97,7 @@ object GearSink { /** * Construct a Sink from Gearpump [[DataSink]]. * - * Upstream Source -> [[SinkTaskModule]] + * Upstream Source -> [[SinkTaskModule]] * */ def to[IN](sink: DataSink): Sink[IN, Unit] = { @@ -108,7 +108,7 @@ object GearSink { /** * Construct a Sink from Gearpump [[org.apache.gearpump.streaming.Processor]]. * - * Upstream Source -> [[ProcessorModule]] + * Upstream Source -> [[ProcessorModule]] * */ def to[IN](processor: Class[_ <: Task], conf: UserConfig): Sink[IN, Unit] = { @@ -131,7 +131,7 @@ object GearSink { * * val flow: Flow[KV] = GroupBy[KV](foo).map{ kv => * Count(kv.key, 1) - * }.fold(Count(null, 0)){(base, add) => + * }.fold(Count(null, 0)) {(base, add) => * Count(add.key, base.count + add.count) * }.log("count of current key") * .flatten() @@ -146,7 +146,7 @@ object GearSink { * sink will only operate on the main stream. * */ -object GroupBy { +object GroupBy{ def apply[T, Group](groupBy: T => Group): Flow[T, T, Unit] = { new Flow[T, T, Unit](new GroupByModule(groupBy)) } @@ -159,18 +159,19 @@ object GroupBy { * * */ -object Reduce { +object Reduce{ def apply[T](reduce: (T, T) => T): Flow[T, T, Unit] = { new Flow[T, T, Unit](new ReduceModule(reduce)) } } + /** * Create a Flow by providing a Gearpump Processor class and configuration * * */ -object Processor { +object Processor{ def apply[In, Out](processor: Class[_ <: Task], conf: UserConfig): Flow[In, Out, Unit] = { new Flow[In, Out, Unit](new ProcessorModule[In, Out, Unit](processor, conf)) } @@ -183,7 +184,7 @@ object Implicits { */ implicit class SourceOps[T, Mat](source: Source[T, Mat]) { - //TODO It is named as groupBy2 to avoid conflict with built-in + // TODO It is named as groupBy2 to avoid conflict with built-in // groupBy. Eventually, we think the built-in groupBy should // be replace with this implementation. def groupBy2[Group](groupBy: T => Group): Source[T, Mat] = { @@ -191,6 +192,7 @@ object Implicits { source.via[T, Unit](stage) } + def reduce(reduce: (T, T) => T): Source[T, Mat] = { val stage = Reduce.apply(reduce) source.via[T, Unit](stage) @@ -237,10 +239,13 @@ object Implicits { } /** - * Does sum on values + * do sum on values * * Before doing this, you need to do groupByKey to group same key together * , otherwise, it will do the sum no matter what current key is. + * + * @param numeric Numeric[V] + * @return */ def sumOnValue(implicit numeric: Numeric[V]): Source[(K, V), Mat] = { val stage = Reduce.apply(sumByKey[K, V](numeric)) @@ -249,13 +254,13 @@ object Implicits { } /** - * Helper util to support groupByKey and sum + * Help util to support groupByKey and sum */ implicit class KVFlowOps[K, V, Mat](flow: Flow[(K, V), (K, V), Mat]) { /** - * If it is a KV Pair, we can group the KV pair by the key. - * + * if it is a KV Pair, we can group the KV pair by the key. + * @return */ def groupByKey: Flow[(K, V), (K, V), Mat] = { val stage = GroupBy.apply(getTupleKey[K, V]) @@ -268,6 +273,8 @@ object Implicits { * Before doing this, you need to do groupByKey to group same key together * , otherwise, it will do the sum no matter what current key is. * + * @param numeric Numeric[V] + * @return */ def sumOnValue(implicit numeric: Numeric[V]): Flow[(K, V), (K, V), Mat] = { val stage = Reduce.apply(sumByKey[K, V](numeric)) @@ -279,4 +286,4 @@ object Implicits { private def sumByKey[K, V](numeric: Numeric[V]): (Tuple2[K, V], Tuple2[K, V]) => Tuple2[K, V] = (tuple1, tuple2) => Tuple2(tuple1._1, numeric.plus(tuple1._2, tuple2._2)) -} \ No newline at end of file +} diff --git a/experiments/akkastream/src/main/scala/akka/stream/gearpump/task/BalanceTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/BalanceTask.scala similarity index 80% rename from experiments/akkastream/src/main/scala/akka/stream/gearpump/task/BalanceTask.scala rename to experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/BalanceTask.scala index 2eb061224..513911711 100644 --- a/experiments/akkastream/src/main/scala/akka/stream/gearpump/task/BalanceTask.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/BalanceTask.scala @@ -7,7 +7,7 @@ * "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 + * 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, @@ -16,22 +16,23 @@ * limitations under the License. */ -package akka.stream.gearpump.task +package org.apache.gearpump.akkastream.task import org.apache.gearpump.Message import org.apache.gearpump.cluster.UserConfig import org.apache.gearpump.streaming.task.TaskContext -class BalanceTask(context: TaskContext, userConf: UserConfig) extends GraphTask(context, userConf) { +class BalanceTask(context: TaskContext, userConf : UserConfig) + extends GraphTask(context, userConf) { val sizeOfOutputs = sizeOfOutPorts var index = 0 - override def onNext(msg: Message): Unit = { + override def onNext(msg : Message) : Unit = { output(index, msg) index += 1 if (index == sizeOfOutputs) { index = 0 } } -} \ No newline at end of file +} diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/BatchTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/BatchTask.scala new file mode 100644 index 000000000..582327b60 --- /dev/null +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/BatchTask.scala @@ -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.gearpump.akkastream.task + +import java.util.concurrent.TimeUnit + +import org.apache.gearpump.Message +import org.apache.gearpump.cluster.UserConfig +import org.apache.gearpump.streaming.task.TaskContext + +import scala.concurrent.Future +import scala.concurrent.duration.FiniteDuration + +class BatchTask[In, Out](context: TaskContext, userConf : UserConfig) + extends GraphTask(context, userConf) { + + val max = userConf.getLong(BatchTask.MAX) + val costFunc = userConf.getValue[In => Long](BatchTask.COST) + val aggregate = userConf.getValue[(Out, In) => Out](BatchTask.AGGREGATE) + val seed = userConf.getValue[In => Out](BatchTask.SEED) + + override def onNext(msg : Message) : Unit = { + val data = msg.msg.asInstanceOf[In] + val time = msg.timestamp + context.output(msg) + } +} + +object BatchTask { + val AGGREGATE = "AGGREGATE" + val COST = "COST" + val MAX = "MAX" + val SEED = "SEED" +} diff --git a/experiments/akkastream/src/main/scala/akka/stream/gearpump/task/BroadcastTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/BroadcastTask.scala similarity index 78% rename from experiments/akkastream/src/main/scala/akka/stream/gearpump/task/BroadcastTask.scala rename to experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/BroadcastTask.scala index 925bf21cf..9f1194fca 100644 --- a/experiments/akkastream/src/main/scala/akka/stream/gearpump/task/BroadcastTask.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/BroadcastTask.scala @@ -7,7 +7,7 @@ * "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 + * 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, @@ -16,14 +16,15 @@ * limitations under the License. */ -package akka.stream.gearpump.task +package org.apache.gearpump.akkastream.task import org.apache.gearpump.Message import org.apache.gearpump.cluster.UserConfig import org.apache.gearpump.streaming.task.TaskContext -class BroadcastTask(context: TaskContext, userConf: UserConfig) extends GraphTask(context, userConf) { - override def onNext(msg: Message): Unit = { +class BroadcastTask(context: TaskContext, userConf : UserConfig) + extends GraphTask(context, userConf) { + override def onNext(msg : Message) : Unit = { context.output(msg) } -} \ No newline at end of file +} diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/ConcatTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/ConcatTask.scala new file mode 100644 index 000000000..241fa7699 --- /dev/null +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/ConcatTask.scala @@ -0,0 +1,38 @@ +/* + * 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.gearpump.akkastream.task + +import org.apache.gearpump.Message +import org.apache.gearpump.cluster.UserConfig +import org.apache.gearpump.streaming.task.TaskContext + +class ConcatTask(context: TaskContext, userConf : UserConfig) + extends GraphTask(context, userConf) { + + val sizeOfOutputs = sizeOfOutPorts + var index = 0 + + override def onNext(msg : Message) : Unit = { + output(index, msg) + index += 1 + if (index == sizeOfOutputs) { + index = 0 + } + } +} diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/DelayInitialTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/DelayInitialTask.scala new file mode 100644 index 000000000..d6c347aee --- /dev/null +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/DelayInitialTask.scala @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gearpump.akkastream.task + +import java.time.Instant +import java.util.concurrent.TimeUnit + +import org.apache.gearpump.Message +import org.apache.gearpump.cluster.UserConfig +import org.apache.gearpump.streaming.task.TaskContext + +import scala.concurrent.Future +import scala.concurrent.duration.FiniteDuration + +case object DelayInitialTime + +class DelayInitialTask[T](context: TaskContext, userConf : UserConfig) + extends GraphTask(context, userConf) { + + val delayInitial = userConf.getValue[FiniteDuration](DelayInitialTask.DELAY_INITIAL). + getOrElse(FiniteDuration(0, TimeUnit.MINUTES)) + var delayInitialActive = true + + override def onStart(startTime: Instant): Unit = { + context.scheduleOnce(delayInitial)( + self ! Message(DelayInitialTime, System.currentTimeMillis()) + ) + } + override def onNext(msg : Message) : Unit = { + msg.msg match { + case DelayInitialTime => + delayInitialActive = false + case _ => + delayInitialActive match { + case true => + case false => + context.output(msg) + } + } + } +} + +object DelayInitialTask { + val DELAY_INITIAL = "DELAY_INITIAL" +} diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/DropWithinTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/DropWithinTask.scala new file mode 100644 index 000000000..9da26b1aa --- /dev/null +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/DropWithinTask.scala @@ -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.gearpump.akkastream.task + +import java.time.Instant +import java.util.concurrent.TimeUnit + +import org.apache.gearpump.Message +import org.apache.gearpump.cluster.UserConfig +import org.apache.gearpump.streaming.task.TaskContext + +import scala.concurrent.duration.FiniteDuration + +case object DropWithinTimeout + +class DropWithinTask[T](context: TaskContext, userConf : UserConfig) + extends GraphTask(context, userConf) { + + val timeout = userConf.getValue[FiniteDuration](DropWithinTask.TIMEOUT). + getOrElse(FiniteDuration(0, TimeUnit.MINUTES)) + var timeoutActive = true + + override def onStart(startTime: Instant): Unit = { + context.scheduleOnce(timeout)( + self ! Message(DropWithinTimeout, System.currentTimeMillis()) + ) + } + + override def onNext(msg : Message) : Unit = { + msg.msg match { + case DropWithinTimeout => + timeoutActive = false + case _ => + + } + timeoutActive match { + case true => + case false => + context.output(msg) + } + } +} + +object DropWithinTask { + val TIMEOUT = "TIMEOUT" +} diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/FlattenMergeTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/FlattenMergeTask.scala new file mode 100644 index 000000000..512164d31 --- /dev/null +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/FlattenMergeTask.scala @@ -0,0 +1,38 @@ +/* + * 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.gearpump.akkastream.task + +import org.apache.gearpump.Message +import org.apache.gearpump.cluster.UserConfig +import org.apache.gearpump.streaming.task.TaskContext + +class FlattenMergeTask(context: TaskContext, userConf : UserConfig) + extends GraphTask(context, userConf) { + + val sizeOfOutputs = sizeOfOutPorts + var index = 0 + + override def onNext(msg : Message) : Unit = { + output(index, msg) + index += 1 + if (index == sizeOfOutputs) { + index = 0 + } + } +} diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/FoldTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/FoldTask.scala new file mode 100644 index 000000000..e2f02d89b --- /dev/null +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/FoldTask.scala @@ -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.gearpump.akkastream.task + +import java.time.Instant + +import org.apache.gearpump.Message +import org.apache.gearpump.cluster.UserConfig +import org.apache.gearpump.streaming.task.TaskContext + +class FoldTask[In, Out](context: TaskContext, userConf : UserConfig) + extends GraphTask(context, userConf) { + + val zero = userConf.getValue[Out](FoldTask.ZERO) + val aggregator = userConf.getValue[(Out, In) => Out](FoldTask.AGGREGATOR) + var aggregated: Out = _ + implicit val ec = context.system.dispatcher + + override def onStart(instant: Instant): Unit = { + zero.foreach(value => { + aggregated = value + }) + } + + override def onNext(msg : Message) : Unit = { + val data = msg.msg.asInstanceOf[In] + val time = msg.timestamp + aggregator.foreach(func => { + aggregated = func(aggregated, data) + LOG.info(s"aggregated = $aggregated") + val msg = new Message(aggregated, time) + context.output(msg) + }) + } +} + +object FoldTask { + val ZERO = "ZERO" + val AGGREGATOR = "AGGREGATOR" +} diff --git a/experiments/akkastream/src/main/scala/akka/stream/gearpump/task/GraphTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/GraphTask.scala similarity index 63% rename from experiments/akkastream/src/main/scala/akka/stream/gearpump/task/GraphTask.scala rename to experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/GraphTask.scala index 9a4e24eeb..8e7a2df44 100644 --- a/experiments/akkastream/src/main/scala/akka/stream/gearpump/task/GraphTask.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/GraphTask.scala @@ -7,7 +7,7 @@ * "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 + * 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, @@ -16,35 +16,36 @@ * limitations under the License. */ -package akka.stream.gearpump.task +package org.apache.gearpump.akkastream.task -import akka.stream.gearpump.task.GraphTask.{Index, PortId} +import java.time.Instant import org.apache.gearpump.Message +import org.apache.gearpump.akkastream.task.GraphTask.{Index, PortId} import org.apache.gearpump.cluster.UserConfig import org.apache.gearpump.streaming.ProcessorId -import org.apache.gearpump.streaming.task.{StartTime, Task, TaskContext, TaskWrapper} +import org.apache.gearpump.streaming.task.{Task, TaskContext, TaskWrapper} -class GraphTask(inputTaskContext: TaskContext, userConf: UserConfig) +class GraphTask(inputTaskContext : TaskContext, userConf : UserConfig) extends Task(inputTaskContext, userConf) { private val context = inputTaskContext.asInstanceOf[TaskWrapper] - private val outMapping = portsMapping(userConf.getValue[List[ProcessorId]]( - GraphTask.OUT_PROCESSORS).get) - private val inMapping = portsMapping(userConf.getValue[List[ProcessorId]]( - GraphTask.IN_PROCESSORS).get) + protected val outMapping = + portsMapping(userConf.getValue[List[ProcessorId]](GraphTask.OUT_PROCESSORS).get) + protected val inMapping = + portsMapping(userConf.getValue[List[ProcessorId]](GraphTask.IN_PROCESSORS).get) val sizeOfOutPorts = outMapping.keys.size val sizeOfInPorts = inMapping.keys.size - + private def portsMapping(processors: List[ProcessorId]): Map[PortId, Index] = { - val portToProcessor = processors.zipWithIndex.map { kv => + val portToProcessor = processors.zipWithIndex.map{kv => (kv._2, kv._1) }.toMap val processorToIndex = processors.sorted.zipWithIndex.toMap - val portToIndex = portToProcessor.map { kv => + val portToIndex = portToProcessor.map{kv => val (outlet, processorId) = kv val index = processorToIndex(processorId) (outlet, index) @@ -56,15 +57,15 @@ class GraphTask(inputTaskContext: TaskContext, userConf: UserConfig) context.output(outMapping(outletId), msg) } - override def onStart(startTime: StartTime): Unit = {} + override def onStart(startTime : Instant) : Unit = {} - override def onStop(): Unit = {} + override def onStop() : Unit = {} } object GraphTask { - val OUT_PROCESSORS = "akka.stream.gearpump.task.outprocessors" - val IN_PROCESSORS = "akka.stream.gearpump.task.inprocessors" + val OUT_PROCESSORS = "org.apache.gearpump.akkastream.task.outprocessors" + val IN_PROCESSORS = "org.apache.gearpump.akkastream.task.inprocessors" type PortId = Int type Index = Int -} \ No newline at end of file +} diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/GroupedWithinTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/GroupedWithinTask.scala new file mode 100644 index 000000000..29d9c91a7 --- /dev/null +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/GroupedWithinTask.scala @@ -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.gearpump.akkastream.task + +import org.apache.gearpump.Message +import org.apache.gearpump.cluster.UserConfig +import org.apache.gearpump.streaming.task.TaskContext + +import scala.collection.immutable.VectorBuilder +import scala.concurrent.duration.FiniteDuration + +class GroupedWithinTask[T](context: TaskContext, userConf : UserConfig) + extends GraphTask(context, userConf) { + + case object GroupedWithinTrigger + val buf: VectorBuilder[T] = new VectorBuilder + val timeWindow = userConf.getValue[FiniteDuration](GroupedWithinTask.TIME_WINDOW) + val batchSize = userConf.getInt(GroupedWithinTask.BATCH_SIZE) + + override def onNext(msg : Message) : Unit = { + + } +} + +object GroupedWithinTask { + val BATCH_SIZE = "BATCH_SIZE" + val TIME_WINDOW = "TIME_WINDOW" +} diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/InterleaveTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/InterleaveTask.scala new file mode 100644 index 000000000..837de6b49 --- /dev/null +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/InterleaveTask.scala @@ -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.gearpump.akkastream.task + +import org.apache.gearpump.Message +import org.apache.gearpump.cluster.UserConfig +import org.apache.gearpump.streaming.task.TaskContext + +class InterleaveTask(context: TaskContext, userConf : UserConfig) + extends GraphTask(context, userConf) { + + val sizeOfInputs = sizeOfInPorts + var index = 0 + + // TODO access upstream and pull + override def onNext(msg : Message) : Unit = { + output(index, msg) + index += 1 + if (index == sizeOfInputs) { + index = 0 + } + } +} + +object InterleaveTask { + val INPUT_PORTS = "INPUT_PORTS" + val SEGMENT_SIZE = "SEGMENT_SIZE" +} diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/MapAsyncTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/MapAsyncTask.scala new file mode 100644 index 000000000..387116db7 --- /dev/null +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/MapAsyncTask.scala @@ -0,0 +1,53 @@ +/* + * 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.gearpump.akkastream.task + +import org.apache.gearpump.Message +import org.apache.gearpump.cluster.UserConfig +import org.apache.gearpump.streaming.task.TaskContext + +import scala.concurrent.Future + +class MapAsyncTask[In, Out](context: TaskContext, userConf : UserConfig) + extends GraphTask(context, userConf) { + + val f = userConf.getValue[In => Future[Out]](MapAsyncTask.MAPASYNC_FUNC) + implicit val ec = context.system.dispatcher + + override def onNext(msg : Message) : Unit = { + val data = msg.msg.asInstanceOf[In] + val time = msg.timestamp + f match { + case Some(func) => + val fout = func(data) + fout.onComplete(value => { + value.foreach(out => { + val msg = new Message(out, time) + context.output(msg) + }) + }) + case None => + } + } +} + +object MapAsyncTask { + val MAPASYNC_FUNC = "MAPASYNC_FUNC" + +} diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/MergeTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/MergeTask.scala new file mode 100644 index 000000000..2b1cd3320 --- /dev/null +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/MergeTask.scala @@ -0,0 +1,39 @@ +/* + * 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. + * SeG the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.gearpump.akkastream.task + +import org.apache.gearpump.Message +import org.apache.gearpump.cluster.UserConfig +import org.apache.gearpump.streaming.task.TaskContext + +class MergeTask(context: TaskContext, userConf : UserConfig) + extends GraphTask(context, userConf) { + + val eagerComplete = userConf.getBoolean(MergeTask.EAGER_COMPLETE) + val inputPorts = userConf.getInt(MergeTask.INPUT_PORTS) + + override def onNext(msg : Message) : Unit = { + context.output(msg) + } +} + +object MergeTask { + val EAGER_COMPLETE = "EAGER_COMPLETE" + val INPUT_PORTS = "INPUT_PORTS" +} diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/SingleSourceTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/SingleSourceTask.scala new file mode 100644 index 000000000..1ff9ccd27 --- /dev/null +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/SingleSourceTask.scala @@ -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.gearpump.akkastream.task + +import java.time.Instant +import java.util.Date +import java.util.concurrent.TimeUnit + +import org.apache.gearpump.Message +import org.apache.gearpump.cluster.UserConfig +import org.apache.gearpump.streaming.task.TaskContext + +import scala.concurrent.duration.FiniteDuration + +class SingleSourceTask[T](context: TaskContext, userConf : UserConfig) + extends GraphTask(context, userConf) { + + val elem = userConf.getValue[T](SingleSourceTask.ELEMENT).get + + override def onNext(msg : Message) : Unit = { + context.output(Message(elem, msg.timestamp)) + } +} + +object SingleSourceTask { + val ELEMENT = "ELEMENT" +} diff --git a/experiments/akkastream/src/main/scala/akka/stream/gearpump/task/SinkBridgeTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/SinkBridgeTask.scala similarity index 77% rename from experiments/akkastream/src/main/scala/akka/stream/gearpump/task/SinkBridgeTask.scala rename to experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/SinkBridgeTask.scala index b68185230..05011e983 100644 --- a/experiments/akkastream/src/main/scala/akka/stream/gearpump/task/SinkBridgeTask.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/SinkBridgeTask.scala @@ -7,7 +7,7 @@ * "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 + * 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, @@ -16,24 +16,24 @@ * limitations under the License. */ -package akka.stream.gearpump.task +package org.apache.gearpump.akkastream.task +import java.time.Instant import java.util import java.util.concurrent.TimeUnit import akka.actor.Actor.Receive import akka.actor.{Actor, ActorRef, ActorSystem, Props} -import akka.stream.gearpump.task.SinkBridgeTask.RequestMessage import akka.util.Timeout -import org.reactivestreams.{Publisher, Subscriber, Subscription} - import org.apache.gearpump.Message +import org.apache.gearpump.akkastream.task.SinkBridgeTask.RequestMessage import org.apache.gearpump.cluster.UserConfig import org.apache.gearpump.cluster.client.ClientContext import org.apache.gearpump.streaming.ProcessorId import org.apache.gearpump.streaming.appmaster.AppMaster.{LookupTaskActorRef, TaskActorRef} -import org.apache.gearpump.streaming.task.{StartTime, Task, TaskContext, TaskId} +import org.apache.gearpump.streaming.task.{Task, TaskContext, TaskId} import org.apache.gearpump.util.LogUtil +import org.reactivestreams.{Publisher, Subscriber, Subscription} /** * Bridge Task when data flow is from remote Gearpump Task to local Akka-Stream Module @@ -46,23 +46,27 @@ import org.apache.gearpump.util.LogUtil * \| * Akka Stream [[Subscriber]] * + * + * @param taskContext TaskContext + * @param userConf UserConfig */ -class SinkBridgeTask(taskContext: TaskContext, userConf: UserConfig) extends Task(taskContext, userConf) { +class SinkBridgeTask(taskContext : TaskContext, userConf : UserConfig) + extends Task(taskContext, userConf) { import taskContext.taskId val queue = new util.LinkedList[Message]() - var subscriber: ActorRef = null + var subscriber: ActorRef = _ var request: Int = 0 - override def onStart(startTime: StartTime): Unit = {} + override def onStart(startTime : Instant) : Unit = {} - override def onNext(msg: Message): Unit = { + override def onNext(msg : Message) : Unit = { queue.add(msg) trySendingData() } - override def onStop(): Unit = {} + override def onStop() : Unit = {} private def trySendingData(): Unit = { if (subscriber != null) { @@ -88,15 +92,16 @@ object SinkBridgeTask { case class RequestMessage(number: Int) - class SinkBridgeTaskClient(system: ActorSystem, context: ClientContext, appId: Int, processorId: ProcessorId) extends Publisher[AnyRef] with Subscription { + class SinkBridgeTaskClient(system: ActorSystem, context: ClientContext, appId: Int, + processorId: ProcessorId) extends Publisher[AnyRef] with Subscription { private val taskId = TaskId(processorId, index = 0) - private val LOG = LogUtil.getLogger(getClass) - private var actor: ActorRef = null + private var actor: ActorRef = _ import system.dispatcher - private val task = context.askAppMaster[TaskActorRef](appId, LookupTaskActorRef(taskId)).map { container => + private val task = + context.askAppMaster[TaskActorRef](appId, LookupTaskActorRef(taskId)).map{container => // println("Successfully resolved taskRef for taskId " + taskId + ", " + container.task) container.task } @@ -111,7 +116,7 @@ object SinkBridgeTask { private implicit val timeout = Timeout(5, TimeUnit.SECONDS) override def request(l: Long): Unit = { - task.foreach { task => + task.foreach{ task => task.tell(RequestMessage(l.toInt), actor) } } @@ -119,7 +124,8 @@ object SinkBridgeTask { class ClientActor(subscriber: Subscriber[_ >: AnyRef]) extends Actor { def receive: Receive = { - case result: AnyRef => subscriber.onNext(result) + case result: AnyRef => + subscriber.onNext(result) } } -} \ No newline at end of file +} diff --git a/experiments/akkastream/src/main/scala/akka/stream/gearpump/task/SourceBridgeTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/SourceBridgeTask.scala similarity index 70% rename from experiments/akkastream/src/main/scala/akka/stream/gearpump/task/SourceBridgeTask.scala rename to experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/SourceBridgeTask.scala index ccbd35014..b0eda1909 100644 --- a/experiments/akkastream/src/main/scala/akka/stream/gearpump/task/SourceBridgeTask.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/SourceBridgeTask.scala @@ -7,7 +7,7 @@ * "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 + * 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, @@ -16,20 +16,21 @@ * limitations under the License. */ -package akka.stream.gearpump.task +package org.apache.gearpump.akkastream.task -import scala.concurrent.ExecutionContext +import java.time.Instant import akka.actor.Actor.Receive -import akka.stream.gearpump.task.SourceBridgeTask.{AkkaStreamMessage, Complete, Error} -import org.reactivestreams.{Subscriber, Subscription} - import org.apache.gearpump.Message +import org.apache.gearpump.akkastream.task.SourceBridgeTask.{AkkaStreamMessage, Complete, Error} import org.apache.gearpump.cluster.UserConfig import org.apache.gearpump.cluster.client.ClientContext import org.apache.gearpump.streaming.ProcessorId import org.apache.gearpump.streaming.appmaster.AppMaster.{LookupTaskActorRef, TaskActorRef} -import org.apache.gearpump.streaming.task.{StartTime, Task, TaskContext, TaskId} +import org.apache.gearpump.streaming.task.{Task, TaskContext, TaskId} +import org.reactivestreams.{Subscriber, Subscription} + +import scala.concurrent.ExecutionContext /** * Bridge Task when data flow is from local Akka-Stream Module to remote Gearpump Task @@ -42,44 +43,51 @@ import org.apache.gearpump.streaming.task.{StartTime, Task, TaskContext, TaskId} * / Local JVM * Akka Stream [[org.reactivestreams.Publisher]] * + * + * @param taskContext TaskContext + * @param userConf UserConfig */ -class SourceBridgeTask(taskContext: TaskContext, userConf: UserConfig) extends Task(taskContext, userConf) { +class SourceBridgeTask(taskContext : TaskContext, userConf : UserConfig) + extends Task(taskContext, userConf) { import taskContext.taskId - override def onStart(startTime: StartTime): Unit = {} + override def onStart(startTime : Instant) : Unit = {} - override def onNext(msg: Message): Unit = { + override def onNext(msg : Message) : Unit = { LOG.info("AkkaStreamSource receiving message " + msg) } - override def onStop(): Unit = {} + override def onStop() : Unit = {} override def receiveUnManagedMessage: Receive = { case Error(ex) => LOG.error("the stream has error", ex) case AkkaStreamMessage(msg) => - LOG.error("we have received message from akka stream source: " + msg) + LOG.info("we have received message from akka stream source: " + msg) taskContext.output(Message(msg, System.currentTimeMillis())) case Complete(description) => - LOG.error("the stream is completed: " + description) + LOG.info("the stream is completed: " + description) case msg => LOG.error("Failed! Received unknown message " + "taskId: " + taskId + ", " + msg.toString) } } + object SourceBridgeTask { case class Error(ex: java.lang.Throwable) case class Complete(description: String) - case class AkkaStreamMessage(msg: AnyRef) + case class AkkaStreamMessage[T >: AnyRef](msg: T) - class SourceBridgeTaskClient[T <: AnyRef](ec: ExecutionContext, context: ClientContext, appId: Int, processorId: ProcessorId) extends Subscriber[T] { + class SourceBridgeTaskClient[T >: AnyRef](ec: ExecutionContext, + context: ClientContext, appId: Int, processorId: ProcessorId) extends Subscriber[T] { val taskId = TaskId(processorId, 0) - var subscription: Subscription = null + var subscription: Subscription = _ implicit val dispatcher = ec - val task = context.askAppMaster[TaskActorRef](appId, LookupTaskActorRef(taskId)).map { container => + val task = context.askAppMaster[TaskActorRef](appId, + LookupTaskActorRef(taskId)).map{container => // println("Successfully resolved taskRef for taskId " + taskId + ", " + container.task) container.task } @@ -89,6 +97,7 @@ object SourceBridgeTask { } override def onSubscribe(subscription: Subscription): Unit = { + // when taskActorRef is resolved, request message from upstream this.subscription = subscription task.map(task => subscription.request(1)) } @@ -98,7 +107,7 @@ object SourceBridgeTask { } override def onNext(t: T): Unit = { - task.map { task => + task.map {task => task ! AkkaStreamMessage(t) } subscription.request(1) diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/StatefulMapConcatTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/StatefulMapConcatTask.scala new file mode 100644 index 000000000..bf2c14ffe --- /dev/null +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/StatefulMapConcatTask.scala @@ -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.gearpump.akkastream.task + +import java.time.Instant + +import org.apache.gearpump.Message +import org.apache.gearpump.cluster.UserConfig +import org.apache.gearpump.streaming.task.TaskContext + +class StatefulMapConcatTask[IN, OUT](context: TaskContext, userConf : UserConfig) + extends GraphTask(context, userConf) { + + val func = userConf.getValue[() => IN => Iterable[OUT]](StatefulMapConcatTask.FUNC).get + var f: IN => Iterable[OUT] = _ + + override def onStart(startTime: Instant) : Unit = { + f = func() + } + + override def onNext(msg : Message) : Unit = { + val in: IN = msg.msg.asInstanceOf[IN] + val out: Iterable[OUT] = f(in) + val iterator = out.iterator + while(iterator.hasNext) { + val nextValue = iterator.next + context.output(Message(nextValue, System.currentTimeMillis())) + } + } +} + +object StatefulMapConcatTask { + val FUNC = "FUNC" +} diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/TakeWithinTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/TakeWithinTask.scala new file mode 100644 index 000000000..ef43fbe41 --- /dev/null +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/TakeWithinTask.scala @@ -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.gearpump.akkastream.task + +import java.time.Instant +import java.util.concurrent.TimeUnit + +import org.apache.gearpump.Message +import org.apache.gearpump.cluster.UserConfig +import org.apache.gearpump.streaming.task.TaskContext + +import scala.concurrent.duration.FiniteDuration + +case object TakeWithinTimeout + +class TakeWithinTask[T](context: TaskContext, userConf : UserConfig) + extends GraphTask(context, userConf) { + + val timeout = userConf.getValue[FiniteDuration](TakeWithinTask.TIMEOUT). + getOrElse(FiniteDuration(0, TimeUnit.MINUTES)) + var timeoutActive = false + + override def onStart(startTime: Instant): Unit = { + context.scheduleOnce(timeout)( + self ! Message(DropWithinTimeout, System.currentTimeMillis()) + ) + } + + override def onNext(msg : Message) : Unit = { + msg.msg match { + case DropWithinTimeout => + timeoutActive = true + case _ => + + } + timeoutActive match { + case true => + case false => + context.output(msg) + } + } +} + +object TakeWithinTask { + val TIMEOUT = "TIMEOUT" +} diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/ThrottleTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/ThrottleTask.scala new file mode 100644 index 000000000..4e09bf2a0 --- /dev/null +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/ThrottleTask.scala @@ -0,0 +1,53 @@ +/* + * 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.gearpump.akkastream.task + +import java.util.concurrent.TimeUnit + +import org.apache.gearpump.Message +import org.apache.gearpump.cluster.UserConfig +import org.apache.gearpump.streaming.task.TaskContext + +import scala.concurrent.Future +import scala.concurrent.duration.FiniteDuration + +class ThrottleTask[T](context: TaskContext, userConf : UserConfig) + extends GraphTask(context, userConf) { + + val cost = userConf.getInt(ThrottleTask.COST).getOrElse(0) + val costCalc = userConf.getValue[T => Int](ThrottleTask.COST_CALC) + val maxBurst = userConf.getInt(ThrottleTask.MAX_BURST) + val timePeriod = userConf.getValue[FiniteDuration](ThrottleTask.TIME_PERIOD). + getOrElse(FiniteDuration(0, TimeUnit.MINUTES)) + val interval = timePeriod.toNanos / cost + + // TODO control rate from TaskActor + override def onNext(msg : Message) : Unit = { + val data = msg.msg.asInstanceOf[T] + val time = msg.timestamp + context.output(msg) + } +} + +object ThrottleTask { + val COST = "COST" + val COST_CALC = "COST_CAL" + val MAX_BURST = "MAX_BURST" + val TIME_PERIOD = "TIME_PERIOD" +} diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/TickSourceTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/TickSourceTask.scala new file mode 100644 index 000000000..b3850caa2 --- /dev/null +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/TickSourceTask.scala @@ -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.gearpump.akkastream.task + +import java.time.Instant +import java.util.Date +import java.util.concurrent.TimeUnit + +import org.apache.gearpump.Message +import org.apache.gearpump.cluster.UserConfig +import org.apache.gearpump.streaming.task.TaskContext + +import scala.concurrent.duration.FiniteDuration + +class TickSourceTask[T](context: TaskContext, userConf : UserConfig) + extends GraphTask(context, userConf) { + + val initialDelay = userConf.getValue[FiniteDuration](TickSourceTask.INITIAL_DELAY). + getOrElse(FiniteDuration(0, TimeUnit.MINUTES)) + (TickSourceTask.INITIAL_DELAY) + val interval = userConf.getValue[FiniteDuration](TickSourceTask.INTERVAL). + getOrElse(FiniteDuration(0, TimeUnit.MINUTES)) + val tick = userConf.getValue[T](TickSourceTask.TICK).get + + override def onStart(startTime: Instant): Unit = { + context.schedule(initialDelay, interval)( + self ! Message(tick, System.currentTimeMillis()) + ) + } + + override def onNext(msg : Message) : Unit = { + context.output(msg) + } +} + +object TickSourceTask { + val INITIAL_DELAY = "INITIAL_DELAY" + val INTERVAL = "INTERVAL" + val TICK = "TICK" +} diff --git a/experiments/akkastream/src/main/scala/akka/stream/gearpump/task/UnZip2Task.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/Unzip2Task.scala similarity index 61% rename from experiments/akkastream/src/main/scala/akka/stream/gearpump/task/UnZip2Task.scala rename to experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/Unzip2Task.scala index 78fabbe20..99f1b550f 100644 --- a/experiments/akkastream/src/main/scala/akka/stream/gearpump/task/UnZip2Task.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/Unzip2Task.scala @@ -7,7 +7,7 @@ * "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 + * 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, @@ -16,30 +16,31 @@ * limitations under the License. */ -package akka.stream.gearpump.task - -import akka.stream.gearpump.task.UnZip2Task.UnZipFunction +package org.apache.gearpump.akkastream.task +import org.apache.gearpump.akkastream.task.Unzip2Task.UnZipFunction import org.apache.gearpump.Message import org.apache.gearpump.cluster.UserConfig import org.apache.gearpump.streaming.task.TaskContext -class UnZip2Task(context: TaskContext, userConf: UserConfig) extends GraphTask(context, userConf) { +class Unzip2Task[In, A1, A2](context: TaskContext, userConf : UserConfig) + extends GraphTask(context, userConf) { - val unzip = userConf.getValue[UnZipFunction](UnZip2Task.UNZIP2_FUNCTION)(context.system).get.unzip + val unzip = userConf. + getValue[UnZipFunction[In, A1, A2]](Unzip2Task.UNZIP2_FUNCTION)(context.system).get.unzip - override def onNext(msg: Message): Unit = { + override def onNext(msg : Message) : Unit = { val message = msg.msg val time = msg.timestamp - val pair = unzip(message) + val pair = unzip(message.asInstanceOf[In]) val (a, b) = pair output(0, Message(a.asInstanceOf[AnyRef], time)) output(1, Message(b.asInstanceOf[AnyRef], time)) } } -object UnZip2Task { - class UnZipFunction(val unzip: Any => (Any, Any)) extends Serializable +object Unzip2Task { + case class UnZipFunction[In, A1, A2](val unzip: In => (A1, A2)) extends Serializable - val UNZIP2_FUNCTION = "akka.stream.gearpump.task.unzip2.function" -} \ No newline at end of file + val UNZIP2_FUNCTION = "org.apache.gearpump.akkastream.task.unzip2.function" +} diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/Zip2Task.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/Zip2Task.scala new file mode 100644 index 000000000..a35b1332a --- /dev/null +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/Zip2Task.scala @@ -0,0 +1,57 @@ +/* + * 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.gearpump.akkastream.task + +import org.apache.gearpump.Message +import org.apache.gearpump.akkastream.task.Zip2Task.ZipFunction +import org.apache.gearpump.cluster.UserConfig +import org.apache.gearpump.streaming.task.TaskContext + +class Zip2Task[A1, A2, OUT](context: TaskContext, userConf : UserConfig) + extends GraphTask(context, userConf) { + + val zip = userConf. + getValue[ZipFunction[A1, A2, OUT]](Zip2Task.ZIP2_FUNCTION)(context.system).get.zip + var a1: Option[A1] = None + var a2: Option[A2] = None + + override def onNext(msg : Message) : Unit = { + val message = msg.msg + val time = msg.timestamp + a1 match { + case Some(x) => + a2 = Some(message.asInstanceOf[A2]) + a1.foreach(v1 => { + a2.foreach(v2 => { + val out = zip(v1, v2) + context.output(Message(out.asInstanceOf[OUT], time)) + + }) + }) + case None => + a1 = Some(message.asInstanceOf[A1]) + } + } +} + +object Zip2Task { + case class ZipFunction[A1, A2, OUT](val zip: (A1, A2) => OUT) extends Serializable + + val ZIP2_FUNCTION = "org.apache.gearpump.akkastream.task.zip2.function" +} diff --git a/experiments/akkastream/src/main/scala/akka/stream/gearpump/util/MaterializedValueOps.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/util/MaterializedValueOps.scala similarity index 75% rename from experiments/akkastream/src/main/scala/akka/stream/gearpump/util/MaterializedValueOps.scala rename to experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/util/MaterializedValueOps.scala index c774fc7c7..c9fe67de7 100644 --- a/experiments/akkastream/src/main/scala/akka/stream/gearpump/util/MaterializedValueOps.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/util/MaterializedValueOps.scala @@ -7,7 +7,7 @@ * "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 + * 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, @@ -16,15 +16,17 @@ * limitations under the License. */ -package akka.stream.gearpump.util +package org.apache.gearpump.akkastream.util import akka.stream.impl.StreamLayout.{Atomic, Combine, Ignore, MaterializedValueNode, Module, Transform} class MaterializedValueOps(mat: MaterializedValueNode) { - def resolve[Mat](materializedValues: Map[Module, Any]): Mat = { - def resolveMaterialized(mat: MaterializedValueNode, materializedValues: Map[Module, Any]): Any = mat match { + def resolve[Mat](materializedValues: scala.collection.mutable.Map[Module, Any]): Mat = { + def resolveMaterialized(mat: MaterializedValueNode, + materializedValues: scala.collection.mutable.Map[Module, Any]): Any = mat match { case Atomic(m) => materializedValues.getOrElse(m, ()) - case Combine(f, d1, d2) => f(resolveMaterialized(d1, materializedValues), resolveMaterialized(d2, materializedValues)) + case Combine(f, d1, d2) => f(resolveMaterialized(d1, materializedValues), + resolveMaterialized(d2, materializedValues)) case Transform(f, d) => f(resolveMaterialized(d, materializedValues)) case Ignore => () } @@ -32,7 +34,7 @@ class MaterializedValueOps(mat: MaterializedValueNode) { } } -object MaterializedValueOps { +object MaterializedValueOps{ def apply(mat: MaterializedValueNode): MaterializedValueOps = new MaterializedValueOps(mat) } diff --git a/experiments/akkastream/src/test/scala/akka/stream/gearpump/AttributesSpec.scala b/experiments/akkastream/src/test/scala/org/apache/gearpump/akkastream/AttributesSpec.scala similarity index 92% rename from experiments/akkastream/src/test/scala/akka/stream/gearpump/AttributesSpec.scala rename to experiments/akkastream/src/test/scala/org/apache/gearpump/akkastream/AttributesSpec.scala index 4ead8396e..3731d41a1 100644 --- a/experiments/akkastream/src/test/scala/akka/stream/gearpump/AttributesSpec.scala +++ b/experiments/akkastream/src/test/scala/org/apache/gearpump/akkastream/AttributesSpec.scala @@ -7,7 +7,7 @@ * "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 + * 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, @@ -16,7 +16,7 @@ * limitations under the License. */ -package akka.stream.gearpump +package org.apache.gearpump.akkastream import akka.stream.Attributes import org.scalatest.{FlatSpec, Matchers} @@ -30,4 +30,5 @@ class AttributesSpec extends FlatSpec with Matchers { assert("aa-bb" == c.nameOrDefault()) } + } diff --git a/project/Build.scala b/project/Build.scala index 17e78dfd1..0b1628edc 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -35,7 +35,8 @@ object Build extends sbt.Build { val copySharedSourceFiles = TaskKey[Unit]("copied shared services source code") - val akkaVersion = "2.4.3" + val akkaVersion = "2.4.10" + val akkaStreamVersion = "2.4-SNAPSHOT" val apacheRepo = "https://repository.apache.org/" val hadoopVersion = "2.6.0" val hbaseVersion = "1.0.0" @@ -145,9 +146,12 @@ object Build extends sbt.Build { "com.typesafe.akka" %% "akka-cluster" % akkaVersion, "com.typesafe.akka" %% "akka-cluster-tools" % akkaVersion, "commons-logging" % "commons-logging" % commonsLoggingVersion, - "com.typesafe.akka" %% "akka-distributed-data-experimental" % akkaVersion, + "com.typesafe.akka" %% "akka-distributed-data-experimental" % akkaVersion + exclude("com.typesafe.akka", "akka-stream_2.11"), + "com.typesafe.akka" %% "akka-stream" % akkaStreamVersion, "org.apache.hadoop" % "hadoop-common" % hadoopVersion % "provided" - ) + ), + dependencyOverrides += "com.typesafe.akka" %% "akka-stream" % akkaStreamVersion ) val streamingDependencies = Seq( @@ -186,7 +190,8 @@ object Build extends sbt.Build { "com.typesafe.akka" %% "akka-http-spray-json-experimental" % akkaVersion, "org.scala-lang" % "scala-reflect" % scalaVersionNumber, "org.scala-lang.modules" %% "scala-parser-combinators" % "1.0.4", - "com.typesafe.akka" %% "akka-testkit" % akkaVersion % "test", + "com.typesafe.akka" %% "akka-testkit" % akkaVersion % "test" + exclude("com.typesafe.akka", "akka-stream_2.11"), "org.scalatest" %% "scalatest" % scalaTestVersion % "test", "org.scalacheck" %% "scalacheck" % scalaCheckVersion % "test", "org.mockito" % "mockito-core" % mockitoVersion % "test", @@ -250,7 +255,7 @@ object Build extends sbt.Build { base = file("."), settings = commonSettings ++ noPublish ++ gearpumpUnidocSetting) .aggregate(shaded, core, daemon, streaming, services, external_kafka, external_monoid, - external_serializer, examples, storm, yarn, external_hbase, packProject, + external_serializer, examples, akkastream, storm, yarn, external_hbase, packProject, external_hadoopfs, integration_test).settings(Defaults.itSettings: _*) .disablePlugins(sbtassembly.AssemblyPlugin) @@ -314,14 +319,17 @@ object Build extends sbt.Build { lazy val serviceJvmSettings = commonSettings ++ noPublish ++ Seq( libraryDependencies ++= Seq( - "com.typesafe.akka" %% "akka-http-testkit" % akkaVersion % "test", + "com.typesafe.akka" %% "akka-http-testkit" % akkaVersion % "test" + exclude("com.typesafe.akka", "akka-stream_2.11"), "org.scalatest" %% "scalatest" % scalaTestVersion % "test", "com.lihaoyi" %% "upickle" % upickleVersion, "com.softwaremill.akka-http-session" %% "core" % "0.2.5", - "com.typesafe.akka" %% "akka-http-spray-json-experimental" % akkaVersion, + "com.typesafe.akka" %% "akka-http-spray-json-experimental" % akkaVersion + exclude("com.typesafe.akka", "akka-stream_2.11"), "com.github.scribejava" % "scribejava-apis" % "2.4.0", "com.ning" % "async-http-client" % "1.9.33", "org.webjars" % "angularjs" % "1.4.9", + "org.apache.hadoop" % "hadoop-common" % hadoopVersion, // angular 1.5 breaks ui-select, but we need ng-touch 1.5 "org.webjars.npm" % "angular-touch" % "1.5.0", @@ -382,14 +390,17 @@ object Build extends sbt.Build { lazy val akkastream = Project( id = "gearpump-experiments-akkastream", base = file("experiments/akkastream"), - settings = commonSettings ++ noPublish ++ myAssemblySettings ++ + settings = commonSettings ++ noPublish ++ Seq( libraryDependencies ++= Seq( - "org.json4s" %% "json4s-jackson" % "3.2.11" + "com.typesafe.akka" %% "akka-stream" % akkaStreamVersion, + "org.json4s" %% "json4s-jackson" % "3.2.11", + "org.scalatest" %% "scalatest" % scalaTestVersion % "test" ), - mainClass in(Compile, packageBin) := Some("akka.stream.gearpump.example.Test") - )) - .dependsOn(streaming % "test->test; provided", daemon % "test->test; provided") + dependencyOverrides += "com.typesafe.akka" %% "akka-stream" % akkaStreamVersion + )) + .dependsOn (services % "test->test; compile->compile", daemon % "test->test; compile->compile") + .disablePlugins(sbtassembly.AssemblyPlugin) lazy val storm = Project( id = "gearpump-experiments-storm", diff --git a/project/Pack.scala b/project/Pack.scala index 1c87653e1..47d30647b 100644 --- a/project/Pack.scala +++ b/project/Pack.scala @@ -69,7 +69,8 @@ object Pack extends sbt.Build { "worker" -> "org.apache.gearpump.cluster.main.Worker", "services" -> "org.apache.gearpump.services.main.Services", "yarnclient" -> "org.apache.gearpump.experiments.yarn.client.Client", - "storm" -> "org.apache.gearpump.experiments.storm.StormRunner" + "storm" -> "org.apache.gearpump.experiments.storm.StormRunner", + "akkastream" -> "org.apache.gearpump.akkastream.example.Test11" ), packJvmOpts := Map( "gear" -> Seq("-Djava.net.preferIPv4Stack=true", "-Dgearpump.home=${PROG_HOME}"), @@ -109,7 +110,13 @@ object Pack extends sbt.Build { "storm" -> Seq( "-server", "-Djava.net.preferIPv4Stack=true", - "-Dgearpump.home=${PROG_HOME}") + "-Dgearpump.home=${PROG_HOME}"), + + "akkastream" -> Seq( + "-server", + "-Djava.net.preferIPv4Stack=true", + "-Dgearpump.home=${PROG_HOME}", + "-Djava.rmi.server.hostname=localhost") ), packLibDir := Map( "lib" -> new ProjectsToPack(core.id, streaming.id), @@ -141,13 +148,14 @@ object Pack extends sbt.Build { "worker" -> daemonClassPath, "services" -> serviceClassPath, "yarnclient" -> yarnClassPath, - "storm" -> stormClassPath + "storm" -> stormClassPath, + "akkstream" -> daemonClassPath ), packArchivePrefix := projectName + "-" + scalaBinaryVersion.value, packArchiveExcludes := Seq("integrationtest") ) - ).dependsOn(core, streaming, services, yarn, storm). + ).dependsOn(core, streaming, services, yarn, storm, akkastream). disablePlugins(sbtassembly.AssemblyPlugin) } diff --git a/scalastyle-config.xml b/project/scalastyle_config.xml similarity index 100% rename from scalastyle-config.xml rename to project/scalastyle_config.xml diff --git a/services/jvm/src/main/scala/org/apache/gearpump/services/AppMasterService.scala b/services/jvm/src/main/scala/org/apache/gearpump/services/AppMasterService.scala index b21736337..46e16cf11 100644 --- a/services/jvm/src/main/scala/org/apache/gearpump/services/AppMasterService.scala +++ b/services/jvm/src/main/scala/org/apache/gearpump/services/AppMasterService.scala @@ -149,7 +149,7 @@ class AppMasterService(val master: ActorRef, } } } ~ - path("metrics" / RestPath) { path => + path("metrics" / RemainingPath) { path => parameterMap { optionMap => parameter("aggregator" ? "") { aggregator => parameter(ReadOption.Key ? ReadOption.ReadLatest) { readOption => diff --git a/services/jvm/src/main/scala/org/apache/gearpump/services/MasterService.scala b/services/jvm/src/main/scala/org/apache/gearpump/services/MasterService.scala index bf7092e44..a763be694 100644 --- a/services/jvm/src/main/scala/org/apache/gearpump/services/MasterService.scala +++ b/services/jvm/src/main/scala/org/apache/gearpump/services/MasterService.scala @@ -102,7 +102,7 @@ class MasterService(val master: ActorRef, failWith(ex) } } ~ - path("metrics" / RestPath) { path => + path("metrics" / RemainingPath) { path => parameters(ParamMagnet(ReadOption.Key ? ReadOption.ReadLatest)) { readOption: String => val query = QueryHistoryMetrics(path.head.toString, readOption) onComplete(askActor[HistoryMetrics](master, query)) { diff --git a/services/jvm/src/main/scala/org/apache/gearpump/services/SecurityService.scala b/services/jvm/src/main/scala/org/apache/gearpump/services/SecurityService.scala index 804b34fbe..8ae8dbee0 100644 --- a/services/jvm/src/main/scala/org/apache/gearpump/services/SecurityService.scala +++ b/services/jvm/src/main/scala/org/apache/gearpump/services/SecurityService.scala @@ -60,8 +60,8 @@ import org.apache.gearpump.services.util.UpickleUtil._ class SecurityService(inner: RouteService, implicit val system: ActorSystem) extends RouteService { // Use scheme "GearpumpBasic" to avoid popping up web browser native authentication box. - private val challenge = HttpChallenge(scheme = "GearpumpBasic", realm = "gearpump", - params = Map.empty) + private val challenge = HttpChallenge(scheme = "GearpumpBasic", realm = Some("gearpump"), + params = Map.empty[String,String]) val LOG = LogUtil.getLogger(getClass, "AUDIT") diff --git a/services/jvm/src/main/scala/org/apache/gearpump/services/StaticService.scala b/services/jvm/src/main/scala/org/apache/gearpump/services/StaticService.scala index 284d3f2e5..7b3398709 100644 --- a/services/jvm/src/main/scala/org/apache/gearpump/services/StaticService.scala +++ b/services/jvm/src/main/scala/org/apache/gearpump/services/StaticService.scala @@ -19,10 +19,12 @@ package org.apache.gearpump.services import akka.actor.ActorSystem +import akka.http.scaladsl.marshalling.ToResponseMarshallable import akka.http.scaladsl.model._ import akka.http.scaladsl.server.Directives._ +import akka.http.scaladsl.marshalling.ToResponseMarshallable._ +import akka.http.scaladsl.server.{RejectionHandler, StandardRoute} import akka.stream.Materializer - import org.apache.gearpump.util.Util // NOTE: This cannot be removed!!! import org.apache.gearpump.services.util.UpickleUtil._ @@ -56,14 +58,14 @@ class StaticService(override val system: ActorSystem, supervisorPath: String) getFromResource("index.html") } ~ path("favicon.ico") { - complete(StatusCodes.NotFound) + complete(ToResponseMarshallable(StatusCodes.NotFound)) } ~ pathPrefix("webjars") { get { getFromResourceDirectory("META-INF/resources/webjars") } } ~ - path(Rest) { path => + path(Remaining) { path => getFromResource("%s" format path) } } diff --git a/services/jvm/src/main/scala/org/apache/gearpump/services/WorkerService.scala b/services/jvm/src/main/scala/org/apache/gearpump/services/WorkerService.scala index 8268d61f8..954fe97f8 100644 --- a/services/jvm/src/main/scala/org/apache/gearpump/services/WorkerService.scala +++ b/services/jvm/src/main/scala/org/apache/gearpump/services/WorkerService.scala @@ -63,7 +63,7 @@ class WorkerService(val master: ActorRef, override val system: ActorSystem) failWith(ex) } } ~ - path("metrics" / RestPath ) { path => + path("metrics" / RemainingPath ) { path => val workerId = WorkerId.parse(workerIdString) parameter(ReadOption.Key ? ReadOption.ReadLatest) { readOption => val query = QueryHistoryMetrics(path.head.toString, readOption) diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/OpTranslator.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/OpTranslator.scala index 8de291c59..b09d9b9cb 100644 --- a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/OpTranslator.scala +++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/OpTranslator.scala @@ -64,7 +64,7 @@ class OpTranslator extends java.io.Serializable { userConfig) case ProcessorOp(processor, parallelism, conf, description) => DefaultProcessor(parallelism, - description = description + "." + func.description, + description = description + " " + func.description, userConfig, processor) case DataSinkOp(dataSink, parallelism, conf, description) => DataSinkProcessor(dataSink, parallelism, description + func.description) diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/task/TaskActor.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/task/TaskActor.scala index c0b6a29c7..eb52700ab 100644 --- a/streaming/src/main/scala/org/apache/gearpump/streaming/task/TaskActor.scala +++ b/streaming/src/main/scala/org/apache/gearpump/streaming/task/TaskActor.scala @@ -23,8 +23,6 @@ import java.util import java.util.concurrent.TimeUnit import akka.actor._ -import org.apache.gearpump.streaming.source.{Watermark, DataSourceTask} -import org.slf4j.Logger import org.apache.gearpump.cluster.UserConfig import org.apache.gearpump.gs.collections.impl.map.mutable.primitive.IntShortHashMap import org.apache.gearpump.metrics.Metrics @@ -32,8 +30,10 @@ import org.apache.gearpump.serializer.SerializationFramework import org.apache.gearpump.streaming.AppMasterToExecutor._ import org.apache.gearpump.streaming.ExecutorToAppMaster._ import org.apache.gearpump.streaming.ProcessorId +import org.apache.gearpump.streaming.source.Watermark import org.apache.gearpump.util.{LogUtil, TimeOutScheduler} import org.apache.gearpump.{Message, TimeStamp} +import org.slf4j.Logger /** * @@ -52,16 +52,15 @@ class TaskActor( def serializerPool: SerializationFramework = inputSerializerPool - import taskContextData._ - import org.apache.gearpump.streaming.Constants._ import org.apache.gearpump.streaming.task.TaskActor._ + import taskContextData._ val config = context.system.settings.config val LOG: Logger = LogUtil.getLogger(getClass, app = appId, executor = executorId, task = taskId) // Metrics - private val metricName = s"app${appId}.processor${taskId.processorId}.task${taskId.index}" + private val metricName = s"app$appId.processor${taskId.processorId}.task${taskId.index}" private val receiveLatency = Metrics(context.system).histogram( s"$metricName:receiveLatency", sampleRate = 1) private val processTime = Metrics(context.system).histogram(s"$metricName:processTime") @@ -76,9 +75,9 @@ class TaskActor( private var life = taskContextData.life // Latency probe - import scala.concurrent.duration._ - import context.dispatcher + + import scala.concurrent.duration._ final val LATENCY_PROBE_INTERVAL = FiniteDuration(1, TimeUnit.SECONDS) // Clock report interval From bc39403525b4065360acf82386fac21a588b59e6 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Tue, 11 Oct 2016 11:57:48 +0800 Subject: [PATCH 2/9] Merge branch 'master' into akka-streams Author: manuzhang Author: darionyaphet Closes #95 from manuzhang/akka-streams. --- .../wordcount/dsl/WindowedWordCount.scala | 87 ++++ .../apache/gearpump/redis/RedisMessage.scala | 456 ++++++++++++++++++ .../org/apache/gearpump/redis/RedisSink.scala | 119 +++++ project/Build.scala | 62 ++- project/BuildShaded.scala | 127 ++--- .../apache/gearpump/streaming/Constants.scala | 1 + .../streaming/StreamApplication.scala | 2 +- .../gearpump/streaming/dsl/Stream.scala | 106 ++-- .../gearpump/streaming/dsl/StreamApp.scala | 34 +- .../streaming/dsl/javaapi/JavaStream.scala | 22 +- .../apache/gearpump/streaming/dsl/op/OP.scala | 109 ----- ...itioner.scala => GroupByPartitioner.scala} | 13 +- .../gearpump/streaming/dsl/plan/OP.scala | 214 ++++++++ .../streaming/dsl/plan/OpTranslator.scala | 222 --------- .../gearpump/streaming/dsl/plan/Planner.scala | 65 ++- .../plan/functions/SingleInputFunction.scala | 107 ++++ .../streaming/dsl/task/CountTriggerTask.scala | 63 +++ .../dsl/task/EventTimeTriggerTask.scala | 59 +++ .../dsl/task/ProcessingTimeTriggerTask.scala | 82 ++++ .../streaming/dsl/task/TransformTask.scala | 47 ++ .../dsl/window/api/AccumulationMode.scala | 24 + .../streaming/dsl/window/api/GroupByFn.scala | 47 ++ .../streaming/dsl/window/api/Trigger.scala | 27 ++ .../streaming/dsl/window/api/Window.scala | 77 +++ .../streaming/dsl/window/api/WindowFn.scala | 63 +++ .../dsl/window/impl/ReduceFnRunner.scala | 29 ++ .../streaming/dsl/window/impl/Window.scala | 75 +++ .../dsl/window/impl/WindowRunner.scala | 114 +++++ .../streaming/source/DataSourceTask.scala | 15 +- .../gearpump/streaming/task/TaskActor.scala | 4 +- .../streaming/dsl/StreamAppSpec.scala | 67 +-- .../gearpump/streaming/dsl/StreamSpec.scala | 24 +- .../partitioner/GroupByPartitionerSpec.scala | 23 +- .../gearpump/streaming/dsl/plan/OpSpec.scala | 244 ++++++++++ .../streaming/dsl/plan/OpTranslatorSpec.scala | 148 ------ .../streaming/dsl/plan/PlannerSpec.scala | 132 +++++ .../functions/SingleInputFunctionSpec.scala | 333 +++++++++++++ .../dsl/task/CountTriggerTaskSpec.scala | 61 +++ .../dsl/task/EventTimeTriggerTaskSpec.scala | 66 +++ .../task/ProcessingTimeTriggerTaskSpec.scala | 69 +++ 40 files changed, 2896 insertions(+), 743 deletions(-) create mode 100644 examples/streaming/wordcount/src/main/scala/org/apache/gearpump/streaming/examples/wordcount/dsl/WindowedWordCount.scala create mode 100644 experiments/redis/src/main/scala/org/apache/gearpump/redis/RedisMessage.scala create mode 100644 experiments/redis/src/main/scala/org/apache/gearpump/redis/RedisSink.scala delete mode 100644 streaming/src/main/scala/org/apache/gearpump/streaming/dsl/op/OP.scala rename streaming/src/main/scala/org/apache/gearpump/streaming/dsl/partitioner/{GroupbyPartitioner.scala => GroupByPartitioner.scala} (77%) create mode 100644 streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/OP.scala delete mode 100644 streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/OpTranslator.scala create mode 100644 streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/functions/SingleInputFunction.scala create mode 100644 streaming/src/main/scala/org/apache/gearpump/streaming/dsl/task/CountTriggerTask.scala create mode 100644 streaming/src/main/scala/org/apache/gearpump/streaming/dsl/task/EventTimeTriggerTask.scala create mode 100644 streaming/src/main/scala/org/apache/gearpump/streaming/dsl/task/ProcessingTimeTriggerTask.scala create mode 100644 streaming/src/main/scala/org/apache/gearpump/streaming/dsl/task/TransformTask.scala create mode 100644 streaming/src/main/scala/org/apache/gearpump/streaming/dsl/window/api/AccumulationMode.scala create mode 100644 streaming/src/main/scala/org/apache/gearpump/streaming/dsl/window/api/GroupByFn.scala create mode 100644 streaming/src/main/scala/org/apache/gearpump/streaming/dsl/window/api/Trigger.scala create mode 100644 streaming/src/main/scala/org/apache/gearpump/streaming/dsl/window/api/Window.scala create mode 100644 streaming/src/main/scala/org/apache/gearpump/streaming/dsl/window/api/WindowFn.scala create mode 100644 streaming/src/main/scala/org/apache/gearpump/streaming/dsl/window/impl/ReduceFnRunner.scala create mode 100644 streaming/src/main/scala/org/apache/gearpump/streaming/dsl/window/impl/Window.scala create mode 100644 streaming/src/main/scala/org/apache/gearpump/streaming/dsl/window/impl/WindowRunner.scala create mode 100644 streaming/src/test/scala/org/apache/gearpump/streaming/dsl/plan/OpSpec.scala delete mode 100644 streaming/src/test/scala/org/apache/gearpump/streaming/dsl/plan/OpTranslatorSpec.scala create mode 100644 streaming/src/test/scala/org/apache/gearpump/streaming/dsl/plan/PlannerSpec.scala create mode 100644 streaming/src/test/scala/org/apache/gearpump/streaming/dsl/plan/functions/SingleInputFunctionSpec.scala create mode 100644 streaming/src/test/scala/org/apache/gearpump/streaming/dsl/task/CountTriggerTaskSpec.scala create mode 100644 streaming/src/test/scala/org/apache/gearpump/streaming/dsl/task/EventTimeTriggerTaskSpec.scala create mode 100644 streaming/src/test/scala/org/apache/gearpump/streaming/dsl/task/ProcessingTimeTriggerTaskSpec.scala diff --git a/examples/streaming/wordcount/src/main/scala/org/apache/gearpump/streaming/examples/wordcount/dsl/WindowedWordCount.scala b/examples/streaming/wordcount/src/main/scala/org/apache/gearpump/streaming/examples/wordcount/dsl/WindowedWordCount.scala new file mode 100644 index 000000000..4f43fd466 --- /dev/null +++ b/examples/streaming/wordcount/src/main/scala/org/apache/gearpump/streaming/examples/wordcount/dsl/WindowedWordCount.scala @@ -0,0 +1,87 @@ +/* + * 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.gearpump.streaming.examples.wordcount.dsl + +import java.time.{Duration, Instant} + +import org.apache.gearpump.Message +import org.apache.gearpump.cluster.client.ClientContext +import org.apache.gearpump.cluster.main.{ArgumentsParser, CLIOption} +import org.apache.gearpump.streaming.dsl.{LoggerSink, StreamApp} +import org.apache.gearpump.streaming.dsl.window.api.{EventTimeTrigger, FixedWindow} +import org.apache.gearpump.streaming.source.DataSource +import org.apache.gearpump.streaming.task.TaskContext +import org.apache.gearpump.util.AkkaApp + +object WindowedWordCount extends AkkaApp with ArgumentsParser { + + override val options: Array[(String, CLIOption[Any])] = Array.empty + + override def main(akkaConf: Config, args: Array[String]): Unit = { + val context = ClientContext(akkaConf) + val app = StreamApp("dsl", context) + app.source[String](new TimedDataSource). + // word => (word, count) + flatMap(line => line.split("[\\s]+")).map((_, 1)). + // fix window + window(FixedWindow.apply(Duration.ofMillis(5L)) + .triggering(EventTimeTrigger)). + // (word, count1), (word, count2) => (word, count1 + count2) + groupBy(_._1). + sum.sink(new LoggerSink) + + context.submit(app) + context.close() + } + + private class TimedDataSource extends DataSource { + + private var data = List( + Message("foo", 1L), + Message("bar", 2L), + Message("foo", 3L), + Message("foo", 5L), + Message("bar", 7L), + Message("bar", 8L) + ) + + private var watermark: Instant = Instant.ofEpochMilli(0) + + override def read(): Message = { + if (data.nonEmpty) { + val msg = data.head + data = data.tail + watermark = Instant.ofEpochMilli(msg.timestamp) + msg + } else { + null + } + } + + override def open(context: TaskContext, startTime: Instant): Unit = {} + + override def close(): Unit = {} + + override def getWatermark: Instant = { + if (data.isEmpty) { + watermark = watermark.plusMillis(1) + } + watermark + } + } +} diff --git a/experiments/redis/src/main/scala/org/apache/gearpump/redis/RedisMessage.scala b/experiments/redis/src/main/scala/org/apache/gearpump/redis/RedisMessage.scala new file mode 100644 index 000000000..84dec70be --- /dev/null +++ b/experiments/redis/src/main/scala/org/apache/gearpump/redis/RedisMessage.scala @@ -0,0 +1,456 @@ +/* + * 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.gearpump.redis + +import java.nio.charset.Charset + +object RedisMessage { + + private def toBytes(strings: List[String]): List[Array[Byte]] = + strings.map(string => string.getBytes(Charset.forName("UTF8"))) + + private def toBytes(string: String): Array[Byte] = + string.getBytes(Charset.forName("UTF8")) + + object Connection { + + /** + * Change the selected database for the current connection + * + * @param index + */ + case class SELECT(index: Int) + + } + + object Geo { + + /** + * Add one geospatial item in the geospatial index represented using a sorted set + * + * @param key + * @param longitude + * @param latitude + * @param member + */ + case class GEOADD(key: Array[Byte], longitude: Double, + latitude: Double, member: Array[Byte]) { + def this(key: String, longitude: Double, + latitude: Double, member: String) = + this(toBytes(key), longitude, latitude, toBytes(member)) + } + + } + + object Hashes { + + /** + * Delete a hash field + * + * @param key + * @param field + */ + case class HDEL(key: Array[Byte], field: Array[Byte]) { + def this(key: String, field: String) = this(toBytes(key), toBytes(field)) + } + + /** + * Increment the integer value of a hash field by the given number + * + * @param key + * @param field + * @param increment + */ + case class HINCRBY(key: Array[Byte], field: Array[Byte], increment: Long) { + def this(key: String, field: String, increment: Long) = + this(toBytes(key), toBytes(field), increment) + } + + /** + * Increment the float value of a hash field by the given amount + * + * @param key + * @param field + * @param increment + */ + case class HINCRBYFLOAT(key: Array[Byte], field: Array[Byte], increment: Float) { + def this(key: String, field: String, increment: Float) = + this(toBytes(key), toBytes(field), increment) + } + + + /** + * Set the string value of a hash field + * + * @param key + * @param field + * @param value + */ + case class HSET(key: Array[Byte], field: Array[Byte], value: Array[Byte]) { + def this(key: String, field: String, value: String) = + this(toBytes(key), toBytes(field), toBytes(value)) + } + + /** + * Set the value of a hash field, only if the field does not exist + * + * @param key + * @param field + * @param value + */ + case class HSETNX(key: Array[Byte], field: Array[Byte], value: Array[Byte]) { + def this(key: String, field: String, value: String) = + this(toBytes(key), toBytes(field), toBytes(value)) + } + + } + + object HyperLogLog { + + /** + * Adds the specified elements to the specified HyperLogLog + * + * @param key + * @param element + */ + case class PFADD(key: String, element: String) + + } + + object Lists { + + + /** + * Prepend one or multiple values to a list + * + * @param key + * @param value + */ + case class LPUSH(key: Array[Byte], value: Array[Byte]) { + + def this(key: String, value: String) = this(key, toBytes(value)) + } + + /** + * Prepend a value to a list, only if the list exists + * + * @param key + * @param value + */ + case class LPUSHX(key: Array[Byte], value: Array[Byte]) { + def this(key: String, value: String) = this(toBytes(key), toBytes(value)) + } + + /** + * Set the value of an element in a list by its index + * + * @param key + * @param index + * @param value + */ + case class LSET(key: Array[Byte], index: Long, value: Array[Byte]) { + def this(key: String, index: Long, value: String) = this(toBytes(key), index, toBytes(value)) + } + + /** + * Append one or multiple values to a list + * + * @param key + * @param value + */ + case class RPUSH(key: Array[Byte], value: Array[Byte]) { + + def this(key: String, value: String) = this(key, toBytes(value)) + } + + /** + * Append a value to a list, only if the list exists + * + * @param key + * @param value + */ + case class RPUSHX(key: Array[Byte], value: Array[Byte]) { + def this(key: String, value: String) = this(toBytes(key), toBytes(value)) + } + + } + + object Keys { + + /** + * Delete a key + * + * @param message + */ + case class DEL(message: Array[Byte]) { + + def this(message: String) = this(toBytes(message)) + } + + /** + * Set a key's time to live in seconds + * + * @param key + */ + case class EXPIRE(key: Array[Byte], seconds: Int) { + def this(key: String, seconds: Int) = this(toBytes(key), seconds) + } + + /** + * Set the expiration for a key as a UNIX timestamp + * + * @param key + * @param timestamp + */ + case class EXPIREAT(key: Array[Byte], timestamp: Long) { + def this(key: String, timestamp: Long) = this(toBytes(key), timestamp) + } + + /** + * Atomically transfer a key from a Redis instance to another one. + * + * @param host + * @param port + * @param key + * @param database + * @param timeout + */ + case class MIGRATE(host: Array[Byte], port: Int, key: Array[Byte], database: Int, timeout: Int) { + def this(host: String, port: Int, key: String, database: Int, timeout: Int) = + this(toBytes(host), port, toBytes(key), database, timeout) + } + + /** + * Move a key to another database + * + * @param key + * @param db + */ + case class MOVE(key: Array[Byte], db: Int) { + def this(key: String, db: Int) = this(toBytes(key), db) + } + + /** + * Remove the expiration from a key + * + * @param key + */ + case class PERSIST(key: Array[Byte]) { + def this(key: String) = this(toBytes(key)) + } + + /** + * Set a key's time to live in milliseconds + * + * @param key + * @param milliseconds + */ + case class PEXPIRE(key: Array[Byte], milliseconds: Long) { + def this(key: String, milliseconds: Long) = this(toBytes(key), milliseconds) + } + + /** + * Set the expiration for a key as a UNIX timestamp specified in milliseconds + * + * @param key + * @param timestamp + */ + case class PEXPIREAT(key: Array[Byte], timestamp: Long) { + def this(key: String, milliseconds: Long) = this(toBytes(key), milliseconds) + } + + /** + * Rename a key + * + * @param key + * @param newKey + */ + case class RENAME(key: Array[Byte], newKey: Array[Byte]) { + def this(key: String, newKey: String) = this(toBytes(key), toBytes(newKey)) + } + + /** + * Rename a key, only if the new key does not exist + * + * @param key + * @param newKey + */ + case class RENAMENX(key: Array[Byte], newKey: Array[Byte]) { + def this(key: String, newKey: String) = this(toBytes(key), toBytes(newKey)) + } + + } + + + object Sets { + + /** + * Add one or more members to a set + * + * @param key + * @param members + */ + case class SADD(key: Array[Byte], members: Array[Byte]) { + + def this(key: String, members: String) = this(key, toBytes(members)) + } + + + /** + * Move a member from one set to another + * + * @param source + * @param destination + * @param member + */ + case class SMOVE(source: Array[Byte], destination: Array[Byte], member: Array[Byte]) { + def this(source: String, destination: String, member: String) = + this(toBytes(source), toBytes(destination), toBytes(member)) + } + + + /** + * Remove one or more members from a set + * + * @param key + * @param member + */ + case class SREM(key: Array[Byte], member: Array[Byte]) { + + def this(key: String, member: String) = this(key, toBytes(member)) + } + + } + + object String { + + /** + * Append a value to a key + * + * @param key + * @param value + */ + case class APPEND(key: Array[Byte], value: Array[Byte]) { + def this(key: String, value: String) = this(toBytes(key), toBytes(value)) + } + + /** + * Decrement the integer value of a key by one + * + * @param key + */ + case class DECR(key: Array[Byte]) { + def this(key: String) = this(toBytes(key)) + } + + /** + * Decrement the integer value of a key by the given number + * + * @param key + * @param decrement + */ + case class DECRBY(key: Array[Byte], decrement: Int) { + def this(key: String, decrement: Int) = this(toBytes(key), decrement) + } + + /** + * Increment the integer value of a key by one + * + * @param key + */ + case class INCR(key: Array[Byte]) { + def this(key: String) = this(toBytes(key)) + } + + /** + * Increment the integer value of a key by the given amount + * + * @param key + * @param increment + */ + case class INCRBY(key: Array[Byte], increment: Int) { + def this(key: String, increment: Int) = this(toBytes(key), increment) + } + + /** + * Increment the float value of a key by the given amount + * + * @param key + * @param increment + */ + case class INCRBYFLOAT(key: Array[Byte], increment: Double) { + def this(key: String, increment: Number) = this(toBytes(key), increment) + } + + + /** + * Set the string value of a key + * + * @param key + * @param value + */ + case class SET(key: Array[Byte], value: Array[Byte]) { + def this(key: String, value: String) = this(toBytes(key), toBytes(value)) + } + + /** + * Sets or clears the bit at offset in the string value stored at key + * + * @param key + * @param offset + * @param value + */ + case class SETBIT(key: Array[Byte], offset: Long, value: Array[Byte]) { + def this(key: String, offset: Long, value: String) = this(toBytes(key), offset, toBytes(value)) + } + + /** + * Set the value and expiration of a key + * + * @param key + * @param seconds + * @param value + */ + case class SETEX(key: Array[Byte], seconds: Int, value: Array[Byte]) { + def this(key: String, seconds: Int, value: String) = this(toBytes(key), seconds, toBytes(value)) + } + + /** + * Set the value of a key, only if the key does not exist + * + * @param key + * @param value + */ + case class SETNX(key: Array[Byte], value: Array[Byte]) { + def this(key: String, value: String) = this(toBytes(key), toBytes(value)) + } + + /** + * Overwrite part of a string at key starting at the specified offset + * + * @param key + * @param offset + * @param value + */ + case class SETRANGE(key: Array[Byte], offset: Int, value: Array[Byte]) { + def this(key: String, offset: Int, value: String) = this(toBytes(key), offset, toBytes(value)) + } + + } + +} diff --git a/experiments/redis/src/main/scala/org/apache/gearpump/redis/RedisSink.scala b/experiments/redis/src/main/scala/org/apache/gearpump/redis/RedisSink.scala new file mode 100644 index 000000000..3f7594907 --- /dev/null +++ b/experiments/redis/src/main/scala/org/apache/gearpump/redis/RedisSink.scala @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gearpump.redis + +import org.apache.gearpump.Message +import org.apache.gearpump.redis.RedisMessage.Geo.GEOADD +import org.apache.gearpump.redis.RedisMessage.Hashes._ +import org.apache.gearpump.redis.RedisMessage.HyperLogLog._ +import org.apache.gearpump.redis.RedisMessage.Keys._ +import org.apache.gearpump.redis.RedisMessage.Lists._ +import org.apache.gearpump.redis.RedisMessage.Sets._ +import org.apache.gearpump.redis.RedisMessage.String._ +import org.apache.gearpump.streaming.sink.DataSink +import org.apache.gearpump.streaming.task.TaskContext +import org.apache.gearpump.util.LogUtil +import redis.clients.jedis.Jedis +import redis.clients.jedis.Protocol.{DEFAULT_DATABASE, DEFAULT_HOST, DEFAULT_PORT, DEFAULT_TIMEOUT} + +/** + * Save message in Redis Instance + * + * @param host + * @param port + * @param timeout + * @param database + * @param password + */ +class RedisSink( + host: String = DEFAULT_HOST, + port: Int = DEFAULT_PORT, + timeout: Int = DEFAULT_TIMEOUT, + database: Int = DEFAULT_DATABASE, + password: String = "") extends DataSink { + + private val LOG = LogUtil.getLogger(getClass) + @transient private lazy val client = new Jedis(host, port, timeout) + + override def open(context: TaskContext): Unit = { + client.select(database) + + if (password != null && password.length != 0) { + client.auth(password) + } + } + + override def write(message: Message): Unit = { + + message.msg match { + // GEO + case msg: GEOADD => client.geoadd(msg.key, msg.longitude, msg.latitude, msg.member) + + // Hashes + case msg: HDEL => client.hdel(msg.key, msg.field) + case msg: HINCRBY => client.hincrBy(msg.key, msg.field, msg.increment) + case msg: HINCRBYFLOAT => client.hincrByFloat(msg.key, msg.field, msg.increment) + case msg: HSET => client.hset(msg.key, msg.field, msg.value) + case msg: HSETNX => client.hsetnx(msg.key, msg.field, msg.value) + + // HyperLogLog + case msg: PFADD => client.pfadd(msg.key, msg.element) + + // Lists + case msg: LPUSH => client.lpush(msg.key, msg.value) + case msg: LPUSHX => client.lpushx(msg.key, msg.value) + case msg: LSET => client.lset(msg.key, msg.index, msg.value) + case msg: RPUSH => client.rpush(msg.key, msg.value) + case msg: RPUSHX => client.rpushx(msg.key, msg.value) + + // Keys + case msg: DEL => client.del(msg.message) + case msg: EXPIRE => client.expire(msg.key, msg.seconds) + case msg: EXPIREAT => client.expireAt(msg.key, msg.timestamp) + case msg: MIGRATE => client.migrate(msg.host, msg.port, msg.key, msg.database, msg.timeout) + case msg: MOVE => client.move(msg.key, msg.db) + case msg: PERSIST => client.persist(msg.key) + case msg: PEXPIRE => client.pexpire(msg.key, msg.milliseconds) + case msg: PEXPIREAT => client.pexpireAt(msg.key, msg.timestamp) + case msg: RENAME => client.rename(msg.key, msg.newKey) + case msg: RENAMENX => client.renamenx(msg.key, msg.newKey) + + // Sets + case msg: SADD => client.sadd(msg.key, msg.members) + case msg: SMOVE => client.smove(msg.source, msg.destination, msg.member) + case msg: SREM => client.srem(msg.key, msg.member) + + // String + case msg: APPEND => client.append(msg.key, msg.value) + case msg: DECR => client.decr(msg.key) + case msg: DECRBY => client.decrBy(msg.key, msg.decrement) + case msg: INCR => client.incr(msg.key) + case msg: INCRBY => client.incrBy(msg.key, msg.increment) + case msg: INCRBYFLOAT => client.incrByFloat(msg.key, msg.increment) + case msg: SET => client.set(msg.key, msg.value) + case msg: SETBIT => client.setbit(msg.key, msg.offset, msg.value) + case msg: SETEX => client.setex(msg.key, msg.seconds, msg.value) + case msg: SETNX => client.setnx(msg.key, msg.value) + case msg: SETRANGE => client.setrange(msg.key, msg.offset, msg.value) + } + } + + override def close(): Unit = { + client.close() + } +} diff --git a/project/Build.scala b/project/Build.scala index 0b1628edc..f1e0443d1 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -154,12 +154,6 @@ object Build extends sbt.Build { dependencyOverrides += "com.typesafe.akka" %% "akka-stream" % akkaStreamVersion ) - val streamingDependencies = Seq( - unmanagedJars in Compile ++= Seq( - getShadedJarFile("gs-collections", version.value) - ) - ) - val coreDependencies = Seq( libraryDependencies ++= Seq( "org.slf4j" % "slf4j-api" % slf4jVersion, @@ -199,9 +193,9 @@ object Build extends sbt.Build { ), unmanagedJars in Compile ++= Seq( - getShadedJarFile("metrics-graphite", version.value), - getShadedJarFile("guava", version.value), - getShadedJarFile("akka-kryo", version.value) + getShadedJarFile(shaded_metrics_graphite.id, version.value), + getShadedJarFile(shaded_guava.id, version.value), + getShadedJarFile(shaded_akka_kryo.id, version.value) ) ) @@ -250,6 +244,20 @@ object Build extends sbt.Build { .map(_.filterNot(_.getCanonicalPath.contains("akka"))) } + private def addShadedDeps(deps: Seq[xml.Node], node: xml.Node): xml.Node = { + node match { + case elem: xml.Elem => + val child = if (elem.label == "dependencies") { + elem.child ++ deps + } else { + elem.child.map(addShadedDeps(deps, _)) + } + xml.Elem(elem.prefix, elem.label, elem.attributes, elem.scope, false, child: _*) + case _ => + node + } + } + lazy val root = Project( id = "gearpump", base = file("."), @@ -262,7 +270,14 @@ object Build extends sbt.Build { lazy val core = Project( id = "gearpump-core", base = file("core"), - settings = commonSettings ++ javadocSettings ++ coreDependencies) + settings = commonSettings ++ javadocSettings ++ coreDependencies ++ Seq( + pomPostProcess := { + (node: xml.Node) => addShadedDeps(List( + getShadedDepXML(organization.value, shaded_akka_kryo.id, version.value), + getShadedDepXML(organization.value, shaded_guava.id, version.value), + getShadedDepXML(organization.value, shaded_metrics_graphite.id, version.value)), node) + } + )) .disablePlugins(sbtassembly.AssemblyPlugin) lazy val daemon = Project( @@ -282,9 +297,18 @@ object Build extends sbt.Build { lazy val streaming = Project( id = "gearpump-streaming", base = file("streaming"), - settings = commonSettings ++ javadocSettings ++ streamingDependencies) - .dependsOn(core % "test->test; compile->compile", daemon % "test->test") - .disablePlugins(sbtassembly.AssemblyPlugin) + settings = commonSettings ++ javadocSettings ++ Seq( + unmanagedJars in Compile ++= Seq( + getShadedJarFile(shaded_gs_collections.id, version.value) + ), + + pomPostProcess := { + (node: xml.Node) => addShadedDeps(List( + getShadedDepXML(organization.value, shaded_gs_collections.id, version.value)), node) + } + )) + .dependsOn(core % "test->test; compile->compile", shaded_gs_collections, daemon % "test->test") + .disablePlugins(sbtassembly.AssemblyPlugin) lazy val external_kafka = Project( id = "gearpump-external-kafka", @@ -402,6 +426,18 @@ object Build extends sbt.Build { .dependsOn (services % "test->test; compile->compile", daemon % "test->test; compile->compile") .disablePlugins(sbtassembly.AssemblyPlugin) + lazy val redis = Project( + id = "gearpump-experiments-redis", + base = file("experiments/redis"), + settings = commonSettings ++ noPublish ++ myAssemblySettings ++ + Seq( + libraryDependencies ++= Seq( + "redis.clients" % "jedis" % "2.9.0" + ), + mainClass in(Compile, packageBin) := Some("org.apache.gearpump.example.Test") + )) + .dependsOn(streaming % "test->test; provided", daemon % "test->test; provided") + lazy val storm = Project( id = "gearpump-experiments-storm", base = file("experiments/storm"), diff --git a/project/BuildShaded.scala b/project/BuildShaded.scala index 1f59bfd2e..a43587ca8 100644 --- a/project/BuildShaded.scala +++ b/project/BuildShaded.scala @@ -35,7 +35,7 @@ object BuildShaded extends sbt.Build { _.copy(includeScala = false) }, assemblyJarName in assembly := { - s"${name.value}-$scalaVersionMajor-${version.value}-assembly.jar" + s"${name.value}_$scalaVersionMajor-${version.value}.jar" }, target in assembly := baseDirectory.value.getParentFile / "target" / scalaVersionMajor ) @@ -44,92 +44,99 @@ object BuildShaded extends sbt.Build { id = "gearpump-shaded", base = file("shaded") ).aggregate(shaded_akka_kryo, shaded_gs_collections, shaded_guava, shaded_metrics_graphite) - .disablePlugins(sbtassembly.AssemblyPlugin) - + .disablePlugins(sbtassembly.AssemblyPlugin) lazy val shaded_akka_kryo = Project( id = "gearpump-shaded-akka-kryo", base = file("shaded/akka-kryo"), - settings = shadeAssemblySettings ++ addArtifact(Artifact("gearpump-shaded-akka-kryo", - "assembly"), sbtassembly.AssemblyKeys.assembly) ++ - Seq( - assemblyShadeRules in assembly := Seq( - ShadeRule.zap("com.google.protobuf.**").inAll, - ShadeRule.zap("com.typesafe.config.**").inAll, - ShadeRule.zap("akka.**").inAll, - ShadeRule.zap("org.jboss.netty.**").inAll, - ShadeRule.zap("net.jpountz.lz4.**").inAll, - ShadeRule.zap("org.uncommons.maths.**").inAll, - ShadeRule.rename("com.romix.**" -> "org.apache.gearpump.romix.@1").inAll, - ShadeRule.rename("com.esotericsoftware.**" -> - "org.apache.gearpump.esotericsoftware.@1").inAll, - ShadeRule.rename("org.objenesis.**" -> "org.apache.gearpump.objenesis.@1").inAll - ) - ) ++ - Seq( - libraryDependencies ++= Seq( - "com.github.romix.akka" %% "akka-kryo-serialization" % kryoVersion - ) + settings = shadeAssemblySettings ++ addArtifact(Artifact("gearpump-shaded-akka-kryo"), + sbtassembly.AssemblyKeys.assembly) ++ + Seq( + assemblyShadeRules in assembly := Seq( + ShadeRule.zap("com.google.protobuf.**").inAll, + ShadeRule.zap("com.typesafe.config.**").inAll, + ShadeRule.zap("akka.**").inAll, + ShadeRule.zap("org.jboss.netty.**").inAll, + ShadeRule.zap("net.jpountz.lz4.**").inAll, + ShadeRule.zap("org.uncommons.maths.**").inAll, + ShadeRule.rename("com.romix.**" -> "org.apache.gearpump.romix.@1").inAll, + ShadeRule.rename("com.esotericsoftware.**" -> + "org.apache.gearpump.esotericsoftware.@1").inAll, + ShadeRule.rename("org.objenesis.**" -> "org.apache.gearpump.objenesis.@1").inAll + ) + ) ++ + Seq( + libraryDependencies ++= Seq( + "com.github.romix.akka" %% "akka-kryo-serialization" % kryoVersion ) + ) ) lazy val shaded_gs_collections = Project( id = "gearpump-shaded-gs-collections", base = file("shaded/gs-collections"), - settings = shadeAssemblySettings ++ addArtifact(Artifact("gearpump-shaded-gs-collections", - "assembly"), sbtassembly.AssemblyKeys.assembly) ++ - Seq( - assemblyShadeRules in assembly := Seq( - ShadeRule.rename("com.gs.collections.**" -> - "org.apache.gearpump.gs.collections.@1").inAll - ) - ) ++ - Seq( - libraryDependencies ++= Seq( - "com.goldmansachs" % "gs-collections" % gsCollectionsVersion - ) + settings = shadeAssemblySettings ++ addArtifact(Artifact("gearpump-shaded-gs-collections"), + sbtassembly.AssemblyKeys.assembly) ++ + Seq( + assemblyShadeRules in assembly := Seq( + ShadeRule.rename("com.gs.collections.**" -> + "org.apache.gearpump.gs.collections.@1").inAll ) + ) ++ + Seq( + libraryDependencies ++= Seq( + "com.goldmansachs" % "gs-collections" % gsCollectionsVersion + ) + ) ) lazy val shaded_guava = Project( id = "gearpump-shaded-guava", base = file("shaded/guava"), - settings = shadeAssemblySettings ++ addArtifact(Artifact("gearpump-shaded-guava", - "assembly"), sbtassembly.AssemblyKeys.assembly) ++ - Seq( - assemblyShadeRules in assembly := Seq( - ShadeRule.rename("com.google.**" -> "org.apache.gearpump.google.@1").inAll - ) - ) ++ - Seq( - libraryDependencies ++= Seq( - "com.google.guava" % "guava" % guavaVersion - ) + settings = shadeAssemblySettings ++ addArtifact(Artifact("gearpump-shaded-guava"), + sbtassembly.AssemblyKeys.assembly) ++ + Seq( + assemblyShadeRules in assembly := Seq( + ShadeRule.rename("com.google.**" -> "org.apache.gearpump.google.@1").inAll + ) + ) ++ + Seq( + libraryDependencies ++= Seq( + "com.google.guava" % "guava" % guavaVersion ) + ) ) lazy val shaded_metrics_graphite = Project( id = "gearpump-shaded-metrics-graphite", base = file("shaded/metrics-graphite"), - settings = shadeAssemblySettings ++ addArtifact(Artifact("gearpump-shaded-metrics-graphite", - "assembly"), sbtassembly.AssemblyKeys.assembly) ++ - Seq( - assemblyShadeRules in assembly := Seq( - ShadeRule.rename("com.codahale.metrics.**" -> - "org.apache.gearpump.codahale.metrics.@1").inAll - ) - ) ++ - Seq( - libraryDependencies ++= Seq( - "com.codahale.metrics" % "metrics-graphite" % codahaleVersion, - "com.codahale.metrics" % "metrics-jvm" % codahaleVersion - ) + settings = shadeAssemblySettings ++ addArtifact(Artifact("gearpump-shaded-metrics-graphite"), + sbtassembly.AssemblyKeys.assembly) ++ + Seq( + assemblyShadeRules in assembly := Seq( + ShadeRule.rename("com.codahale.metrics.**" -> + "org.apache.gearpump.codahale.metrics.@1").inAll ) + ) ++ + Seq( + libraryDependencies ++= Seq( + "com.codahale.metrics" % "metrics-graphite" % codahaleVersion, + "com.codahale.metrics" % "metrics-jvm" % codahaleVersion + ) + ) ) def getShadedJarFile(name: String, gearpumpVersion: String): File = { shaded.base / "target" / scalaVersionMajor / - s"gearpump-shaded-$name-$scalaVersionMajor-$gearpumpVersion-assembly.jar" + s"${name}_$scalaVersionMajor-$gearpumpVersion.jar" + } + + def getShadedDepXML(groupId: String, artifactId: String, version: String): scala.xml.Node = { + + {groupId} + {artifactId} + {version} + } } \ No newline at end of file diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/Constants.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/Constants.scala index cd33b507a..f99a43649 100644 --- a/streaming/src/main/scala/org/apache/gearpump/streaming/Constants.scala +++ b/streaming/src/main/scala/org/apache/gearpump/streaming/Constants.scala @@ -22,6 +22,7 @@ object Constants { val GEARPUMP_STREAMING_OPERATOR = "gearpump.streaming.dsl.operator" val GEARPUMP_STREAMING_SOURCE = "gearpump.streaming.source" val GEARPUMP_STREAMING_GROUPBY_FUNCTION = "gearpump.streaming.dsl.groupby-function" + val GEARPUMP_STREAMING_WINDOW_FUNCTION = "gearpump.streaming.dsl.window-function" val GEARPUMP_STREAMING_LOCALITIES = "gearpump.streaming.localities" diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/StreamApplication.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/StreamApplication.scala index 66ec87303..a6588a14e 100644 --- a/streaming/src/main/scala/org/apache/gearpump/streaming/StreamApplication.scala +++ b/streaming/src/main/scala/org/apache/gearpump/streaming/StreamApplication.scala @@ -123,7 +123,7 @@ object LifeTime { */ class StreamApplication( override val name: String, val inputUserConfig: UserConfig, - val dag: Graph[ProcessorDescription, PartitionerDescription]) + dag: Graph[ProcessorDescription, PartitionerDescription]) extends Application { require(!dag.hasDuplicatedEdge(), "Graph should not have duplicated edges") diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/Stream.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/Stream.scala index 786d496dc..440a45ed8 100644 --- a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/Stream.scala +++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/Stream.scala @@ -20,7 +20,10 @@ package org.apache.gearpump.streaming.dsl import org.apache.gearpump.Message import org.apache.gearpump.cluster.UserConfig -import org.apache.gearpump.streaming.dsl.op._ +import org.apache.gearpump.streaming.dsl.plan._ +import org.apache.gearpump.streaming.dsl.plan.functions.{FlatMapFunction, ReduceFunction} +import org.apache.gearpump.streaming.dsl.window.api._ +import org.apache.gearpump.streaming.dsl.window.impl._ import org.apache.gearpump.streaming.sink.DataSink import org.apache.gearpump.streaming.task.{Task, TaskContext} import org.apache.gearpump.util.Graph @@ -35,12 +38,12 @@ class Stream[T]( /** * converts a value[T] to a list of value[R] * - * @param fun FlatMap function + * @param fn FlatMap function * @param description The description message for this operation * @return A new stream with type [R] */ - def flatMap[R](fun: T => TraversableOnce[R], description: String = null): Stream[R] = { - val flatMapOp = FlatMapOp(fun, Option(description).getOrElse("flatmap")) + def flatMap[R](fn: T => TraversableOnce[R], description: String = "flatMap"): Stream[R] = { + val flatMapOp = ChainableOp(new FlatMapFunction[T, R](fn, description)) graph.addVertex(flatMapOp) graph.addEdge(thisNode, edge.getOrElse(Direct), flatMapOp) new Stream[R](graph, flatMapOp) @@ -49,36 +52,36 @@ class Stream[T]( /** * Maps message of type T message of type R * - * @param fun Function + * @param fn Function * @return A new stream with type [R] */ - def map[R](fun: T => R, description: String = null): Stream[R] = { + def map[R](fn: T => R, description: String = "map"): Stream[R] = { this.flatMap({ data => - Option(fun(data)) - }, Option(description).getOrElse("map")) + Option(fn(data)) + }, description) } /** * Keeps records when fun(T) == true * - * @param fun the filter + * @param fn the filter * @return a new stream after filter */ - def filter(fun: T => Boolean, description: String = null): Stream[T] = { + def filter(fn: T => Boolean, description: String = "filter"): Stream[T] = { this.flatMap({ data => - if (fun(data)) Option(data) else None - }, Option(description).getOrElse("filter")) + if (fn(data)) Option(data) else None + }, description) } /** * Reduces operations. * - * @param fun reduction function + * @param fn reduction function * @param description description message for this operator * @return a new stream after reduction */ - def reduce(fun: (T, T) => T, description: String = null): Stream[T] = { - val reduceOp = ReduceOp(fun, Option(description).getOrElse("reduce")) + def reduce(fn: (T, T) => T, description: String = "reduce"): Stream[T] = { + val reduceOp = ChainableOp(new ReduceFunction(fn, description)) graph.addVertex(reduceOp) graph.addEdge(thisNode, edge.getOrElse(Direct), reduceOp) new Stream(graph, reduceOp) @@ -88,7 +91,10 @@ class Stream[T]( * Log to task log file */ def log(): Unit = { - this.map(msg => LoggerFactory.getLogger("dsl").info(msg.toString), "log") + this.map(msg => { + LoggerFactory.getLogger("dsl").info(msg.toString) + msg + }, "log") } /** @@ -97,8 +103,8 @@ class Stream[T]( * @param other the other stream * @return the merged stream */ - def merge(other: Stream[T], description: String = null): Stream[T] = { - val mergeOp = MergeOp(Option(description).getOrElse("merge")) + def merge(other: Stream[T], description: String = "merge"): Stream[T] = { + val mergeOp = MergeOp(description, UserConfig.empty) graph.addVertex(mergeOp) graph.addEdge(thisNode, edge.getOrElse(Direct), mergeOp) graph.addEdge(other.thisNode, other.edge.getOrElse(Shuffle), mergeOp) @@ -115,20 +121,29 @@ class Stream[T]( * * For example, * {{{ - * Stream[People].groupBy(_.gender).flatmap(..).filter.(..).reduce(..) + * Stream[People].groupBy(_.gender).flatMap(..).filter(..).reduce(..) * }}} * - * @param fun Group by function + * @param fn Group by function * @param parallelism Parallelism level * @param description The description * @return the grouped stream */ - def groupBy[Group](fun: T => Group, parallelism: Int = 1, description: String = null) - : Stream[T] = { - val groupOp = GroupByOp(fun, parallelism, Option(description).getOrElse("groupBy")) - graph.addVertex(groupOp) - graph.addEdge(thisNode, edge.getOrElse(Shuffle), groupOp) - new Stream[T](graph, groupOp) + def groupBy[GROUP](fn: T => GROUP, parallelism: Int = 1, + description: String = "groupBy"): Stream[T] = { + window(CountWindow.apply(1).accumulating) + .groupBy[GROUP](fn, parallelism, description) + } + + /** + * Window function + * + * @param win window definition + * @param description window description + * @return [[WindowStream]] where groupBy could be applied + */ + def window(win: Window, description: String = "window"): WindowStream[T] = { + new WindowStream[T](graph, edge, thisNode, win, description) } /** @@ -140,15 +155,28 @@ class Stream[T]( */ def process[R]( processor: Class[_ <: Task], parallelism: Int, conf: UserConfig = UserConfig.empty, - description: String = null): Stream[R] = { - val processorOp = ProcessorOp(processor, parallelism, conf, - Option(description).getOrElse("process")) + description: String = "process"): Stream[R] = { + val processorOp = ProcessorOp(processor, parallelism, conf, description) graph.addVertex(processorOp) graph.addEdge(thisNode, edge.getOrElse(Shuffle), processorOp) new Stream[R](graph, processorOp, Some(Shuffle)) } } +class WindowStream[T](graph: Graph[Op, OpEdge], edge: Option[OpEdge], thisNode: Op, + window: Window, winDesc: String) { + + def groupBy[GROUP](fn: T => GROUP, parallelism: Int = 1, + description: String = "groupBy"): Stream[T] = { + val groupBy: GroupByFn[T, (GROUP, List[Bucket])] = GroupAlsoByWindow(fn, window) + val groupOp = GroupByOp[T, (GROUP, List[Bucket])](groupBy, parallelism, + s"$winDesc.$description") + graph.addVertex(groupOp) + graph.addEdge(thisNode, edge.getOrElse(Shuffle), groupOp) + new Stream[T](graph, groupOp) + } +} + class KVStream[K, V](stream: Stream[Tuple2[K, V]]) { /** * GroupBy key @@ -192,30 +220,18 @@ object Stream { } implicit class Sink[T](stream: Stream[T]) extends java.io.Serializable { - def sink[T](dataSink: DataSink, parallism: Int, conf: UserConfig, description: String) - : Stream[T] = { - implicit val sink = DataSinkOp[T](dataSink, parallism, conf, - Some(description).getOrElse("traversable")) + def sink(dataSink: DataSink, parallelism: Int = 1, + conf: UserConfig = UserConfig.empty, description: String = "sink"): Stream[T] = { + implicit val sink = DataSinkOp(dataSink, parallelism, conf, description) stream.graph.addVertex(sink) stream.graph.addEdge(stream.thisNode, Shuffle, sink) new Stream[T](stream.graph, sink) } - - def sink[T]( - sink: Class[_ <: Task], parallism: Int, conf: UserConfig = UserConfig.empty, - description: String = null): Stream[T] = { - val sinkOp = ProcessorOp(sink, parallism, conf, Option(description).getOrElse("source")) - stream.graph.addVertex(sinkOp) - stream.graph.addEdge(stream.thisNode, Shuffle, sinkOp) - new Stream[T](stream.graph, sinkOp) - } } } class LoggerSink[T] extends DataSink { - var logger: Logger = null - - private var context: TaskContext = null + var logger: Logger = _ override def open(context: TaskContext): Unit = { this.logger = context.logger diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/StreamApp.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/StreamApp.scala index d45737b0e..81161466c 100644 --- a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/StreamApp.scala +++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/StreamApp.scala @@ -24,10 +24,9 @@ import akka.actor.ActorSystem import org.apache.gearpump.cluster.UserConfig import org.apache.gearpump.cluster.client.ClientContext import org.apache.gearpump.streaming.StreamApplication -import org.apache.gearpump.streaming.dsl.op.{DataSourceOp, Op, OpEdge, ProcessorOp} -import org.apache.gearpump.streaming.dsl.plan.Planner +import org.apache.gearpump.streaming.dsl.plan._ import org.apache.gearpump.streaming.source.DataSource -import org.apache.gearpump.streaming.task.{Task, TaskContext} +import org.apache.gearpump.streaming.task.TaskContext import org.apache.gearpump.util.Graph import org.apache.gearpump.Message @@ -50,7 +49,8 @@ import scala.language.implicitConversions * @param name name of app */ class StreamApp( - val name: String, system: ActorSystem, userConfig: UserConfig, val graph: Graph[Op, OpEdge]) { + name: String, system: ActorSystem, userConfig: UserConfig, + private val graph: Graph[Op, OpEdge]) { def this(name: String, system: ActorSystem, userConfig: UserConfig) = { this(name, system, userConfig, Graph.empty[Op, OpEdge]) @@ -76,34 +76,16 @@ object StreamApp { implicit class Source(app: StreamApp) extends java.io.Serializable { - def source[T](dataSource: DataSource, parallelism: Int): Stream[T] = { - source(dataSource, parallelism, UserConfig.empty) - } - - def source[T](dataSource: DataSource, parallelism: Int, description: String): Stream[T] = { - source(dataSource, parallelism, UserConfig.empty, description) - } - - def source[T](dataSource: DataSource, parallelism: Int, conf: UserConfig): Stream[T] = { - source(dataSource, parallelism, conf, description = null) - } - - def source[T](dataSource: DataSource, parallelism: Int, conf: UserConfig, description: String) - : Stream[T] = { + def source[T](dataSource: DataSource, parallelism: Int = 1, + conf: UserConfig = UserConfig.empty, description: String = "source"): Stream[T] = { implicit val sourceOp = DataSourceOp(dataSource, parallelism, conf, description) app.graph.addVertex(sourceOp) new Stream[T](app.graph, sourceOp) } + def source[T](seq: Seq[T], parallelism: Int, description: String): Stream[T] = { this.source(new CollectionDataSource[T](seq), parallelism, UserConfig.empty, description) } - - def source[T](source: Class[_ <: Task], parallelism: Int, conf: UserConfig, description: String) - : Stream[T] = { - val sourceOp = ProcessorOp(source, parallelism, conf, Option(description).getOrElse("source")) - app.graph.addVertex(sourceOp) - new Stream[T](app.graph, sourceOp) - } } } @@ -115,7 +97,7 @@ class CollectionDataSource[T](seq: Seq[T]) extends DataSource { override def read(): Message = { if (iterator.hasNext) { - Message(iterator.next()) + Message(iterator.next(), Instant.now().toEpochMilli) } else { null } diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/javaapi/JavaStream.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/javaapi/JavaStream.scala index 6eff20cf2..3003b981f 100644 --- a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/javaapi/JavaStream.scala +++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/javaapi/JavaStream.scala @@ -19,9 +19,9 @@ package org.apache.gearpump.streaming.dsl.javaapi import scala.collection.JavaConverters._ - import org.apache.gearpump.cluster.UserConfig -import org.apache.gearpump.streaming.dsl.Stream +import org.apache.gearpump.streaming.dsl.window.api.Window +import org.apache.gearpump.streaming.dsl.{Stream, WindowStream} import org.apache.gearpump.streaming.javaapi.dsl.functions._ import org.apache.gearpump.streaming.task.Task @@ -63,9 +63,13 @@ class JavaStream[T](val stream: Stream[T]) { * Group by a stream and turns it to a list of sub-streams. Operations chained after * groupBy applies to sub-streams. */ - def groupBy[Group](fn: GroupByFunction[T, Group], parallelism: Int, description: String) - : JavaStream[T] = { - new JavaStream[T](stream.groupBy({t: T => fn(t)}, parallelism, description)) + def groupBy[GROUP](fn: GroupByFunction[T, GROUP], + parallelism: Int, description: String): JavaStream[T] = { + new JavaStream[T](stream.groupBy((t: T) => fn, parallelism, description)) + } + + def window(win: Window, description: String): JavaWindowStream[T] = { + new JavaWindowStream[T](stream.window(win, description)) } /** Add a low level Processor to process messages */ @@ -75,3 +79,11 @@ class JavaStream[T](val stream: Stream[T]) { new JavaStream[R](stream.process(processor, parallelism, conf, description)) } } + +class JavaWindowStream[T](stream: WindowStream[T]) { + + def groupBy[GROUP](fn: GroupByFunction[T, GROUP], parallelism: Int, + description: String): JavaStream[T] = { + new JavaStream[T](stream.groupBy((t: T) => fn, parallelism, description)) + } +} diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/op/OP.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/op/OP.scala deleted file mode 100644 index 49d9dec72..000000000 --- a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/op/OP.scala +++ /dev/null @@ -1,109 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.gearpump.streaming.dsl.op - -import org.apache.gearpump.cluster.UserConfig -import org.apache.gearpump.streaming.sink.DataSink -import org.apache.gearpump.streaming.source.DataSource -import org.apache.gearpump.streaming.task.Task - -/** - * Operators for the DSL - */ -sealed trait Op { - def description: String - def conf: UserConfig -} - -/** - * When translated to running DAG, SlaveOP can be attach to MasterOP or other SlaveOP - * "Attach" means running in same Actor. - */ -trait SlaveOp[T] extends Op - -case class FlatMapOp[T, R]( - fun: (T) => TraversableOnce[R], description: String, conf: UserConfig = UserConfig.empty) - extends SlaveOp[T] - -case class ReduceOp[T](fun: (T, T) => T, description: String, conf: UserConfig = UserConfig.empty) - extends SlaveOp[T] - -trait MasterOp extends Op - -trait ParameterizedOp[T] extends MasterOp - -case class MergeOp(description: String, override val conf: UserConfig = UserConfig.empty) - extends MasterOp - -case class GroupByOp[T, R]( - fun: T => R, parallelism: Int, description: String, - override val conf: UserConfig = UserConfig.empty) - extends ParameterizedOp[T] - -case class ProcessorOp[T <: Task]( - processor: Class[T], parallelism: Int, conf: UserConfig, description: String) - extends ParameterizedOp[T] - -case class DataSourceOp[T]( - dataSource: DataSource, parallelism: Int, conf: UserConfig, description: String) - extends ParameterizedOp[T] - -case class DataSinkOp[T]( - dataSink: DataSink, parallelism: Int, conf: UserConfig, description: String) - extends ParameterizedOp[T] - -/** - * Contains operators which can be chained to single one. - * - * For example, flatmap().map().reduce() can be chained to single operator as - * no data shuffling is required. - * @param ops list of operations - */ -case class OpChain(ops: List[Op]) extends Op { - def head: Op = ops.head - def last: Op = ops.last - - def description: String = null - - override def conf: UserConfig = { - // The head's conf has priority - ops.reverse.foldLeft(UserConfig.empty) { (conf, op) => - conf.withConfig(op.conf) - } - } -} - -trait OpEdge - -/** - * The upstream OP and downstream OP doesn't require network data shuffle. - * - * For example, map, flatmap operation doesn't require network shuffle, we can use Direct - * to represent the relation with upstream operators. - */ -case object Direct extends OpEdge - -/** - * The upstream OP and downstream OP DOES require network data shuffle. - * - * For example, map, flatmap operation doesn't require network shuffle, we can use Direct - * to represent the relation with upstream operators. - */ -case object Shuffle extends OpEdge - diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/partitioner/GroupbyPartitioner.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/partitioner/GroupByPartitioner.scala similarity index 77% rename from streaming/src/main/scala/org/apache/gearpump/streaming/dsl/partitioner/GroupbyPartitioner.scala rename to streaming/src/main/scala/org/apache/gearpump/streaming/dsl/partitioner/GroupByPartitioner.scala index b2e29328a..2ec881b7e 100644 --- a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/partitioner/GroupbyPartitioner.scala +++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/partitioner/GroupByPartitioner.scala @@ -20,6 +20,7 @@ package org.apache.gearpump.streaming.dsl.partitioner import org.apache.gearpump.Message import org.apache.gearpump.partitioner.UnicastPartitioner +import org.apache.gearpump.streaming.dsl.window.api.GroupByFn /** * Partition messages by applying group by function first. @@ -35,12 +36,14 @@ import org.apache.gearpump.partitioner.UnicastPartitioner * } * }}} * - * @param groupBy First apply message with groupBy function, then pick the hashCode of the output + * @param fn First apply message with groupBy function, then pick the hashCode of the output * to do the partitioning. You must define hashCode() for output type of groupBy function. */ -class GroupByPartitioner[T, GROUP](groupBy: T => GROUP = null) extends UnicastPartitioner { - override def getPartition(msg: Message, partitionNum: Int, currentPartitionId: Int): Int = { - val hashCode = groupBy(msg.msg.asInstanceOf[T]).hashCode() +class GroupByPartitioner[T, Group](fn: GroupByFn[T, Group]) + extends UnicastPartitioner { + override def getPartition(message: Message, partitionNum: Int, currentPartitionId: Int): Int = { + val hashCode = fn.groupBy(message).hashCode() (hashCode & Integer.MAX_VALUE) % partitionNum } -} \ No newline at end of file +} + diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/OP.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/OP.scala new file mode 100644 index 000000000..744976b7c --- /dev/null +++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/OP.scala @@ -0,0 +1,214 @@ +/* + * 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.gearpump.streaming.dsl.plan + +import akka.actor.ActorSystem +import org.apache.gearpump.cluster.UserConfig +import org.apache.gearpump.streaming.Constants._ +import org.apache.gearpump.streaming.Processor.DefaultProcessor +import org.apache.gearpump.streaming.dsl.plan.functions.SingleInputFunction +import org.apache.gearpump.streaming.{Constants, Processor} +import org.apache.gearpump.streaming.dsl.task.TransformTask +import org.apache.gearpump.streaming.dsl.window.api.GroupByFn +import org.apache.gearpump.streaming.sink.{DataSink, DataSinkProcessor} +import org.apache.gearpump.streaming.source.{DataSource, DataSourceTask} +import org.apache.gearpump.streaming.task.Task + +import scala.reflect.ClassTag + +/** + * This is a vertex on the logical plan. + */ +sealed trait Op { + + def description: String + + def userConfig: UserConfig + + def chain(op: Op)(implicit system: ActorSystem): Op + + def getProcessor(implicit system: ActorSystem): Processor[_ <: Task] +} + +/** + * This represents a low level Processor. + */ +case class ProcessorOp[T <: Task]( + processor: Class[T], + parallelism: Int, + userConfig: UserConfig, + description: String) + extends Op { + + def this( + parallelism: Int = 1, + userConfig: UserConfig = UserConfig.empty, + description: String = "processor")(implicit classTag: ClassTag[T]) = { + this(classTag.runtimeClass.asInstanceOf[Class[T]], parallelism, userConfig, description) + } + + override def chain(other: Op)(implicit system: ActorSystem): Op = { + throw new OpChainException(this, other) + } + + override def getProcessor(implicit system: ActorSystem): Processor[_ <: Task] = { + DefaultProcessor(parallelism, description, userConfig, processor) + } +} + +/** + * This represents a DataSource. + */ +case class DataSourceOp( + dataSource: DataSource, + parallelism: Int = 1, + userConfig: UserConfig = UserConfig.empty, + description: String = "source") + extends Op { + + override def chain(other: Op)(implicit system: ActorSystem): Op = { + other match { + case op: ChainableOp[_, _] => + DataSourceOp(dataSource, parallelism, + userConfig.withValue(Constants.GEARPUMP_STREAMING_OPERATOR, op.fn), + description) + case _ => + throw new OpChainException(this, other) + } + } + + override def getProcessor(implicit system: ActorSystem): Processor[_ <: Task] = { + Processor[DataSourceTask[Any, Any]](parallelism, description, + userConfig.withValue(GEARPUMP_STREAMING_SOURCE, dataSource)) + } +} + +/** + * This represents a DataSink. + */ +case class DataSinkOp( + dataSink: DataSink, + parallelism: Int = 1, + userConfig: UserConfig = UserConfig.empty, + description: String = "sink") + extends Op { + + override def chain(op: Op)(implicit system: ActorSystem): Op = { + throw new OpChainException(this, op) + } + + override def getProcessor(implicit system: ActorSystem): Processor[_ <: Task] = { + DataSinkProcessor(dataSink, parallelism, description) + } +} + +/** + * This represents operations that can be chained together + * (e.g. flatMap, map, filter, reduce) and further chained + * to another Op to be used + */ +case class ChainableOp[IN, OUT]( + fn: SingleInputFunction[IN, OUT]) extends Op { + + override def description: String = fn.description + + override def userConfig: UserConfig = UserConfig.empty + + override def chain(other: Op)(implicit system: ActorSystem): Op = { + other match { + case op: ChainableOp[OUT, _] => + // TODO: preserve type info + ChainableOp(fn.andThen(op.fn)) + case _ => + throw new OpChainException(this, other) + } + } + + override def getProcessor(implicit system: ActorSystem): Processor[_ <: Task] = { + throw new UnsupportedOperationException("ChainedOp cannot be translated to Processor") + } +} + +/** + * This represents a Processor with window aggregation + */ +case class GroupByOp[IN, GROUP]( + groupByFn: GroupByFn[IN, GROUP], + parallelism: Int = 1, + description: String = "groupBy", + override val userConfig: UserConfig = UserConfig.empty) + extends Op { + + override def chain(other: Op)(implicit system: ActorSystem): Op = { + other match { + case op: ChainableOp[_, _] => + GroupByOp(groupByFn, parallelism, description, + userConfig.withValue(Constants.GEARPUMP_STREAMING_OPERATOR, op.fn)) + case _ => + throw new OpChainException(this, other) + } + } + + override def getProcessor(implicit system: ActorSystem): Processor[_ <: Task] = { + groupByFn.getProcessor(parallelism, description, userConfig) + } +} + +/** + * This represents a Processor transforming merged streams + */ +case class MergeOp(description: String, userConfig: UserConfig = UserConfig.empty) + extends Op { + + override def chain(other: Op)(implicit system: ActorSystem): Op = { + other match { + case op: ChainableOp[_, _] => + MergeOp(description, userConfig.withValue(Constants.GEARPUMP_STREAMING_OPERATOR, op.fn)) + case _ => + throw new OpChainException(this, other) + } + } + + override def getProcessor(implicit system: ActorSystem): Processor[_ <: Task] = { + Processor[TransformTask[Any, Any]](1, description, userConfig) + } + +} + +/** + * This is an edge on the logical plan. + */ +trait OpEdge + +/** + * The upstream OP and downstream OP doesn't require network data shuffle. + * e.g. ChainableOp + */ +case object Direct extends OpEdge + +/** + * The upstream OP and downstream OP DOES require network data shuffle. + * e.g. GroupByOp + */ +case object Shuffle extends OpEdge + +/** + * Runtime exception thrown on chaining. + */ +class OpChainException(op1: Op, op2: Op) extends RuntimeException(s"$op1 cannot be chained by $op2") \ No newline at end of file diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/OpTranslator.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/OpTranslator.scala deleted file mode 100644 index b09d9b9cb..000000000 --- a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/OpTranslator.scala +++ /dev/null @@ -1,222 +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.gearpump.streaming.dsl.plan - -import scala.collection.TraversableOnce -import akka.actor.ActorSystem -import org.slf4j.Logger -import org.apache.gearpump._ -import org.apache.gearpump.cluster.UserConfig -import org.apache.gearpump.streaming.Constants._ -import org.apache.gearpump.streaming.Processor -import org.apache.gearpump.streaming.Processor.DefaultProcessor -import org.apache.gearpump.streaming.dsl.op._ -import org.apache.gearpump.streaming.dsl.plan.OpTranslator._ -import org.apache.gearpump.streaming.sink.DataSinkProcessor -import org.apache.gearpump.streaming.source.DataSourceTask -import org.apache.gearpump.streaming.task.{Task, TaskContext} -import org.apache.gearpump.util.LogUtil - -/** - * Translates a OP to a TaskDescription - */ -class OpTranslator extends java.io.Serializable { - val LOG: Logger = LogUtil.getLogger(getClass) - - def translate(ops: OpChain)(implicit system: ActorSystem): Processor[_ <: Task] = { - - val baseConfig = ops.conf - - ops.ops.head match { - case op: MasterOp => - val tail = ops.ops.tail - val func = toFunction(tail) - val userConfig = baseConfig.withValue(GEARPUMP_STREAMING_OPERATOR, func) - - op match { - case DataSourceOp(dataSource, parallelism, conf, description) => - Processor[DataSourceTask[Any, Any]](parallelism, - description = description + "." + func.description, - userConfig.withValue(GEARPUMP_STREAMING_SOURCE, dataSource)) - case groupby@GroupByOp(_, parallelism, description, _) => - Processor[GroupByTask[Object, Object, Object]](parallelism, - description = description + "." + func.description, - userConfig.withValue(GEARPUMP_STREAMING_GROUPBY_FUNCTION, groupby)) - case merge: MergeOp => - Processor[TransformTask[Object, Object]](1, - description = op.description + "." + func.description, - userConfig) - case ProcessorOp(processor, parallelism, conf, description) => - DefaultProcessor(parallelism, - description = description + " " + func.description, - userConfig, processor) - case DataSinkOp(dataSink, parallelism, conf, description) => - DataSinkProcessor(dataSink, parallelism, description + func.description) - } - case op: SlaveOp[_] => - val func = toFunction(ops.ops) - val userConfig = baseConfig.withValue(GEARPUMP_STREAMING_OPERATOR, func) - - Processor[TransformTask[Object, Object]](1, - description = func.description, - taskConf = userConfig) - case chain: OpChain => - throw new RuntimeException("Not supposed to be called!") - } - } - - private def toFunction(ops: List[Op]): SingleInputFunction[Object, Object] = { - val func: SingleInputFunction[Object, Object] = new DummyInputFunction[Object]() - val totalFunction = ops.foldLeft(func) { (fun, op) => - - val opFunction = op match { - case flatmap: FlatMapOp[Object @unchecked, Object @unchecked] => - new FlatMapFunction(flatmap.fun, flatmap.description) - case reduce: ReduceOp[Object @unchecked] => - new ReduceFunction(reduce.fun, reduce.description) - case _ => - throw new RuntimeException("Not supposed to be called!") - } - fun.andThen(opFunction.asInstanceOf[SingleInputFunction[Object, Object]]) - } - totalFunction.asInstanceOf[SingleInputFunction[Object, Object]] - } -} - -object OpTranslator { - - trait SingleInputFunction[IN, OUT] extends Serializable { - def process(value: IN): TraversableOnce[OUT] - def andThen[OUTER](other: SingleInputFunction[OUT, OUTER]): SingleInputFunction[IN, OUTER] = { - new AndThen(this, other) - } - - def description: String - } - - class DummyInputFunction[T] extends SingleInputFunction[T, T] { - override def andThen[OUTER](other: SingleInputFunction[T, OUTER]) - : SingleInputFunction[T, OUTER] = { - other - } - - // Should never be called - override def process(value: T): TraversableOnce[T] = None - - override def description: String = "" - } - - class AndThen[IN, MIDDLE, OUT]( - first: SingleInputFunction[IN, MIDDLE], second: SingleInputFunction[MIDDLE, OUT]) - extends SingleInputFunction[IN, OUT] { - - override def process(value: IN): TraversableOnce[OUT] = { - first.process(value).flatMap(second.process) - } - - override def description: String = { - Option(first.description).flatMap { description => - Option(second.description).map(description + "." + _) - }.orNull - } - } - - class FlatMapFunction[IN, OUT](fun: IN => TraversableOnce[OUT], descriptionMessage: String) - extends SingleInputFunction[IN, OUT] { - - override def process(value: IN): TraversableOnce[OUT] = { - fun(value) - } - - override def description: String = { - this.descriptionMessage - } - } - - class ReduceFunction[T](fun: (T, T) => T, descriptionMessage: String) - extends SingleInputFunction[T, T] { - - private var state: Any = _ - - override def process(value: T): TraversableOnce[T] = { - if (state == null) { - state = value - } else { - state = fun(state.asInstanceOf[T], value) - } - Some(state.asInstanceOf[T]) - } - - override def description: String = descriptionMessage - } - - class GroupByTask[IN, GROUP, OUT]( - groupBy: IN => GROUP, taskContext: TaskContext, userConf: UserConfig) - extends Task(taskContext, userConf) { - - def this(taskContext: TaskContext, userConf: UserConfig) = { - this(userConf.getValue[GroupByOp[IN, GROUP]]( - GEARPUMP_STREAMING_GROUPBY_FUNCTION )(taskContext.system).get.fun, - taskContext, userConf) - } - - private var groups = Map.empty[GROUP, SingleInputFunction[IN, OUT]] - - override def onNext(msg: Message): Unit = { - val time = msg.timestamp - - val group = groupBy(msg.msg.asInstanceOf[IN]) - if (!groups.contains(group)) { - val operator = - userConf.getValue[SingleInputFunction[IN, OUT]](GEARPUMP_STREAMING_OPERATOR).get - groups += group -> operator - } - - val operator = groups(group) - - operator.process(msg.msg.asInstanceOf[IN]).foreach { msg => - taskContext.output(new Message(msg.asInstanceOf[AnyRef], time)) - } - } - } - - class TransformTask[IN, OUT]( - operator: Option[SingleInputFunction[IN, OUT]], taskContext: TaskContext, - userConf: UserConfig) extends Task(taskContext, userConf) { - - def this(taskContext: TaskContext, userConf: UserConfig) = { - this(userConf.getValue[SingleInputFunction[IN, OUT]]( - GEARPUMP_STREAMING_OPERATOR)(taskContext.system), taskContext, userConf) - } - - override def onNext(msg: Message): Unit = { - val time = msg.timestamp - - operator match { - case Some(op) => - op.process(msg.msg.asInstanceOf[IN]).foreach { msg => - taskContext.output(new Message(msg.asInstanceOf[AnyRef], time)) - } - case None => - taskContext.output(new Message(msg.msg, time)) - } - } - } - -} \ No newline at end of file diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/Planner.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/Planner.scala index f5bbd654d..16d5c06ea 100644 --- a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/Planner.scala +++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/Planner.scala @@ -22,7 +22,6 @@ import akka.actor.ActorSystem import org.apache.gearpump.partitioner.{CoLocationPartitioner, HashPartitioner, Partitioner} import org.apache.gearpump.streaming.Processor -import org.apache.gearpump.streaming.dsl.op._ import org.apache.gearpump.streaming.dsl.partitioner.GroupByPartitioner import org.apache.gearpump.streaming.task.Task import org.apache.gearpump.util.Graph @@ -33,64 +32,60 @@ class Planner { * Converts Dag of Op to Dag of TaskDescription. TaskDescription is part of the low * level Graph API. */ - def plan(dag: Graph[Op, OpEdge])(implicit system: ActorSystem) - : Graph[Processor[_ <: Task], _ <: Partitioner] = { + def plan(dag: Graph[Op, OpEdge]) + (implicit system: ActorSystem): Graph[Processor[_ <: Task], _ <: Partitioner] = { - val opTranslator = new OpTranslator() - - val newDag = optimize(dag) - newDag.mapEdge { (node1, edge, node2) => + val graph = optimize(dag) + graph.mapEdge { (node1, edge, node2) => edge match { case Shuffle => - node2.head match { - case groupBy: GroupByOp[Any @unchecked, Any @unchecked] => - new GroupByPartitioner(groupBy.fun) + node2 match { + case groupBy: GroupByOp[_, _] => + new GroupByPartitioner(groupBy.groupByFn) case _ => new HashPartitioner } case Direct => new CoLocationPartitioner } - }.mapVertex { opChain => - opTranslator.translate(opChain) - } + }.mapVertex(_.getProcessor) } - private def optimize(dag: Graph[Op, OpEdge]): Graph[OpChain, OpEdge] = { - val newGraph = dag.mapVertex(op => OpChain(List(op))) - - val nodes = newGraph.topologicalOrderWithCirclesIterator.toList.reverse + private def optimize(dag: Graph[Op, OpEdge]) + (implicit system: ActorSystem): Graph[Op, OpEdge] = { + val graph = dag.copy + val nodes = graph.topologicalOrderWithCirclesIterator.toList.reverse for (node <- nodes) { - val outGoingEdges = newGraph.outgoingEdgesOf(node) + val outGoingEdges = graph.outgoingEdgesOf(node) for (edge <- outGoingEdges) { - merge(newGraph, edge._1, edge._3) + merge(graph, edge._1, edge._3) } } - newGraph + graph } - private def merge(dag: Graph[OpChain, OpEdge], node1: OpChain, node2: OpChain) - : Graph[OpChain, OpEdge] = { - if (dag.outDegreeOf(node1) == 1 && - dag.inDegreeOf(node2) == 1 && + private def merge(graph: Graph[Op, OpEdge], node1: Op, node2: Op) + (implicit system: ActorSystem): Unit = { + if (graph.outDegreeOf(node1) == 1 && + graph.inDegreeOf(node2) == 1 && // For processor node, we don't allow it to merge with downstream operators - !node1.head.isInstanceOf[ProcessorOp[_ <: Task]]) { - val (_, edge, _) = dag.outgoingEdgesOf(node1).head + !node1.isInstanceOf[ProcessorOp[_ <: Task]] && + !node2.isInstanceOf[ProcessorOp[_ <: Task]]) { + val (_, edge, _) = graph.outgoingEdgesOf(node1).head if (edge == Direct) { - val opList = OpChain(node1.ops ++ node2.ops) - dag.addVertex(opList) - for (incomingEdge <- dag.incomingEdgesOf(node1)) { - dag.addEdge(incomingEdge._1, incomingEdge._2, opList) + val chainedOp = node1.chain(node2) + graph.addVertex(chainedOp) + for (incomingEdge <- graph.incomingEdgesOf(node1)) { + graph.addEdge(incomingEdge._1, incomingEdge._2, chainedOp) } - for (outgoingEdge <- dag.outgoingEdgesOf(node2)) { - dag.addEdge(opList, outgoingEdge._2, outgoingEdge._3) + for (outgoingEdge <- graph.outgoingEdgesOf(node2)) { + graph.addEdge(chainedOp, outgoingEdge._2, outgoingEdge._3) } // Remove the old vertex - dag.removeVertex(node1) - dag.removeVertex(node2) + graph.removeVertex(node1) + graph.removeVertex(node2) } } - dag } } \ No newline at end of file diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/functions/SingleInputFunction.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/functions/SingleInputFunction.scala new file mode 100644 index 000000000..609fbb06f --- /dev/null +++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/functions/SingleInputFunction.scala @@ -0,0 +1,107 @@ +/* + * 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.gearpump.streaming.dsl.plan.functions + +trait SingleInputFunction[IN, OUT] extends Serializable { + def process(value: IN): TraversableOnce[OUT] + def andThen[OUTER](other: SingleInputFunction[OUT, OUTER]): SingleInputFunction[IN, OUTER] = { + new AndThen(this, other) + } + def finish(): TraversableOnce[OUT] = None + def clearState(): Unit = {} + def description: String +} + +class AndThen[IN, MIDDLE, OUT]( + first: SingleInputFunction[IN, MIDDLE], second: SingleInputFunction[MIDDLE, OUT]) + extends SingleInputFunction[IN, OUT] { + + override def process(value: IN): TraversableOnce[OUT] = { + first.process(value).flatMap(second.process) + } + + override def finish(): TraversableOnce[OUT] = { + val firstResult = first.finish().flatMap(second.process) + if (firstResult.isEmpty) { + second.finish() + } else { + firstResult + } + } + + override def clearState(): Unit = { + first.clearState() + second.clearState() + } + + override def description: String = { + Option(first.description).flatMap { description => + Option(second.description).map(description + "." + _) + }.orNull + } +} + +class FlatMapFunction[IN, OUT](fn: IN => TraversableOnce[OUT], descriptionMessage: String) + extends SingleInputFunction[IN, OUT] { + + override def process(value: IN): TraversableOnce[OUT] = { + fn(value) + } + + override def description: String = descriptionMessage +} + + +class ReduceFunction[T](fn: (T, T) => T, descriptionMessage: String) + extends SingleInputFunction[T, T] { + + private var state: Option[T] = None + + override def process(value: T): TraversableOnce[T] = { + if (state.isEmpty) { + state = Option(value) + } else { + state = state.map(fn(_, value)) + } + None + } + + override def finish(): TraversableOnce[T] = { + state + } + + override def clearState(): Unit = { + state = None + } + + override def description: String = descriptionMessage +} + +class EmitFunction[T](emit: T => Unit) extends SingleInputFunction[T, Unit] { + + override def process(value: T): TraversableOnce[Unit] = { + emit(value) + None + } + + override def andThen[R](other: SingleInputFunction[Unit, R]): SingleInputFunction[T, R] = { + throw new UnsupportedOperationException("andThen is not supposed to be called on EmitFunction") + } + + override def description: String = "" +} diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/task/CountTriggerTask.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/task/CountTriggerTask.scala new file mode 100644 index 000000000..4ee2fa8c1 --- /dev/null +++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/task/CountTriggerTask.scala @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gearpump.streaming.dsl.task + +import java.time.Instant + +import akka.actor.ActorSystem +import org.apache.gearpump.Message +import org.apache.gearpump.cluster.UserConfig +import org.apache.gearpump.streaming.Constants._ +import org.apache.gearpump.streaming.dsl.window.api.CountWindowFn +import org.apache.gearpump.streaming.dsl.window.impl.{DefaultWindowRunner, GroupAlsoByWindow, WindowRunner} +import org.apache.gearpump.streaming.task.{Task, TaskContext} + +/** + * This task triggers output on number of messages in a window. + */ +class CountTriggerTask[IN, GROUP]( + groupBy: GroupAlsoByWindow[IN, GROUP], + windowRunner: WindowRunner, + taskContext: TaskContext, + userConfig: UserConfig) + extends Task(taskContext, userConfig) { + + def this(groupBy: GroupAlsoByWindow[IN, GROUP], + taskContext: TaskContext, userConfig: UserConfig) = { + this(groupBy, new DefaultWindowRunner(taskContext, userConfig, groupBy)(taskContext.system), + taskContext, userConfig) + } + + def this(taskContext: TaskContext, userConfig: UserConfig) = { + this(userConfig.getValue[GroupAlsoByWindow[IN, GROUP]]( + GEARPUMP_STREAMING_GROUPBY_FUNCTION)(taskContext.system).get, + taskContext, userConfig) + } + + private val windowSize = groupBy.window.windowFn.asInstanceOf[CountWindowFn].size + private var num = 0 + + override def onNext(msg: Message): Unit = { + windowRunner.process(msg) + num += 1 + if (windowSize == num) { + windowRunner.trigger(Instant.ofEpochMilli(windowSize)) + num = 0 + } + } +} diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/task/EventTimeTriggerTask.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/task/EventTimeTriggerTask.scala new file mode 100644 index 000000000..4b7649f71 --- /dev/null +++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/task/EventTimeTriggerTask.scala @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gearpump.streaming.dsl.task + +import java.time.Instant + +import akka.actor.ActorSystem +import org.apache.gearpump.Message +import org.apache.gearpump.cluster.UserConfig +import org.apache.gearpump.streaming.Constants._ +import org.apache.gearpump.streaming.dsl.window.impl.{DefaultWindowRunner, GroupAlsoByWindow, WindowRunner} +import org.apache.gearpump.streaming.task.{Task, TaskContext} + +/** + * This task triggers output on watermark progress. + */ +class EventTimeTriggerTask[IN, GROUP]( + groupBy: GroupAlsoByWindow[IN, GROUP], + windowRunner: WindowRunner, + taskContext: TaskContext, + userConfig: UserConfig) + extends Task(taskContext, userConfig) { + + def this(groupBy: GroupAlsoByWindow[IN, GROUP], + taskContext: TaskContext, userConfig: UserConfig) = { + this(groupBy, new DefaultWindowRunner(taskContext, userConfig, groupBy)(taskContext.system), + taskContext, userConfig) + } + + def this(taskContext: TaskContext, userConfig: UserConfig) = { + this(userConfig.getValue[GroupAlsoByWindow[IN, GROUP]]( + GEARPUMP_STREAMING_GROUPBY_FUNCTION)(taskContext.system).get, + taskContext, userConfig) + } + + override def onNext(message: Message): Unit = { + windowRunner.process(message) + } + + override def onWatermarkProgress(watermark: Instant): Unit = { + windowRunner.trigger(watermark) + } + +} diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/task/ProcessingTimeTriggerTask.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/task/ProcessingTimeTriggerTask.scala new file mode 100644 index 000000000..980a54b23 --- /dev/null +++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/task/ProcessingTimeTriggerTask.scala @@ -0,0 +1,82 @@ +/* + * 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.gearpump.streaming.dsl.task + +import java.time.Instant +import java.util.concurrent.TimeUnit + +import akka.actor.Actor.Receive +import akka.actor.ActorSystem +import org.apache.gearpump.Message +import org.apache.gearpump.cluster.UserConfig +import org.apache.gearpump.streaming.Constants._ +import org.apache.gearpump.streaming.dsl.task.ProcessingTimeTriggerTask.Triggering +import org.apache.gearpump.streaming.dsl.window.api.SlidingWindowFn +import org.apache.gearpump.streaming.dsl.window.impl.{DefaultWindowRunner, GroupAlsoByWindow, WindowRunner} +import org.apache.gearpump.streaming.task.{Task, TaskContext} + +import scala.concurrent.duration.FiniteDuration + +object ProcessingTimeTriggerTask { + case object Triggering +} + +/** + * This task triggers output on scheduled system time interval. + */ +class ProcessingTimeTriggerTask[IN, GROUP]( + groupBy: GroupAlsoByWindow[IN, GROUP], + windowRunner: WindowRunner, + taskContext: TaskContext, + userConfig: UserConfig) + extends Task(taskContext, userConfig) { + + def this(groupBy: GroupAlsoByWindow[IN, GROUP], + taskContext: TaskContext, userConfig: UserConfig) = { + this(groupBy, new DefaultWindowRunner(taskContext, userConfig, groupBy)(taskContext.system), + taskContext, userConfig) + } + + def this(taskContext: TaskContext, userConfig: UserConfig) = { + this(userConfig.getValue[GroupAlsoByWindow[IN, GROUP]]( + GEARPUMP_STREAMING_GROUPBY_FUNCTION)(taskContext.system).get, + taskContext, userConfig) + } + + private val windowFn = groupBy.window.windowFn.asInstanceOf[SlidingWindowFn] + private val windowSizeMs = windowFn.size.toMillis + private val windowStepMs = windowFn.step.toMillis + + override def onStart(startTime: Instant): Unit = { + val initialDelay = windowSizeMs - Instant.now.toEpochMilli % windowSizeMs + taskContext.scheduleOnce( + new FiniteDuration(initialDelay, TimeUnit.MILLISECONDS))(self ! Triggering) + } + + override def onNext(message: Message): Unit = { + windowRunner.process(message) + } + + override def receiveUnManagedMessage: Receive = { + case Triggering => + windowRunner.trigger(Instant.now) + taskContext.scheduleOnce( + new FiniteDuration(windowStepMs, TimeUnit.MILLISECONDS))(self ! Triggering) + } + +} diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/task/TransformTask.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/task/TransformTask.scala new file mode 100644 index 000000000..e35f08577 --- /dev/null +++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/task/TransformTask.scala @@ -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.gearpump.streaming.dsl.task + +import org.apache.gearpump.Message +import org.apache.gearpump.cluster.UserConfig +import org.apache.gearpump.streaming.Constants._ +import org.apache.gearpump.streaming.dsl.plan.functions.SingleInputFunction +import org.apache.gearpump.streaming.task.{Task, TaskContext} + +class TransformTask[IN, OUT]( + operator: Option[SingleInputFunction[IN, OUT]], taskContext: TaskContext, + userConf: UserConfig) extends Task(taskContext, userConf) { + + def this(taskContext: TaskContext, userConf: UserConfig) = { + this(userConf.getValue[SingleInputFunction[IN, OUT]]( + GEARPUMP_STREAMING_OPERATOR)(taskContext.system), taskContext, userConf) + } + + override def onNext(msg: Message): Unit = { + val time = msg.timestamp + + operator match { + case Some(op) => + op.process(msg.msg.asInstanceOf[IN]).foreach { msg => + taskContext.output(new Message(msg, time)) + } + case None => + taskContext.output(new Message(msg.msg, time)) + } + } +} diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/window/api/AccumulationMode.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/window/api/AccumulationMode.scala new file mode 100644 index 000000000..a4524a8e6 --- /dev/null +++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/window/api/AccumulationMode.scala @@ -0,0 +1,24 @@ +/* + * 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.gearpump.streaming.dsl.window.api + +sealed trait AccumulationMode + +case object Accumulating extends AccumulationMode + +case object Discarding extends AccumulationMode diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/window/api/GroupByFn.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/window/api/GroupByFn.scala new file mode 100644 index 000000000..30e68ba39 --- /dev/null +++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/window/api/GroupByFn.scala @@ -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.gearpump.streaming.dsl.window.api + +import akka.actor.ActorSystem +import org.apache.gearpump.Message +import org.apache.gearpump.cluster.UserConfig +import org.apache.gearpump.streaming.Processor +import org.apache.gearpump.streaming.task.Task + +/** + * Divides messages into groups according its payload and timestamp. + * Check [[org.apache.gearpump.streaming.dsl.window.impl.GroupAlsoByWindow]] + * for default implementation. + */ +trait GroupByFn[T, GROUP] { + + /** + * Used by + * 1. GroupByPartitioner to shuffle messages + * 2. WindowRunner to group messages for time-based aggregation + */ + def groupBy(message: Message): GROUP + + /** + * Returns a Processor according to window trigger during planning + */ + def getProcessor(parallelism: Int, description: String, + userConfig: UserConfig)(implicit system: ActorSystem): Processor[_ <: Task] +} + + diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/window/api/Trigger.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/window/api/Trigger.scala new file mode 100644 index 000000000..9865e18d2 --- /dev/null +++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/window/api/Trigger.scala @@ -0,0 +1,27 @@ +/* + * 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.gearpump.streaming.dsl.window.api + +sealed trait Trigger + +case object EventTimeTrigger extends Trigger + +case object ProcessingTimeTrigger extends Trigger + +case object CountTrigger extends Trigger + diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/window/api/Window.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/window/api/Window.scala new file mode 100644 index 000000000..4b94879bc --- /dev/null +++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/window/api/Window.scala @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gearpump.streaming.dsl.window.api + +import java.time.Duration + +/** + * + * @param windowFn + * @param trigger + * @param accumulationMode + */ +case class Window( + windowFn: WindowFn, + trigger: Trigger = EventTimeTrigger, + accumulationMode: AccumulationMode = Discarding) { + + def triggering(trigger: Trigger): Window = { + Window(windowFn, trigger) + } + + def accumulating: Window = { + Window(windowFn, trigger, Accumulating) + } + + def discarding: Window = { + Window(windowFn, trigger, Discarding) + } +} + +object CountWindow { + + def apply(size: Int): Window = { + Window(CountWindowFn(size), CountTrigger) + } +} + +object FixedWindow { + + /** + * Defines a FixedWindow. + * @param size window size + * @return a Window definition + */ + def apply(size: Duration): Window = { + Window(SlidingWindowFn(size, size)) + } +} + +object SlidingWindow { + + /** + * Defines a SlidingWindow + * @param size window size + * @param step window step to slide forward + * @return a Window definition + */ + def apply(size: Duration, step: Duration): Window = { + Window(SlidingWindowFn(size, step)) + } +} + diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/window/api/WindowFn.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/window/api/WindowFn.scala new file mode 100644 index 000000000..0768730c1 --- /dev/null +++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/window/api/WindowFn.scala @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gearpump.streaming.dsl.window.api + +import java.time.{Duration, Instant} + +import org.apache.gearpump.TimeStamp +import org.apache.gearpump.streaming.dsl.window.impl.Bucket + +import scala.collection.mutable.ArrayBuffer + +sealed trait WindowFn { + def apply(timestamp: Instant): List[Bucket] +} + +case class SlidingWindowFn(size: Duration, step: Duration) + extends WindowFn { + + def this(size: Duration) = { + this(size, size) + } + + override def apply(timestamp: Instant): List[Bucket] = { + val sizeMillis = size.toMillis + val stepMillis = step.toMillis + val timeMillis = timestamp.toEpochMilli + val windows = ArrayBuffer.empty[Bucket] + var start = lastStartFor(timeMillis, stepMillis) + windows += Bucket.ofEpochMilli(start, start + sizeMillis) + start -= stepMillis + while (start >= timeMillis) { + windows += Bucket.ofEpochMilli(start, start + sizeMillis) + start -= stepMillis + } + windows.toList + } + + private def lastStartFor(timestamp: TimeStamp, windowStep: Long): TimeStamp = { + timestamp - (timestamp + windowStep) % windowStep + } +} + +case class CountWindowFn(size: Int) extends WindowFn { + + override def apply(timestamp: Instant): List[Bucket] = { + List(Bucket.ofEpochMilli(0, size)) + } +} diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/window/impl/ReduceFnRunner.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/window/impl/ReduceFnRunner.scala new file mode 100644 index 000000000..e978983ff --- /dev/null +++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/window/impl/ReduceFnRunner.scala @@ -0,0 +1,29 @@ +/* + * 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.gearpump.streaming.dsl.window.impl + +import org.apache.gearpump.Message +import org.apache.gearpump.streaming.dsl.window.api.Trigger + +trait ReduceFnRunner { + + def process(message: Message): Unit + + def onTrigger(trigger: Trigger): Unit + +} diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/window/impl/Window.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/window/impl/Window.scala new file mode 100644 index 000000000..53cf5d07a --- /dev/null +++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/window/impl/Window.scala @@ -0,0 +1,75 @@ +/* + * 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.gearpump.streaming.dsl.window.impl + +import java.time.Instant + +import akka.actor.ActorSystem +import org.apache.gearpump.cluster.UserConfig +import org.apache.gearpump.streaming.Constants._ +import org.apache.gearpump.streaming.Processor +import org.apache.gearpump.{Message, TimeStamp} +import org.apache.gearpump.streaming.dsl.window.api._ +import org.apache.gearpump.streaming.dsl.task.{CountTriggerTask, EventTimeTriggerTask, ProcessingTimeTriggerTask} +import org.apache.gearpump.streaming.task.Task + +object Bucket { + def ofEpochMilli(startTime: TimeStamp, endTime: TimeStamp): Bucket = { + Bucket(Instant.ofEpochMilli(startTime), Instant.ofEpochMilli(endTime)) + } +} + +/** + * A window unit including startTime and excluding endTime. + */ +case class Bucket(startTime: Instant, endTime: Instant) extends Comparable[Bucket] { + override def compareTo(o: Bucket): Int = { + val ret = startTime.compareTo(o.startTime) + if (ret != 0) { + ret + } else { + endTime.compareTo(o.endTime) + } + } +} + +case class GroupAlsoByWindow[T, GROUP](groupByFn: T => GROUP, window: Window) + extends GroupByFn[T, (GROUP, List[Bucket])] { + + override def groupBy(message: Message): (GROUP, List[Bucket]) = { + val group = groupByFn(message.msg.asInstanceOf[T]) + val buckets = window.windowFn(Instant.ofEpochMilli(message.timestamp)) + group -> buckets + } + + override def getProcessor(parallelism: Int, description: String, + userConfig: UserConfig)(implicit system: ActorSystem): Processor[_ <: Task] = { + val config = userConfig.withValue(GEARPUMP_STREAMING_GROUPBY_FUNCTION, this) + window.trigger match { + case CountTrigger => + Processor[CountTriggerTask[T, GROUP]](parallelism, description, config) + case ProcessingTimeTrigger => + Processor[ProcessingTimeTriggerTask[T, GROUP]](parallelism, description, config) + case EventTimeTrigger => + Processor[EventTimeTriggerTask[T, GROUP]](parallelism, description, config) + } + } + +} + + diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/window/impl/WindowRunner.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/window/impl/WindowRunner.scala new file mode 100644 index 000000000..9af5e6148 --- /dev/null +++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/window/impl/WindowRunner.scala @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gearpump.streaming.dsl.window.impl + +import java.time.Instant + +import akka.actor.ActorSystem +import org.apache.gearpump.Message +import org.apache.gearpump.cluster.UserConfig +import org.apache.gearpump.gs.collections.api.block.procedure.Procedure +import org.apache.gearpump.gs.collections.impl.list.mutable.FastList +import org.apache.gearpump.gs.collections.impl.map.mutable.UnifiedMap +import org.apache.gearpump.gs.collections.impl.map.sorted.mutable.TreeSortedMap +import org.apache.gearpump.streaming.Constants._ +import org.apache.gearpump.streaming.dsl.plan.functions.{EmitFunction, SingleInputFunction} +import org.apache.gearpump.streaming.dsl.window.api.Discarding +import org.apache.gearpump.streaming.task.TaskContext +import org.apache.gearpump.util.LogUtil +import org.slf4j.Logger + +trait WindowRunner { + + def process(message: Message): Unit + + def trigger(time: Instant): Unit + +} + +object DefaultWindowRunner { + + private val LOG: Logger = LogUtil.getLogger(classOf[DefaultWindowRunner[_, _, _]]) + + case class WindowGroup[GROUP](bucket: Bucket, group: GROUP) + extends Comparable[WindowGroup[GROUP]] { + override def compareTo(o: WindowGroup[GROUP]): Int = { + val ret = bucket.compareTo(o.bucket) + if (ret != 0) { + ret + } else if (group.equals(o.group)) { + 0 + } else { + -1 + } + } + } +} + +class DefaultWindowRunner[IN, GROUP, OUT]( + taskContext: TaskContext, userConfig: UserConfig, + groupBy: GroupAlsoByWindow[IN, GROUP])(implicit system: ActorSystem) + extends WindowRunner { + import org.apache.gearpump.streaming.dsl.window.impl.DefaultWindowRunner._ + + private val windowGroups = new TreeSortedMap[WindowGroup[GROUP], FastList[IN]] + private val groupFns = new UnifiedMap[GROUP, SingleInputFunction[IN, OUT]] + + + override def process(message: Message): Unit = { + val (group, buckets) = groupBy.groupBy(message) + buckets.foreach { bucket => + val wg = WindowGroup(bucket, group) + val inputs = windowGroups.getOrDefault(wg, new FastList[IN](1)) + inputs.add(message.msg.asInstanceOf[IN]) + windowGroups.put(wg, inputs) + } + groupFns.putIfAbsent(group, + userConfig.getValue[SingleInputFunction[IN, OUT]](GEARPUMP_STREAMING_OPERATOR).get) + } + + override def trigger(time: Instant): Unit = { + onTrigger() + + @annotation.tailrec + def onTrigger(): Unit = { + if (windowGroups.notEmpty()) { + val first = windowGroups.firstKey + if (!time.isBefore(first.bucket.endTime)) { + val inputs = windowGroups.remove(first) + val reduceFn = groupFns.get(first.group) + .andThen[Unit](new EmitFunction[OUT](emitResult(_, time))) + inputs.forEach(new Procedure[IN] { + override def value(t: IN): Unit = { + reduceFn.process(t) + } + }) + reduceFn.finish() + if (groupBy.window.accumulationMode == Discarding) { + reduceFn.clearState() + } + onTrigger() + } + } + } + + def emitResult(result: OUT, time: Instant): Unit = { + taskContext.output(Message(result, time.toEpochMilli)) + } + } +} diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/source/DataSourceTask.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/source/DataSourceTask.scala index fb2d89817..535497c23 100644 --- a/streaming/src/main/scala/org/apache/gearpump/streaming/source/DataSourceTask.scala +++ b/streaming/src/main/scala/org/apache/gearpump/streaming/source/DataSourceTask.scala @@ -23,7 +23,7 @@ import java.time.Instant import org.apache.gearpump._ import org.apache.gearpump.cluster.UserConfig import org.apache.gearpump.streaming.Constants._ -import org.apache.gearpump.streaming.dsl.plan.OpTranslator.{DummyInputFunction, SingleInputFunction} +import org.apache.gearpump.streaming.dsl.plan.functions.SingleInputFunction import org.apache.gearpump.streaming.task.{Task, TaskContext} /** @@ -57,15 +57,10 @@ class DataSourceTask[IN, OUT] private[source]( private val processMessage: Message => Unit = operator match { case Some(op) => - op match { - case bad: DummyInputFunction[IN] => - (message: Message) => context.output(message) - case _ => - (message: Message) => { - op.process(message.msg.asInstanceOf[IN]).foreach { m: OUT => - context.output(Message(m, message.timestamp)) - } - } + (message: Message) => { + op.process(message.msg.asInstanceOf[IN]).foreach { m: OUT => + context.output(Message(m, message.timestamp)) + } } case None => (message: Message) => context.output(message) diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/task/TaskActor.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/task/TaskActor.scala index eb52700ab..f72e5b862 100644 --- a/streaming/src/main/scala/org/apache/gearpump/streaming/task/TaskActor.scala +++ b/streaming/src/main/scala/org/apache/gearpump/streaming/task/TaskActor.scala @@ -60,7 +60,7 @@ class TaskActor( val LOG: Logger = LogUtil.getLogger(getClass, app = appId, executor = executorId, task = taskId) // Metrics - private val metricName = s"app$appId.processor${taskId.processorId}.task${taskId.index}" + private val metricName = s"app${appId}.processor${taskId.processorId}.task${taskId.index}" private val receiveLatency = Metrics(context.system).histogram( s"$metricName:receiveLatency", sampleRate = 1) private val processTime = Metrics(context.system).histogram(s"$metricName:processTime") @@ -307,9 +307,9 @@ class TaskActor( private def updateUpstreamMinClock(upstreamClock: TimeStamp): Unit = { if (upstreamClock > this.upstreamMinClock) { + this.upstreamMinClock = upstreamClock task.onWatermarkProgress(Instant.ofEpochMilli(this.upstreamMinClock)) } - this.upstreamMinClock = upstreamClock val subMinClock = subscriptions.foldLeft(Long.MaxValue) { (min, sub) => val subMin = sub._2.minClock diff --git a/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/StreamAppSpec.scala b/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/StreamAppSpec.scala index e919a34cb..e0407ec37 100644 --- a/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/StreamAppSpec.scala +++ b/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/StreamAppSpec.scala @@ -21,7 +21,10 @@ package org.apache.gearpump.streaming.dsl import akka.actor.ActorSystem import org.apache.gearpump.cluster.TestUtil import org.apache.gearpump.cluster.client.ClientContext +import org.apache.gearpump.partitioner.PartitionerDescription +import org.apache.gearpump.streaming.{ProcessorDescription, StreamApplication} import org.apache.gearpump.streaming.source.DataSourceTask +import org.apache.gearpump.util.Graph import org.mockito.Mockito.when import org.scalatest._ import org.scalatest.mock.MockitoSugar @@ -30,7 +33,7 @@ import scala.concurrent.Await import scala.concurrent.duration.Duration class StreamAppSpec extends FlatSpec with Matchers with BeforeAndAfterAll with MockitoSugar { - implicit var system: ActorSystem = null + implicit var system: ActorSystem = _ override def beforeAll(): Unit = { system = ActorSystem("test", TestUtil.DEFAULT_CONFIG) @@ -45,49 +48,25 @@ class StreamAppSpec extends FlatSpec with Matchers with BeforeAndAfterAll with M val context: ClientContext = mock[ClientContext] when(context.system).thenReturn(system) - val app = StreamApp("dsl", context) - app.source(List("A"), 1, "") - app.source(List("B"), 1, "") + val dsl = StreamApp("dsl", context) + dsl.source(List("A"), 2, "A") shouldBe a [Stream[_]] + dsl.source(List("B"), 3, "B") shouldBe a [Stream[_]] - assert(app.graph.vertices.size == 2) - } - - it should "plan the dsl to Processsor(TaskDescription) DAG" in { - val context: ClientContext = mock[ClientContext] - when(context.system).thenReturn(system) - - val app = StreamApp("dsl", context) - val parallism = 3 - app.source(List("A", "B", "C"), parallism, "").flatMap(Array(_)).reduce(_ + _) - val task = app.plan.dag.vertices.iterator.next() - assert(task.taskClass == classOf[DataSourceTask[_, _]].getName) - assert(task.parallelism == parallism) - } - - it should "produce 3 messages" in { - val context: ClientContext = mock[ClientContext] - when(context.system).thenReturn(system) - val app = StreamApp("dsl", context) - val list = List[String]( - "0", - "1", - "2" - ) - val producer = app.source(list, 1, "producer").flatMap(Array(_)).reduce(_ + _) - val task = app.plan.dag.vertices.iterator.next() - /* - val task = app.plan.dag.vertices.iterator.map(desc => { - LOG.info(s"${desc.taskClass}") - }) - val sum = producer.flatMap(msg => { - LOG.info("in flatMap") - assert(msg.msg.isInstanceOf[String]) - val num = msg.msg.asInstanceOf[String].toInt - Array(num) - }).reduce(_+_) - val task = app.plan.dag.vertices.iterator.map(desc => { - LOG.info(s"${desc.taskClass}") - }) - */ + val application = dsl.plan() + application shouldBe a [StreamApplication] + application.name shouldBe "dsl" + val dag = application.userConfig + .getValue[Graph[ProcessorDescription, PartitionerDescription]](StreamApplication.DAG).get + dag.vertices.size shouldBe 2 + dag.vertices.foreach { processor => + processor.taskClass shouldBe classOf[DataSourceTask[_, _]].getName + if (processor.description == "A") { + processor.parallelism shouldBe 2 + } else if (processor.description == "B") { + processor.parallelism shouldBe 3 + } else { + fail(s"undefined source ${processor.description}") + } + } } } diff --git a/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/StreamSpec.scala b/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/StreamSpec.scala index 816feef91..fdc721b5a 100644 --- a/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/StreamSpec.scala +++ b/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/StreamSpec.scala @@ -22,10 +22,11 @@ import akka.actor._ import org.apache.gearpump.Message import org.apache.gearpump.cluster.client.ClientContext import org.apache.gearpump.cluster.{TestUtil, UserConfig} -import org.apache.gearpump.partitioner.{CoLocationPartitioner, HashPartitioner} +import org.apache.gearpump.partitioner.{CoLocationPartitioner, HashPartitioner, PartitionerDescription} +import org.apache.gearpump.streaming.{ProcessorDescription, StreamApplication} import org.apache.gearpump.streaming.dsl.StreamSpec.Join import org.apache.gearpump.streaming.dsl.partitioner.GroupByPartitioner -import org.apache.gearpump.streaming.dsl.plan.OpTranslator._ +import org.apache.gearpump.streaming.dsl.task.{CountTriggerTask, TransformTask} import org.apache.gearpump.streaming.source.DataSourceTask import org.apache.gearpump.streaming.task.{Task, TaskContext} import org.apache.gearpump.util.Graph @@ -40,7 +41,6 @@ import scala.util.{Either, Left, Right} class StreamSpec extends FlatSpec with Matchers with BeforeAndAfterAll with MockitoSugar { - implicit var system: ActorSystem = _ override def beforeAll(): Unit = { @@ -56,7 +56,7 @@ class StreamSpec extends FlatSpec with Matchers with BeforeAndAfterAll with Mock val context: ClientContext = mock[ClientContext] when(context.system).thenReturn(system) - val app = StreamApp("dsl", context) + val dsl = StreamApp("dsl", context) val data = """ @@ -66,30 +66,32 @@ class StreamSpec extends FlatSpec with Matchers with BeforeAndAfterAll with Mock five four five """ - val stream = app.source(data.lines.toList, 1, ""). + val stream = dsl.source(data.lines.toList, 1, ""). flatMap(line => line.split("[\\s]+")).filter(_.nonEmpty). map(word => (word, 1)). groupBy(_._1, parallelism = 2). reduce((left, right) => (left._1, left._2 + right._2)). map[Either[(String, Int), String]](Left(_)) - val query = app.source(List("two"), 1, "").map[Either[(String, Int), String]](Right(_)) + val query = dsl.source(List("two"), 1, "").map[Either[(String, Int), String]](Right(_)) stream.merge(query).process[(String, Int)](classOf[Join], 1) - val appDescription = app.plan() + val app: StreamApplication = dsl.plan() + val dag = app.userConfig + .getValue[Graph[ProcessorDescription, PartitionerDescription]](StreamApplication.DAG).get - val dagTopology = appDescription.dag.mapVertex(_.taskClass).mapEdge { (node1, edge, node2) => + val dagTopology = dag.mapVertex(_.taskClass).mapEdge { (node1, edge, node2) => edge.partitionerFactory.partitioner.getClass.getName } val expectedDagTopology = getExpectedDagTopology - assert(dagTopology.vertices.toSet.equals(expectedDagTopology.vertices.toSet)) - assert(dagTopology.edges.toSet.equals(expectedDagTopology.edges.toSet)) + dagTopology.vertices.toSet should contain theSameElementsAs expectedDagTopology.vertices.toSet + dagTopology.edges.toSet should contain theSameElementsAs expectedDagTopology.edges.toSet } private def getExpectedDagTopology: Graph[String, String] = { val source = classOf[DataSourceTask[_, _]].getName - val group = classOf[GroupByTask[_, _, _]].getName + val group = classOf[CountTriggerTask[_, _]].getName val merge = classOf[TransformTask[_, _]].getName val join = classOf[Join].getName diff --git a/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/partitioner/GroupByPartitionerSpec.scala b/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/partitioner/GroupByPartitionerSpec.scala index fcc646dd0..f49eb0496 100644 --- a/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/partitioner/GroupByPartitionerSpec.scala +++ b/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/partitioner/GroupByPartitionerSpec.scala @@ -18,24 +18,33 @@ package org.apache.gearpump.streaming.dsl.partitioner -import org.scalatest.{BeforeAndAfterAll, FlatSpec, Matchers} +import java.time.Duration +import org.scalatest.{BeforeAndAfterAll, FlatSpec, Matchers} import org.apache.gearpump.Message import org.apache.gearpump.streaming.dsl.partitioner.GroupByPartitionerSpec.People +import org.apache.gearpump.streaming.dsl.window.api.{FixedWindow, GroupByFn} +import org.apache.gearpump.streaming.dsl.window.impl.{Bucket, GroupAlsoByWindow} class GroupByPartitionerSpec extends FlatSpec with Matchers with BeforeAndAfterAll { - it should "use the outpout of groupBy function to do partition" in { + + it should "group by message payload and window" in { val mark = People("Mark", "male") val tom = People("Tom", "male") val michelle = People("Michelle", "female") val partitionNum = 10 - val groupBy = new GroupByPartitioner[People, String](_.gender) - assert(groupBy.getPartition(Message(mark), partitionNum) - == groupBy.getPartition(Message(tom), partitionNum)) + val groupByFn: GroupByFn[People, (String, List[Bucket])] = + GroupAlsoByWindow[People, String](_.gender, FixedWindow.apply(Duration.ofMillis(5))) + val groupBy = new GroupByPartitioner[People, (String, List[Bucket])](groupByFn) + groupBy.getPartition(Message(mark, 1L), partitionNum) shouldBe + groupBy.getPartition(Message(tom, 2L), partitionNum) + + groupBy.getPartition(Message(mark, 1L), partitionNum) should not be + groupBy.getPartition(Message(tom, 6L), partitionNum) - assert(groupBy.getPartition(Message(mark), partitionNum) - != groupBy.getPartition(Message(michelle), partitionNum)) + groupBy.getPartition(Message(mark, 2L), partitionNum) should not be + groupBy.getPartition(Message(michelle, 3L), partitionNum) } } diff --git a/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/plan/OpSpec.scala b/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/plan/OpSpec.scala new file mode 100644 index 000000000..bf52abcf0 --- /dev/null +++ b/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/plan/OpSpec.scala @@ -0,0 +1,244 @@ +/* + * 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.gearpump.streaming.dsl.plan + +import java.time.Instant + +import akka.actor.ActorSystem +import org.apache.gearpump.Message +import org.apache.gearpump.cluster.{TestUtil, UserConfig} +import org.apache.gearpump.streaming.Processor +import org.apache.gearpump.streaming.Processor.DefaultProcessor +import org.apache.gearpump.streaming.dsl.plan.OpSpec.{AnySink, AnySource, AnyTask} +import org.apache.gearpump.streaming.dsl.plan.functions.SingleInputFunction +import org.apache.gearpump.streaming.dsl.window.api.GroupByFn +import org.apache.gearpump.streaming.sink.DataSink +import org.apache.gearpump.streaming.source.DataSource +import org.apache.gearpump.streaming.task.{Task, TaskContext} +import org.mockito.Matchers._ +import org.mockito.Mockito._ +import org.scalatest.{BeforeAndAfterAll, Matchers, WordSpec} +import org.scalatest.mock.MockitoSugar + +import scala.concurrent.Await +import scala.concurrent.duration.Duration + +class OpSpec extends WordSpec with Matchers with BeforeAndAfterAll with MockitoSugar { + + private val unchainableOps: List[Op] = List( + mock[DataSourceOp], + mock[DataSinkOp], + mock[GroupByOp[Any, Any]], + mock[MergeOp], + mock[ProcessorOp[AnyTask]]) + + implicit var system: ActorSystem = _ + + override def beforeAll(): Unit = { + system = ActorSystem("test", TestUtil.DEFAULT_CONFIG) + } + + override def afterAll(): Unit = { + system.terminate() + Await.result(system.whenTerminated, Duration.Inf) + } + + "DataSourceOp" should { + + "chain ChainableOp" in { + val dataSource = new AnySource + val dataSourceOp = DataSourceOp(dataSource) + val chainableOp = mock[ChainableOp[Any, Any]] + val fn = mock[SingleInputFunction[Any, Any]] + + val chainedOp = dataSourceOp.chain(chainableOp) + + chainedOp shouldBe a[DataSourceOp] + verify(chainableOp).fn + + unchainableOps.foreach { op => + intercept[OpChainException] { + dataSourceOp.chain(op) + } + } + } + + "get Processor of DataSource" in { + val dataSource = new AnySource + val dataSourceOp = DataSourceOp(dataSource) + val processor = dataSourceOp.getProcessor + processor shouldBe a[Processor[_]] + processor.parallelism shouldBe dataSourceOp.parallelism + processor.description shouldBe dataSourceOp.description + } + } + + "DataSinkOp" should { + + "not chain any Op" in { + val dataSink = new AnySink + val dataSinkOp = DataSinkOp(dataSink) + val chainableOp = mock[ChainableOp[Any, Any]] + val ops = chainableOp +: unchainableOps + ops.foreach { op => + intercept[OpChainException] { + dataSinkOp.chain(op) + } + } + } + + "get Processor of DataSink" in { + val dataSink = new AnySink + val dataSinkOp = DataSinkOp(dataSink) + val processor = dataSinkOp.getProcessor + processor shouldBe a[Processor[_]] + processor.parallelism shouldBe dataSinkOp.parallelism + processor.description shouldBe dataSinkOp.description + } + } + + "ProcessorOp" should { + + "not chain any Op" in { + val processorOp = new ProcessorOp[AnyTask] + val chainableOp = mock[ChainableOp[Any, Any]] + val ops = chainableOp +: unchainableOps + ops.foreach { op => + intercept[OpChainException] { + processorOp.chain(op) + } + } + } + + "get Processor" in { + val processorOp = new ProcessorOp[AnyTask] + val processor = processorOp.getProcessor + processor shouldBe a [DefaultProcessor[_]] + processor.parallelism shouldBe processorOp.parallelism + processor.description shouldBe processorOp.description + } + } + + "ChainableOp" should { + + "chain ChainableOp" in { + val fn1 = mock[SingleInputFunction[Any, Any]] + val chainableOp1 = ChainableOp[Any, Any](fn1) + + val fn2 = mock[SingleInputFunction[Any, Any]] + val chainableOp2 = ChainableOp[Any, Any](fn2) + + val chainedOp = chainableOp1.chain(chainableOp2) + + verify(fn1).andThen(fn2) + chainedOp shouldBe a[ChainableOp[_, _]] + + unchainableOps.foreach { op => + intercept[OpChainException] { + chainableOp1.chain(op) + } + } + } + + "throw exception on getProcessor" in { + val fn1 = mock[SingleInputFunction[Any, Any]] + val chainableOp1 = ChainableOp[Any, Any](fn1) + intercept[UnsupportedOperationException] { + chainableOp1.getProcessor + } + } + } + + "GroupByOp" should { + + "chain ChainableOp" in { + val groupByFn = mock[GroupByFn[Any, Any]] + val groupByOp = GroupByOp[Any, Any](groupByFn) + val fn = mock[SingleInputFunction[Any, Any]] + val chainableOp = mock[ChainableOp[Any, Any]] + when(chainableOp.fn).thenReturn(fn) + + val chainedOp = groupByOp.chain(chainableOp) + chainedOp shouldBe a[GroupByOp[_, _]] + + unchainableOps.foreach { op => + intercept[OpChainException] { + groupByOp.chain(op) + } + } + } + + "delegate to groupByFn on getProcessor" in { + val groupByFn = mock[GroupByFn[Any, Any]] + val groupByOp = GroupByOp[Any, Any](groupByFn) + + groupByOp.getProcessor + verify(groupByFn).getProcessor(anyInt, anyString, any[UserConfig])(any[ActorSystem]) + } + } + + "MergeOp" should { + + val mergeOp = MergeOp("merge") + + "chain ChainableOp" in { + val fn = mock[SingleInputFunction[Any, Any]] + val chainableOp = mock[ChainableOp[Any, Any]] + when(chainableOp.fn).thenReturn(fn) + + val chainedOp = mergeOp.chain(chainableOp) + chainedOp shouldBe a [MergeOp] + + unchainableOps.foreach { op => + intercept[OpChainException] { + mergeOp.chain(op) + } + } + } + + "get Processor" in { + val processor = mergeOp.getProcessor + processor shouldBe a[Processor[_]] + processor.parallelism shouldBe 1 + } + } +} + +object OpSpec { + class AnyTask(context: TaskContext, config: UserConfig) extends Task(context, config) + + class AnySource extends DataSource { + + override def open(context: TaskContext, startTime: Instant): Unit = {} + + override def read(): Message = Message("any") + + override def close(): Unit = {} + + override def getWatermark: Instant = Instant.now() + } + + class AnySink extends DataSink { + + override def open(context: TaskContext): Unit = {} + + override def write(message: Message): Unit = {} + + override def close(): Unit = {} +} +} diff --git a/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/plan/OpTranslatorSpec.scala b/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/plan/OpTranslatorSpec.scala deleted file mode 100644 index 2112fd082..000000000 --- a/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/plan/OpTranslatorSpec.scala +++ /dev/null @@ -1,148 +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.gearpump.streaming.dsl.plan - -import java.time.Instant - -import scala.concurrent.Await -import scala.concurrent.duration.Duration -import akka.actor.ActorSystem -import org.mockito.ArgumentCaptor -import org.mockito.Matchers._ -import org.mockito.Mockito._ -import org.scalatest._ -import org.apache.gearpump.Message -import org.apache.gearpump.cluster.{TestUtil, UserConfig} -import org.apache.gearpump.streaming.Constants._ -import org.apache.gearpump.streaming.MockUtil -import org.apache.gearpump.streaming.dsl.CollectionDataSource -import org.apache.gearpump.streaming.dsl.plan.OpTranslator._ -import org.apache.gearpump.streaming.source.DataSourceTask - -class OpTranslatorSpec extends FlatSpec with Matchers with BeforeAndAfterAll { - - - "andThen" should "chain multiple single input function" in { - val dummy = new DummyInputFunction[String] - val split = new FlatMapFunction[String, String](line => line.split("\\s"), "split") - - val filter = new FlatMapFunction[String, String](word => - if (word.isEmpty) None else Some(word), "filter") - - val map = new FlatMapFunction[String, Int](word => Some(1), "map") - - val sum = new ReduceFunction[Int]({ (left, right) => left + right }, "sum") - - val all = dummy.andThen(split).andThen(filter).andThen(map).andThen(sum) - - assert(all.description == "split.filter.map.sum") - - val data = - """ - five four three two one - five four three two - five four three - five four - five - """ - val count = all.process(data).toList.last - assert(count == 15) - } - - "Source" should "iterate over input source and apply attached operator" in { - - val taskContext = MockUtil.mockTaskContext - implicit val actorSystem = MockUtil.system - - val data = "one two three".split("\\s") - val dataSource = new CollectionDataSource[String](data) - val conf = UserConfig.empty.withValue(GEARPUMP_STREAMING_SOURCE, dataSource) - - // Source with no transformer - val source = new DataSourceTask[String, String]( - taskContext, conf) - source.onStart(Instant.EPOCH) - source.onNext(Message("next")) - data.foreach { s => - verify(taskContext, times(1)).output(Message(s)) - } - - // Source with transformer - val anotherTaskContext = MockUtil.mockTaskContext - val double = new FlatMapFunction[String, String](word => List(word, word), "double") - val another = new DataSourceTask(anotherTaskContext, - conf.withValue(GEARPUMP_STREAMING_OPERATOR, double)) - another.onStart(Instant.EPOCH) - another.onNext(Message("next")) - data.foreach { s => - verify(anotherTaskContext, times(2)).output(Message(s)) - } - } - - "GroupByTask" should "group input by groupBy Function and " + - "apply attached operator for each group" in { - - val data = "1 2 2 3 3 3" - - val concat = new ReduceFunction[String]({ (left, right) => - left + right - }, "concat") - - implicit val system = ActorSystem("test", TestUtil.DEFAULT_CONFIG) - val config = UserConfig.empty.withValue[SingleInputFunction[String, String]]( - GEARPUMP_STREAMING_OPERATOR, concat) - - val taskContext = MockUtil.mockTaskContext - - val task = new GroupByTask[String, String, String](input => input, taskContext, config) - task.onStart(Instant.EPOCH) - - val peopleCaptor = ArgumentCaptor.forClass(classOf[Message]) - - data.split("\\s+").foreach { word => - task.onNext(Message(word)) - } - verify(taskContext, times(6)).output(peopleCaptor.capture()) - - import scala.collection.JavaConverters._ - - val values = peopleCaptor.getAllValues.asScala.map(input => input.msg.asInstanceOf[String]) - assert(values.mkString(",") == "1,2,22,3,33,333") - system.terminate() - Await.result(system.whenTerminated, Duration.Inf) - } - - "MergeTask" should "accept two stream and apply the attached operator" in { - - // Source with transformer - val taskContext = MockUtil.mockTaskContext - val conf = UserConfig.empty - val double = new FlatMapFunction[String, String](word => List(word, word), "double") - val task = new TransformTask[String, String](Some(double), taskContext, conf) - task.onStart(Instant.EPOCH) - - val data = "1 2 2 3 3 3".split("\\s+") - - data.foreach { input => - task.onNext(Message(input)) - } - - verify(taskContext, times(data.length * 2)).output(anyObject()) - } -} \ No newline at end of file diff --git a/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/plan/PlannerSpec.scala b/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/plan/PlannerSpec.scala new file mode 100644 index 000000000..f8666ba10 --- /dev/null +++ b/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/plan/PlannerSpec.scala @@ -0,0 +1,132 @@ +/* + * 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.gearpump.streaming.dsl.plan + +import java.time.Instant + +import akka.actor.ActorSystem +import org.apache.gearpump.Message +import org.apache.gearpump.cluster.{TestUtil, UserConfig} +import org.apache.gearpump.partitioner.CoLocationPartitioner +import org.apache.gearpump.streaming.dsl.partitioner.GroupByPartitioner +import org.apache.gearpump.streaming.dsl.plan.PlannerSpec._ +import org.apache.gearpump.streaming.dsl.plan.functions.{FlatMapFunction, ReduceFunction} +import org.apache.gearpump.streaming.dsl.window.api.GroupByFn +import org.apache.gearpump.streaming.sink.DataSink +import org.apache.gearpump.streaming.source.DataSource +import org.apache.gearpump.streaming.{MockUtil, Processor} +import org.apache.gearpump.streaming.task.{Task, TaskContext} +import org.apache.gearpump.util.Graph +import org.scalatest.mock.MockitoSugar +import org.scalatest.{BeforeAndAfterAll, FlatSpec, Matchers} + +import scala.concurrent.Await +import scala.concurrent.duration.Duration + +class PlannerSpec extends FlatSpec with Matchers with BeforeAndAfterAll with MockitoSugar { + + implicit var system: ActorSystem = _ + + override def beforeAll(): Unit = { + system = ActorSystem("test", TestUtil.DEFAULT_CONFIG) + } + + override def afterAll(): Unit = { + system.terminate() + Await.result(system.whenTerminated, Duration.Inf) + } + + "Planner" should "chain operations" in { + val graph = Graph.empty[Op, OpEdge] + val sourceOp = DataSourceOp(new AnySource) + val groupByOp = GroupByOp(new AnyGroupByFn) + val flatMapOp = ChainableOp[Any, Any](anyFlatMapFunction) + val reduceOp = ChainableOp[Any, Any](anyReduceFunction) + val processorOp = new ProcessorOp[AnyTask] + val sinkOp = DataSinkOp(new AnySink) + val directEdge = Direct + val shuffleEdge = Shuffle + + graph.addVertex(sourceOp) + graph.addVertex(groupByOp) + graph.addEdge(sourceOp, shuffleEdge, groupByOp) + graph.addVertex(flatMapOp) + graph.addEdge(groupByOp, directEdge, flatMapOp) + graph.addVertex(reduceOp) + graph.addEdge(flatMapOp, directEdge, reduceOp) + graph.addVertex(processorOp) + graph.addEdge(reduceOp, directEdge, processorOp) + graph.addVertex(sinkOp) + graph.addEdge(processorOp, directEdge, sinkOp) + + implicit val system = MockUtil.system + + val planner = new Planner + val plan = planner.plan(graph) + .mapVertex(_.description) + + plan.vertices.toSet should contain theSameElementsAs + Set("source", "groupBy", "processor", "sink") + plan.outgoingEdgesOf("source").iterator.next()._2 shouldBe a[GroupByPartitioner[_, _]] + plan.outgoingEdgesOf("groupBy").iterator.next()._2 shouldBe a[CoLocationPartitioner] + plan.outgoingEdgesOf("processor").iterator.next()._2 shouldBe a[CoLocationPartitioner] + } +} + +object PlannerSpec { + + private val anyParallelism = 1 + private val anyFlatMapFunction = new FlatMapFunction[Any, Any](Option(_), "flatMap") + private val anyReduceFunction = new ReduceFunction[Any]( + (left: Any, right: Any) => (left, right), "reduce") + + class AnyTask(context: TaskContext, config: UserConfig) extends Task(context, config) + + class AnySource extends DataSource { + + override def open(context: TaskContext, startTime: Instant): Unit = {} + + override def read(): Message = Message("any") + + override def close(): Unit = {} + + override def getWatermark: Instant = Instant.now() + } + + class AnySink extends DataSink { + + override def open(context: TaskContext): Unit = {} + + override def write(message: Message): Unit = {} + + override def close(): Unit = {} + } + + class AnyGroupByFn extends GroupByFn[Any, Any] { + + override def groupBy(message: Message): Any = message.msg + + override def getProcessor( + parallelism: Int, + description: String, + userConfig: UserConfig)(implicit system: ActorSystem): Processor[_ <: Task] = { + Processor[AnyTask](anyParallelism, description) + } + } +} diff --git a/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/plan/functions/SingleInputFunctionSpec.scala b/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/plan/functions/SingleInputFunctionSpec.scala new file mode 100644 index 000000000..94feae4d6 --- /dev/null +++ b/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/plan/functions/SingleInputFunctionSpec.scala @@ -0,0 +1,333 @@ +/* + * 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.gearpump.streaming.dsl.plan.functions + +import java.time.Instant + +import akka.actor.ActorSystem +import org.apache.gearpump.Message +import org.apache.gearpump.cluster.{TestUtil, UserConfig} +import org.apache.gearpump.streaming.MockUtil +import org.apache.gearpump.streaming.dsl.CollectionDataSource +import org.apache.gearpump.streaming.source.DataSourceTask +import org.apache.gearpump.streaming.Constants._ +import org.apache.gearpump.streaming.dsl.task.{CountTriggerTask, TransformTask} +import org.apache.gearpump.streaming.dsl.window.api.CountWindow +import org.apache.gearpump.streaming.dsl.window.impl.GroupAlsoByWindow +import org.mockito.ArgumentCaptor +import org.mockito.Matchers._ +import org.mockito.Mockito._ +import org.scalatest.{Matchers, WordSpec} +import org.scalatest.mock.MockitoSugar + +import scala.concurrent.Await +import scala.concurrent.duration.Duration + +class SingleInputFunctionSpec extends WordSpec with Matchers with MockitoSugar { + import org.apache.gearpump.streaming.dsl.plan.functions.SingleInputFunctionSpec._ + + "AndThen" should { + + val first = mock[SingleInputFunction[R, S]] + val second = mock[SingleInputFunction[S, T]] + val andThen = new AndThen(first, second) + + "chain first and second functions when processing input value" in { + val input = mock[R] + val firstOutput = mock[S] + val secondOutput = mock[T] + when(first.process(input)).thenReturn(Some(firstOutput)) + when(second.process(firstOutput)).thenReturn(Some(secondOutput)) + + andThen.process(input).toList shouldBe List(secondOutput) + } + + "return chained description" in { + when(first.description).thenReturn("first") + when(second.description).thenReturn("second") + andThen.description shouldBe "first.second" + } + + "return either first result or second on finish" in { + val firstResult = mock[S] + val processedFirst = mock[T] + val secondResult = mock[T] + + when(first.finish()).thenReturn(Some(firstResult)) + when(second.process(firstResult)).thenReturn(Some(processedFirst)) + andThen.finish().toList shouldBe List(processedFirst) + + when(first.finish()).thenReturn(None) + when(second.finish()).thenReturn(Some(secondResult)) + andThen.finish().toList shouldBe List(secondResult) + } + + "clear both states on clearState" in { + andThen.clearState() + + verify(first).clearState() + verify(second).clearState() + } + + "return AndThen on andThen" in { + val third = mock[SingleInputFunction[T, Any]] + andThen.andThen[Any](third) shouldBe an [AndThen[_, _, _]] + } + } + + "FlatMapFunction" should { + + val flatMap = mock[R => TraversableOnce[S]] + val flatMapFunction = new FlatMapFunction[R, S](flatMap, "flatMap") + + "call flatMap function when processing input value" in { + val input = mock[R] + flatMapFunction.process(input) + verify(flatMap).apply(input) + } + + "return passed in description" in { + flatMapFunction.description shouldBe "flatMap" + } + + "return None on finish" in { + flatMapFunction.finish() shouldBe List.empty[S] + } + + "do nothing on clearState" in { + flatMapFunction.clearState() + verifyZeroInteractions(flatMap) + } + + "return AndThen on andThen" in { + val other = mock[SingleInputFunction[S, T]] + flatMapFunction.andThen[T](other) shouldBe an [AndThen[_, _, _]] + } + } + + "ReduceFunction" should { + + + "call reduce function when processing input value" in { + val reduce = mock[(T, T) => T] + val reduceFunction = new ReduceFunction[T](reduce, "reduce") + val input1 = mock[T] + val input2 = mock[T] + val output = mock[T] + + when(reduce.apply(input1, input2)).thenReturn(output, output) + + reduceFunction.process(input1) shouldBe List.empty[T] + reduceFunction.process(input2) shouldBe List.empty[T] + reduceFunction.finish() shouldBe List(output) + + reduceFunction.clearState() + reduceFunction.process(input1) shouldBe List.empty[T] + reduceFunction.clearState() + reduceFunction.process(input2) shouldBe List.empty[T] + reduceFunction.finish() shouldBe List(input2) + } + + "return passed in description" in { + val reduce = mock[(T, T) => T] + val reduceFunction = new ReduceFunction[T](reduce, "reduce") + reduceFunction.description shouldBe "reduce" + } + + "return None on finish" in { + val reduce = mock[(T, T) => T] + val reduceFunction = new ReduceFunction[T](reduce, "reduce") + reduceFunction.finish() shouldBe List.empty[T] + } + + "do nothing on clearState" in { + val reduce = mock[(T, T) => T] + val reduceFunction = new ReduceFunction[T](reduce, "reduce") + reduceFunction.clearState() + verifyZeroInteractions(reduce) + } + + "return AndThen on andThen" in { + val reduce = mock[(T, T) => T] + val reduceFunction = new ReduceFunction[T](reduce, "reduce") + val other = mock[SingleInputFunction[T, Any]] + reduceFunction.andThen[Any](other) shouldBe an[AndThen[_, _, _]] + } + } + + "EmitFunction" should { + + val emit = mock[T => Unit] + val emitFunction = new EmitFunction[T](emit) + + "emit input value when processing input value" in { + val input = mock[T] + + emitFunction.process(input) shouldBe List.empty[Unit] + + verify(emit).apply(input) + } + + "return empty description" in { + emitFunction.description shouldBe "" + } + + "return None on finish" in { + emitFunction.finish() shouldBe List.empty[Unit] + } + + "do nothing on clearState" in { + emitFunction.clearState() + verifyZeroInteractions(emit) + } + + "throw exception on andThen" in { + val other = mock[SingleInputFunction[Unit, Any]] + intercept[UnsupportedOperationException] { + emitFunction.andThen(other) + } + } + } + + "andThen" should { + "chain multiple single input function" in { + val split = new FlatMapFunction[String, String](line => line.split("\\s"), "split") + + val filter = new FlatMapFunction[String, String](word => + if (word.isEmpty) None else Some(word), "filter") + + val map = new FlatMapFunction[String, Int](word => Some(1), "map") + + val sum = new ReduceFunction[Int]({ (left, right) => left + right }, "sum") + + val all = split.andThen(filter).andThen(map).andThen(sum) + + assert(all.description == "split.filter.map.sum") + + val data = + """ + five four three two one + five four three two + five four three + five four + five + """ + // force eager evaluation + all.process(data).toList + val result = all.finish().toList + assert(result.nonEmpty) + assert(result.last == 15) + } + } + + "Source" should { + "iterate over input source and apply attached operator" in { + + val taskContext = MockUtil.mockTaskContext + implicit val actorSystem = MockUtil.system + + val data = "one two three".split("\\s") + val dataSource = new CollectionDataSource[String](data) + val conf = UserConfig.empty.withValue(GEARPUMP_STREAMING_SOURCE, dataSource) + + // Source with no transformer + val source = new DataSourceTask[String, String]( + taskContext, conf) + source.onStart(Instant.EPOCH) + source.onNext(Message("next")) + data.foreach { s => + verify(taskContext, times(1)).output(MockUtil.argMatch[Message]( + message => message.msg == s)) + } + + // Source with transformer + val anotherTaskContext = MockUtil.mockTaskContext + val double = new FlatMapFunction[String, String](word => List(word, word), "double") + val another = new DataSourceTask(anotherTaskContext, + conf.withValue(GEARPUMP_STREAMING_OPERATOR, double)) + another.onStart(Instant.EPOCH) + another.onNext(Message("next")) + data.foreach { s => + verify(anotherTaskContext, times(2)).output(MockUtil.argMatch[Message]( + message => message.msg == s)) + } + } + } + + "CountTriggerTask" should { + "group input by groupBy Function and " + + "apply attached operator for each group" in { + + val data = "1 2 2 3 3 3" + + val concat = new ReduceFunction[String]({ (left, right) => + left + right + }, "concat") + + implicit val system = ActorSystem("test", TestUtil.DEFAULT_CONFIG) + val config = UserConfig.empty.withValue[SingleInputFunction[String, String]]( + GEARPUMP_STREAMING_OPERATOR, concat) + + val taskContext = MockUtil.mockTaskContext + + val groupBy = GroupAlsoByWindow((input: String) => input, CountWindow.apply(1).accumulating) + val task = new CountTriggerTask[String, String](groupBy, taskContext, config) + task.onStart(Instant.EPOCH) + + val peopleCaptor = ArgumentCaptor.forClass(classOf[Message]) + + data.split("\\s+").foreach { word => + task.onNext(Message(word)) + } + verify(taskContext, times(6)).output(peopleCaptor.capture()) + + import scala.collection.JavaConverters._ + + val values = peopleCaptor.getAllValues.asScala.map(input => input.msg.asInstanceOf[String]) + assert(values.mkString(",") == "1,2,22,3,33,333") + system.terminate() + Await.result(system.whenTerminated, Duration.Inf) + } + } + + "MergeTask" should { + "accept two stream and apply the attached operator" in { + + // Source with transformer + val taskContext = MockUtil.mockTaskContext + val conf = UserConfig.empty + val double = new FlatMapFunction[String, String](word => List(word, word), "double") + val task = new TransformTask[String, String](Some(double), taskContext, conf) + task.onStart(Instant.EPOCH) + + val data = "1 2 2 3 3 3".split("\\s+") + + data.foreach { input => + task.onNext(Message(input)) + } + + verify(taskContext, times(data.length * 2)).output(anyObject()) + } + } +} + +object SingleInputFunctionSpec { + type R = AnyRef + type S = AnyRef + type T = AnyRef +} diff --git a/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/task/CountTriggerTaskSpec.scala b/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/task/CountTriggerTaskSpec.scala new file mode 100644 index 000000000..871d75119 --- /dev/null +++ b/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/task/CountTriggerTaskSpec.scala @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gearpump.streaming.dsl.task + +import java.time.Instant + +import org.apache.gearpump.Message +import org.apache.gearpump.cluster.UserConfig +import org.apache.gearpump.streaming.MockUtil +import org.apache.gearpump.streaming.dsl.window.api.CountWindow +import org.apache.gearpump.streaming.dsl.window.impl.{GroupAlsoByWindow, WindowRunner} +import org.mockito.Mockito._ +import org.scalacheck.Gen +import org.scalatest.mock.MockitoSugar +import org.scalatest.prop.PropertyChecks +import org.scalatest.{Matchers, PropSpec} + +class CountTriggerTaskSpec extends PropSpec with PropertyChecks + with Matchers with MockitoSugar { + + property("CountTriggerTask should trigger output by number of messages in a window") { + + implicit val system = MockUtil.system + + val numGen = Gen.chooseNum[Int](1, 1000) + + forAll(numGen, numGen) { (windowSize: Int, msgNum: Int) => + + val groupBy = mock[GroupAlsoByWindow[Any, Any]] + val window = CountWindow.apply(windowSize) + when(groupBy.window).thenReturn(window) + val windowRunner = mock[WindowRunner] + val userConfig = UserConfig.empty + + val task = new CountTriggerTask[Any, Any](groupBy, windowRunner, + MockUtil.mockTaskContext, userConfig) + val message = mock[Message] + + for (i <- 1 to msgNum) { + task.onNext(message) + } + verify(windowRunner, times(msgNum)).process(message) + verify(windowRunner, times(msgNum / windowSize)).trigger(Instant.ofEpochMilli(windowSize)) + } + } +} diff --git a/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/task/EventTimeTriggerTaskSpec.scala b/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/task/EventTimeTriggerTaskSpec.scala new file mode 100644 index 000000000..a69abe6f0 --- /dev/null +++ b/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/task/EventTimeTriggerTaskSpec.scala @@ -0,0 +1,66 @@ +/* + * 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.gearpump.streaming.dsl.task + +import java.time.{Duration, Instant} + +import org.apache.gearpump.Message +import org.apache.gearpump.cluster.UserConfig +import org.apache.gearpump.streaming.MockUtil +import org.apache.gearpump.streaming.dsl.window.api.{EventTimeTrigger, SlidingWindow} +import org.apache.gearpump.streaming.dsl.window.impl.{GroupAlsoByWindow, WindowRunner} +import org.mockito.Matchers._ +import org.mockito.Mockito._ +import org.scalacheck.Gen +import org.scalatest.{Matchers, PropSpec} +import org.scalatest.mock.MockitoSugar +import org.scalatest.prop.PropertyChecks + +class EventTimeTriggerTaskSpec extends PropSpec with PropertyChecks + with Matchers with MockitoSugar { + + property("EventTimeTriggerTask should trigger on watermark") { + val longGen = Gen.chooseNum[Long](1L, 1000L) + val windowSizeGen = longGen + val windowStepGen = longGen + val watermarkGen = longGen.map(Instant.ofEpochMilli) + + forAll(windowSizeGen, windowStepGen, watermarkGen) { + (windowSize: Long, windowStep: Long, watermark: Instant) => + + val window = SlidingWindow.apply(Duration.ofMillis(windowSize), + Duration.ofMillis(windowStep)).triggering(EventTimeTrigger) + val groupBy = mock[GroupAlsoByWindow[Any, Any]] + val windowRunner = mock[WindowRunner] + val context = MockUtil.mockTaskContext + val config = UserConfig.empty + + when(groupBy.window).thenReturn(window) + + val task = new EventTimeTriggerTask[Any, Any](groupBy, windowRunner, context, config) + + val message = mock[Message] + task.onNext(message) + verify(windowRunner).process(message) + + task.onWatermarkProgress(watermark) + verify(windowRunner).trigger(any[Instant]) + } + } + +} diff --git a/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/task/ProcessingTimeTriggerTaskSpec.scala b/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/task/ProcessingTimeTriggerTaskSpec.scala new file mode 100644 index 000000000..39e1b4ce4 --- /dev/null +++ b/streaming/src/test/scala/org/apache/gearpump/streaming/dsl/task/ProcessingTimeTriggerTaskSpec.scala @@ -0,0 +1,69 @@ +/* + * 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.gearpump.streaming.dsl.task + +import java.time.{Duration, Instant} + +import org.apache.gearpump.Message +import org.apache.gearpump.cluster.UserConfig +import org.apache.gearpump.streaming.MockUtil +import org.apache.gearpump.streaming.dsl.task.ProcessingTimeTriggerTask.Triggering +import org.apache.gearpump.streaming.dsl.window.api.{ProcessingTimeTrigger, SlidingWindow} +import org.apache.gearpump.streaming.dsl.window.impl.{GroupAlsoByWindow, WindowRunner} +import org.mockito.Matchers._ +import org.mockito.Mockito._ +import org.scalacheck.Gen +import org.scalatest.{Matchers, PropSpec} +import org.scalatest.mock.MockitoSugar +import org.scalatest.prop.PropertyChecks + +class ProcessingTimeTriggerTaskSpec extends PropSpec with PropertyChecks + with Matchers with MockitoSugar { + + property("ProcessingTimeTriggerTask should trigger on system time interval") { + val longGen = Gen.chooseNum[Long](1L, 1000L) + val windowSizeGen = longGen + val windowStepGen = longGen + val startTimeGen = longGen.map(Instant.ofEpochMilli) + + forAll(windowSizeGen, windowStepGen, startTimeGen) { + (windowSize: Long, windowStep: Long, startTime: Instant) => + + val window = SlidingWindow.apply(Duration.ofMillis(windowSize), + Duration.ofMillis(windowStep)).triggering(ProcessingTimeTrigger) + val groupBy = mock[GroupAlsoByWindow[Any, Any]] + val windowRunner = mock[WindowRunner] + val context = MockUtil.mockTaskContext + val config = UserConfig.empty + + when(groupBy.window).thenReturn(window) + + val task = new ProcessingTimeTriggerTask[Any, Any](groupBy, windowRunner, context, config) + + task.onStart(startTime) + + val message = mock[Message] + task.onNext(message) + verify(windowRunner).process(message) + + task.receiveUnManagedMessage(Triggering) + verify(windowRunner).trigger(any[Instant]) + } + } + +} From f1bec67098b80f9bffe719f885f8d089344a5579 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Wed, 12 Oct 2016 14:30:49 +0800 Subject: [PATCH 3/9] update akkastream against latests Graph DSL Author: manuzhang Closes #97 from manuzhang/akka-streams-new. --- .../src/main/resources/geardefault.conf | 2 +- .../akkastream/GearpumpMaterializer.scala | 64 ++++++++++--------- .../gearpump/akkastream/example/Test.scala | 3 +- .../akkastream/example/WikipediaApp.scala | 12 ++-- .../materializer/RemoteMaterializerImpl.scala | 40 ++++++------ .../gearpump/akkastream/scaladsl/Api.scala | 2 +- .../gearpump/akkastream/task/Unzip2Task.scala | 2 +- project/Build.scala | 18 ++---- project/Pack.scala | 16 ++--- .../streaming/StreamApplication.scala | 2 +- .../gearpump/streaming/dsl/plan/OP.scala | 19 ++++-- 11 files changed, 92 insertions(+), 88 deletions(-) diff --git a/experiments/akkastream/src/main/resources/geardefault.conf b/experiments/akkastream/src/main/resources/geardefault.conf index e9da531b0..56524d455 100644 --- a/experiments/akkastream/src/main/resources/geardefault.conf +++ b/experiments/akkastream/src/main/resources/geardefault.conf @@ -4,5 +4,5 @@ gearpump.serializers { "scala.collection.immutable.Map$Map2" = "" } akka { - version = "2.4.10" + version = "2.4.11" } diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearpumpMaterializer.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearpumpMaterializer.scala index 75dc95a2c..9ff701ccf 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearpumpMaterializer.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearpumpMaterializer.scala @@ -29,14 +29,13 @@ import akka.stream.impl.Stages.SymbolicGraphStage import akka.stream.impl.StreamLayout._ import akka.stream.impl._ import akka.stream.impl.fusing.{GraphInterpreterShell, GraphStageModule} -import akka.stream.scaladsl.ModuleExtractor import akka.stream.stage.GraphStage import org.apache.gearpump.akkastream.GearpumpMaterializer.Edge import org.apache.gearpump.akkastream.graph.GraphPartitioner.Strategy import org.apache.gearpump.akkastream.graph.LocalGraph.LocalGraphMaterializer import org.apache.gearpump.akkastream.graph.RemoteGraph.RemoteGraphMaterializer import org.apache.gearpump.akkastream.graph._ -import org.apache.gearpump.akkastream.util.MaterializedValueOps +import org.apache.gearpump.util.{Graph => GGraph} import scala.collection.mutable import scala.concurrent.{ExecutionContextExecutor, Promise} @@ -137,7 +136,7 @@ class GearpumpMaterializer(override val system: ActorSystem, override def logger: LoggingAdapter = Logging.getLogger(system, this) - override def isShutdown: Boolean = system.isTerminated + override def isShutdown: Boolean = system.whenTerminated.isCompleted override def effectiveSettings(opAttr: Attributes): ActorMaterializerSettings = { import ActorAttributes._ @@ -177,7 +176,6 @@ class GearpumpMaterializer(override val system: ActorSystem, Nil) val info = Fusing.aggressive(runnableGraph).module.info - import _root_.org.apache.gearpump.util.{Graph => GGraph} val graph = GGraph.empty[Module, Edge] info.allModules.foreach(module => { @@ -204,33 +202,7 @@ class GearpumpMaterializer(override val system: ActorSystem, }) if(Debug) { - val iterator = graph.topologicalOrderIterator - while (iterator.hasNext) { - val module = iterator.next() - // scalastyle:off println - module match { - case graphStageModule: GraphStageModule => - graphStageModule.stage match { - case symbolicGraphStage: SymbolicGraphStage[_, _, _] => - val symbolicName = symbolicGraphStage.symbolicStage.getClass.getSimpleName - println( - s"${module.getClass.getSimpleName}(${symbolicName})" - ) - case graphStage: GraphStage[_] => - val name = graphStage.getClass.getSimpleName - println( - s"${module.getClass.getSimpleName}(${name})" - ) - case other => - println( - s"${module.getClass.getSimpleName}(${other.getClass.getSimpleName})" - ) - } - case _ => - println(module.getClass.getSimpleName) - } - // scalastyle:on println - } + printGraph(graph) } val subGraphs = GraphPartitioner(strategy).partition(graph) @@ -266,6 +238,36 @@ class GearpumpMaterializer(override val system: ActorSystem, rt.getOrElse(null).asInstanceOf[Mat] } + private def printGraph(graph: GGraph[Module, Edge]): Unit = { + val iterator = graph.topologicalOrderIterator + while (iterator.hasNext) { + val module = iterator.next() + // scalastyle:off println + module match { + case graphStageModule: GraphStageModule => + graphStageModule.stage match { + case symbolicGraphStage: SymbolicGraphStage[_, _, _] => + val symbolicName = symbolicGraphStage.symbolicStage.getClass.getSimpleName + println( + s"${module.getClass.getSimpleName}(${symbolicName})" + ) + case graphStage: GraphStage[_] => + val name = graphStage.getClass.getSimpleName + println( + s"${module.getClass.getSimpleName}(${name})" + ) + case other => + println( + s"${module.getClass.getSimpleName}(${other.getClass.getSimpleName})" + ) + } + case _ => + println(module.getClass.getSimpleName) + } + // scalastyle:on println + } + } + override def materialize[Mat](runnableGraph: Graph[ClosedShape, Mat], subflowFuser: GraphInterpreterShell => ActorRef): Mat = { materialize(runnableGraph) diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test.scala index 2ce4e1905..40cd556e1 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test.scala @@ -21,6 +21,7 @@ package org.apache.gearpump.akkastream.example import akka.actor.{Actor, ActorSystem, Props} import akka.stream.scaladsl.{Sink, Source} import org.apache.gearpump.akkastream.GearpumpMaterializer +import org.apache.gearpump.akkastream.graph.GraphPartitioner import org.apache.gearpump.cluster.main.ArgumentsParser import org.apache.gearpump.util.AkkaApp @@ -37,7 +38,7 @@ object Test extends AkkaApp with ArgumentsParser { // scalastyle:off println override def main(akkaConf: Config, args: Array[String]): Unit = { implicit val system = ActorSystem("Test", akkaConf) - implicit val materializer = GearpumpMaterializer() + implicit val materializer = GearpumpMaterializer(GraphPartitioner.AllRemoteStrategy) val echo = system.actorOf(Props(new Echo())) val sink = Sink.actorRef(echo, "COMPLETE") diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/WikipediaApp.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/WikipediaApp.scala index 7e2211d42..830f27810 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/WikipediaApp.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/WikipediaApp.scala @@ -81,13 +81,11 @@ object WikipediaApp extends ArgumentsParser with AkkaApp { } ) - g.run().onComplete { x => - x match { - case Success((t, f)) => printResults(t, f) - // scalastyle:off println - case Failure(tr) => println("Something went wrong") - // scalastyle:on println - } + g.run().onComplete { + case Success((t, f)) => printResults(t, f) + // scalastyle:off println + case Failure(tr) => println("Something went wrong") + // scalastyle:on println } Await.result(system.whenTerminated, 60.minutes) } diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/materializer/RemoteMaterializerImpl.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/materializer/RemoteMaterializerImpl.scala index f3f80947a..936ac2918 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/materializer/RemoteMaterializerImpl.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/materializer/RemoteMaterializerImpl.scala @@ -28,14 +28,16 @@ import akka.stream.impl.{HeadOptionStage, Stages, Throttle} import akka.stream.scaladsl._ import akka.stream.stage.AbstractStage.PushPullGraphStageWithMaterializedValue import akka.stream.stage.GraphStage -import akka.stream.{FanInShape, FanOutShape} import org.apache.gearpump.akkastream.GearAttributes import org.apache.gearpump.akkastream.GearpumpMaterializer.Edge import org.apache.gearpump.akkastream.module._ import org.apache.gearpump.akkastream.task._ import org.apache.gearpump.cluster.UserConfig import org.apache.gearpump.streaming.dsl.StreamApp -import org.apache.gearpump.streaming.dsl.op._ +import org.apache.gearpump.streaming.dsl.plan._ +import org.apache.gearpump.streaming.dsl.plan.functions.FlatMapFunction +import org.apache.gearpump.streaming.dsl.window.api.CountWindow +import org.apache.gearpump.streaming.dsl.window.impl.GroupAlsoByWindow import org.apache.gearpump.streaming.{ProcessorId, StreamApplication} import org.apache.gearpump.util.Graph import org.slf4j.LoggerFactory @@ -96,14 +98,14 @@ class RemoteMaterializerImpl(graph: Graph[Module, Edge], system: ActorSystem) { vertex.shape.inlets.flatMap { inlet => graph.incomingEdgesOf(vertex).find( _._2.to == inlet).map(_._1 - ).flatMap(processorIds.get(_)) + ).flatMap(processorIds.get) }.toList } def outProcessors(vertex: Module): List[ProcessorId] = { vertex.shape.outlets.flatMap { outlet => graph.outgoingEdgesOf(vertex).find( _._2.from == outlet).map(_._3 - ).flatMap(processorIds.get(_)) + ).flatMap(processorIds.get) }.toList } processorIds.get(vertex).map(processorId => { @@ -165,6 +167,8 @@ class RemoteMaterializerImpl(graph: Graph[Module, Edge], system: ActorSystem) { reduceOp(reduce.f, conf) case graphStage: GraphStageModule => translateGraphStageWithMaterializedValue(graphStage, parallelism, conf) + case _ => + null } if (op == null) { throw new UnsupportedOperationException( @@ -174,12 +178,11 @@ class RemoteMaterializerImpl(graph: Graph[Module, Edge], system: ActorSystem) { op }.mapEdge[OpEdge] { (n1, edge, n2) => n2 match { - case master: MasterOp => - Shuffle - case slave: SlaveOp[_] if n1.isInstanceOf[ProcessorOp[_]] => - Shuffle - case slave: SlaveOp[_] => + case chainableOp: ChainableOp[_, _] + if !n1.isInstanceOf[ProcessorOp[_]] && !n2.isInstanceOf[ProcessorOp[_]] => Direct + case _ => + Shuffle } } (opGraph, matValues.toMap) @@ -237,7 +240,8 @@ class RemoteMaterializerImpl(graph: Graph[Module, Edge], system: ActorSystem) { withValue(FoldTask.AGGREGATOR, fold.f) ProcessorOp(classOf[FoldTask[_, _]], parallelism, foldConf, "fold") case groupBy: GroupBy[_, _] => - GroupByOp(groupBy.keyFor, groupBy.maxSubstreams, "groupBy", conf) + GroupByOp(GroupAlsoByWindow(groupBy.keyFor, CountWindow.apply(1).accumulating), + groupBy.maxSubstreams, "groupBy", conf) case groupedWithin: GroupedWithin[_] => val diConf = conf.withValue[FiniteDuration](GroupedWithinTask.TIME_WINDOW, groupedWithin.d). withInt(GroupedWithinTask.BATCH_SIZE, groupedWithin.n) @@ -318,11 +322,11 @@ class RemoteMaterializerImpl(graph: Graph[Module, Edge], system: ActorSystem) { // TODO null case unzip: Unzip[_, _] => - ProcessorOp(classOf[Unzip2Task[_, _, _]], - parallelism, - conf.withValue( - Unzip2Task.UNZIP2_FUNCTION, Unzip2Task.UnZipFunction(unzip.unzipper) - ), "unzip") +// ProcessorOp(classOf[Unzip2Task[_, _, _]], parallelism, +// conf.withValue( +// Unzip2Task.UNZIP2_FUNCTION, Unzip2Task.UnZipFunction(unzip.unzipper)), "unzip") + // TODO + null case zip: Zip[_, _] => zipWithOp(zip.zipper, conf) case zipWith2: ZipWith2[_, _, _] => @@ -474,10 +478,10 @@ object RemoteMaterializerImpl { def flatMapOp[In, Out](fun: In => TraversableOnce[Out], description: String, conf: UserConfig): Op = { - FlatMapOp(fun, description, conf) + ChainableOp(new FlatMapFunction[In, Out](fun, description), conf) } - def conflatOp[In, Out](seed: In => Out, aggregate: (Out, In) => Out, + def conflateOp[In, Out](seed: In => Out, aggregate: (Out, In) => Out, conf: UserConfig): Op = { var agg = None: Option[Out] val flatMap = {elem: In => @@ -489,7 +493,7 @@ object RemoteMaterializerImpl { } List(agg.get) } - flatMapOp (flatMap, "conflat", conf) + flatMapOp (flatMap, "conflate", conf) } def foldOp[In, Out](zero: Out, fold: (Out, In) => Out, conf: UserConfig): Op = { diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/scaladsl/Api.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/scaladsl/Api.scala index 85b1d5e7a..80619ef51 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/scaladsl/Api.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/scaladsl/Api.scala @@ -56,7 +56,7 @@ object GearSource{ * */ def from[OUT](source: DataSource): Source[OUT, Unit] = { - val taskSource = new Source[OUT, Unit](new SourceTaskModule(source, UserConfig.empty)) + val taskSource = new Source[OUT, Unit](SourceTaskModule(source, UserConfig.empty)) taskSource } diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/Unzip2Task.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/Unzip2Task.scala index 99f1b550f..7dd91fcd9 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/Unzip2Task.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/Unzip2Task.scala @@ -40,7 +40,7 @@ class Unzip2Task[In, A1, A2](context: TaskContext, userConf : UserConfig) } object Unzip2Task { - case class UnZipFunction[In, A1, A2](val unzip: In => (A1, A2)) extends Serializable + case class UnZipFunction[In, A1, A2](unzip: In => (A1, A2)) extends Serializable val UNZIP2_FUNCTION = "org.apache.gearpump.akkastream.task.unzip2.function" } diff --git a/project/Build.scala b/project/Build.scala index f1e0443d1..a1e6ca55a 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -35,8 +35,7 @@ object Build extends sbt.Build { val copySharedSourceFiles = TaskKey[Unit]("copied shared services source code") - val akkaVersion = "2.4.10" - val akkaStreamVersion = "2.4-SNAPSHOT" + val akkaVersion = "2.4.11" val apacheRepo = "https://repository.apache.org/" val hadoopVersion = "2.6.0" val hbaseVersion = "1.0.0" @@ -148,10 +147,8 @@ object Build extends sbt.Build { "commons-logging" % "commons-logging" % commonsLoggingVersion, "com.typesafe.akka" %% "akka-distributed-data-experimental" % akkaVersion exclude("com.typesafe.akka", "akka-stream_2.11"), - "com.typesafe.akka" %% "akka-stream" % akkaStreamVersion, "org.apache.hadoop" % "hadoop-common" % hadoopVersion % "provided" - ), - dependencyOverrides += "com.typesafe.akka" %% "akka-stream" % akkaStreamVersion + ) ) val coreDependencies = Seq( @@ -353,7 +350,6 @@ object Build extends sbt.Build { "com.github.scribejava" % "scribejava-apis" % "2.4.0", "com.ning" % "async-http-client" % "1.9.33", "org.webjars" % "angularjs" % "1.4.9", - "org.apache.hadoop" % "hadoop-common" % hadoopVersion, // angular 1.5 breaks ui-select, but we need ng-touch 1.5 "org.webjars.npm" % "angular-touch" % "1.5.0", @@ -417,12 +413,12 @@ object Build extends sbt.Build { settings = commonSettings ++ noPublish ++ Seq( libraryDependencies ++= Seq( - "com.typesafe.akka" %% "akka-stream" % akkaStreamVersion, + "com.typesafe.akka" %% "akka-stream" % akkaVersion, + "org.apache.hadoop" % "hadoop-common" % hadoopVersion, "org.json4s" %% "json4s-jackson" % "3.2.11", "org.scalatest" %% "scalatest" % scalaTestVersion % "test" - ), - dependencyOverrides += "com.typesafe.akka" %% "akka-stream" % akkaStreamVersion - )) + ) + )) .dependsOn (services % "test->test; compile->compile", daemon % "test->test; compile->compile") .disablePlugins(sbtassembly.AssemblyPlugin) @@ -436,7 +432,7 @@ object Build extends sbt.Build { ), mainClass in(Compile, packageBin) := Some("org.apache.gearpump.example.Test") )) - .dependsOn(streaming % "test->test; provided", daemon % "test->test; provided") + .dependsOn(streaming % "test->test; provided", daemon % "test->test; provided") lazy val storm = Project( id = "gearpump-experiments-storm", diff --git a/project/Pack.scala b/project/Pack.scala index 47d30647b..1c87653e1 100644 --- a/project/Pack.scala +++ b/project/Pack.scala @@ -69,8 +69,7 @@ object Pack extends sbt.Build { "worker" -> "org.apache.gearpump.cluster.main.Worker", "services" -> "org.apache.gearpump.services.main.Services", "yarnclient" -> "org.apache.gearpump.experiments.yarn.client.Client", - "storm" -> "org.apache.gearpump.experiments.storm.StormRunner", - "akkastream" -> "org.apache.gearpump.akkastream.example.Test11" + "storm" -> "org.apache.gearpump.experiments.storm.StormRunner" ), packJvmOpts := Map( "gear" -> Seq("-Djava.net.preferIPv4Stack=true", "-Dgearpump.home=${PROG_HOME}"), @@ -110,13 +109,7 @@ object Pack extends sbt.Build { "storm" -> Seq( "-server", "-Djava.net.preferIPv4Stack=true", - "-Dgearpump.home=${PROG_HOME}"), - - "akkastream" -> Seq( - "-server", - "-Djava.net.preferIPv4Stack=true", - "-Dgearpump.home=${PROG_HOME}", - "-Djava.rmi.server.hostname=localhost") + "-Dgearpump.home=${PROG_HOME}") ), packLibDir := Map( "lib" -> new ProjectsToPack(core.id, streaming.id), @@ -148,14 +141,13 @@ object Pack extends sbt.Build { "worker" -> daemonClassPath, "services" -> serviceClassPath, "yarnclient" -> yarnClassPath, - "storm" -> stormClassPath, - "akkstream" -> daemonClassPath + "storm" -> stormClassPath ), packArchivePrefix := projectName + "-" + scalaBinaryVersion.value, packArchiveExcludes := Seq("integrationtest") ) - ).dependsOn(core, streaming, services, yarn, storm, akkastream). + ).dependsOn(core, streaming, services, yarn, storm). disablePlugins(sbtassembly.AssemblyPlugin) } diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/StreamApplication.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/StreamApplication.scala index a6588a14e..66ec87303 100644 --- a/streaming/src/main/scala/org/apache/gearpump/streaming/StreamApplication.scala +++ b/streaming/src/main/scala/org/apache/gearpump/streaming/StreamApplication.scala @@ -123,7 +123,7 @@ object LifeTime { */ class StreamApplication( override val name: String, val inputUserConfig: UserConfig, - dag: Graph[ProcessorDescription, PartitionerDescription]) + val dag: Graph[ProcessorDescription, PartitionerDescription]) extends Application { require(!dag.hasDuplicatedEdge(), "Graph should not have duplicated edges") diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/OP.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/OP.scala index 744976b7c..b2c55065b 100644 --- a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/OP.scala +++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/OP.scala @@ -25,7 +25,8 @@ import org.apache.gearpump.streaming.Processor.DefaultProcessor import org.apache.gearpump.streaming.dsl.plan.functions.SingleInputFunction import org.apache.gearpump.streaming.{Constants, Processor} import org.apache.gearpump.streaming.dsl.task.TransformTask -import org.apache.gearpump.streaming.dsl.window.api.GroupByFn +import org.apache.gearpump.streaming.dsl.window.api.{CountWindow, GroupByFn} +import org.apache.gearpump.streaming.dsl.window.impl.GroupAlsoByWindow import org.apache.gearpump.streaming.sink.{DataSink, DataSinkProcessor} import org.apache.gearpump.streaming.source.{DataSource, DataSourceTask} import org.apache.gearpump.streaming.task.Task @@ -124,11 +125,11 @@ case class DataSinkOp( * to another Op to be used */ case class ChainableOp[IN, OUT]( - fn: SingleInputFunction[IN, OUT]) extends Op { + fn: SingleInputFunction[IN, OUT], + userConfig: UserConfig = UserConfig.empty) extends Op { override def description: String = fn.description - override def userConfig: UserConfig = UserConfig.empty override def chain(other: Op)(implicit system: ActorSystem): Op = { other match { @@ -141,7 +142,17 @@ case class ChainableOp[IN, OUT]( } override def getProcessor(implicit system: ActorSystem): Processor[_ <: Task] = { - throw new UnsupportedOperationException("ChainedOp cannot be translated to Processor") + Processor[TransformTask[Any, Any]](1, description, + userConfig.withValue(Constants.GEARPUMP_STREAMING_OPERATOR, fn)) + } +} + +object GroupByOp { + + def apply[IN, GROUP](groupBy: IN => GROUP, parallelism: Int, + description: String, userConfig: UserConfig): Op = { + GroupByOp(GroupAlsoByWindow(groupBy, CountWindow.apply(1).accumulating), parallelism, + description, userConfig) } } From 3e61236faef44bf93d7943e8eea76c8cdfca5ac2 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Fri, 20 Jan 2017 13:38:31 +0800 Subject: [PATCH 4/9] Fix compile errors --- .../akkastream/GearpumpMaterializer.scala | 49 ++++++++------- .../gearpump/akkastream/example/Test16.scala | 5 +- .../gearpump/akkastream/example/Test3.scala | 2 +- .../gearpump/akkastream/example/Test4.scala | 2 +- .../gearpump/akkastream/example/Test6.scala | 2 +- .../akkastream/graph/RemoteGraph.scala | 2 +- .../materializer/LocalMaterializerImpl.scala | 61 +++++++++++-------- .../materializer/RemoteMaterializerImpl.scala | 32 +++++----- project/BuildDashboard.scala | 6 +- project/BuildExperiments.scala | 2 +- project/Dependencies.scala | 8 +-- .../streaming/StreamApplication.scala | 2 +- .../gearpump/streaming/dsl/plan/OP.scala | 5 +- 13 files changed, 97 insertions(+), 81 deletions(-) diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearpumpMaterializer.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearpumpMaterializer.scala index 9ff701ccf..c41f9c28f 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearpumpMaterializer.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearpumpMaterializer.scala @@ -24,11 +24,11 @@ import akka.NotUsed import akka.actor.{ActorContext, ActorRef, ActorRefFactory, ActorSystem, Cancellable, ExtendedActorSystem} import akka.event.{Logging, LoggingAdapter} import akka.stream.Attributes.Attribute -import akka.stream._ import akka.stream.impl.Stages.SymbolicGraphStage -import akka.stream.impl.StreamLayout._ -import akka.stream.impl._ +import akka.stream.impl.StreamLayout.{Atomic, Combine, CopiedModule, Ignore, MaterializedValueNode, Module, Transform} +import akka.stream.{ActorAttributes, ActorMaterializerSettings, Attributes, ClosedShape, Fusing, Graph, InPort, OutPort, SinkShape} import akka.stream.impl.fusing.{GraphInterpreterShell, GraphStageModule} +import akka.stream.impl.{ExtendedActorMaterializer, StreamSupervisor} import akka.stream.stage.GraphStage import org.apache.gearpump.akkastream.GearpumpMaterializer.Edge import org.apache.gearpump.akkastream.graph.GraphPartitioner.Strategy @@ -43,16 +43,13 @@ import scala.concurrent.duration.FiniteDuration object GearpumpMaterializer { - final val Debug = true - final case class Edge(from: OutPort, to: InPort) final case class MaterializedValueSourceAttribute(mat: MaterializedValueNode) extends Attribute implicit def boolToAtomic(bool: Boolean): AtomicBoolean = new AtomicBoolean(bool) - def apply(strategy: Strategy)(implicit context: ActorRefFactory): - ExtendedActorMaterializer = { + def apply(strategy: Strategy)(implicit context: ActorRefFactory): ExtendedActorMaterializer = { val system = actorSystemOf(context) apply(ActorMaterializerSettings( @@ -166,19 +163,10 @@ class GearpumpMaterializer(override val system: ActorSystem, system.scheduler.scheduleOnce(delay, task)(executionContext) override def materialize[Mat](runnableGraph: Graph[ClosedShape, Mat]): Mat = { - val initialAttributes = Attributes( - Attributes.InputBuffer( - settings.initialInputBufferSize, - settings.maxInputBufferSize - ) :: - ActorAttributes.Dispatcher(settings.dispatcher) :: - ActorAttributes.SupervisionStrategy(settings.supervisionDecider) :: - Nil) - val info = Fusing.aggressive(runnableGraph).module.info val graph = GGraph.empty[Module, Edge] - info.allModules.foreach(module => { + info.subModules.foreach(module => { if (module.isCopied) { val original = module.asInstanceOf[CopiedModule].copyOf graph.addVertex(original) @@ -201,9 +189,7 @@ class GearpumpMaterializer(override val system: ActorSystem, } }) - if(Debug) { - printGraph(graph) - } + printGraph(graph) val subGraphs = GraphPartitioner(strategy).partition(graph) val matValues = subGraphs.foldLeft(mutable.Map.empty[Module, Any]) { (map, subGraph) => @@ -226,7 +212,7 @@ class GearpumpMaterializer(override val system: ActorSystem, } }).toList val matModule = subGraphs.last.graph.topologicalOrderIterator.toList.last - val mat2 = resolveMaterialized(matModule.materializedValueComputation, matValues) + resolveMaterialized(matModule.materializedValueComputation, matValues) val rt = Some(mat).flatMap(any => { any match { case promise: Promise[_] => @@ -235,7 +221,7 @@ class GearpumpMaterializer(override val system: ActorSystem, Some(other) } }) - rt.getOrElse(null).asInstanceOf[Mat] + rt.orNull.asInstanceOf[Mat] } private def printGraph(graph: GGraph[Module, Edge]): Unit = { @@ -269,10 +255,24 @@ class GearpumpMaterializer(override val system: ActorSystem, } override def materialize[Mat](runnableGraph: Graph[ClosedShape, Mat], - subflowFuser: GraphInterpreterShell => ActorRef): Mat = { + initialAttributes: Attributes): Mat = { + materialize(runnableGraph) + } + + override def materialize[Mat](runnableGraph: Graph[ClosedShape, Mat], + subflowFuser: (GraphInterpreterShell) => ActorRef): Mat = { + materialize(runnableGraph) + } + + override def materialize[Mat](runnableGraph: Graph[ClosedShape, Mat], + subflowFuser: (GraphInterpreterShell) => ActorRef, initialAttributes: Attributes): Mat = { materialize(runnableGraph) } + override def makeLogger(logSource: Class[_]): LoggingAdapter = { + logger + } + def shutdown: Unit = { subMaterializers.values.foreach(_.shutdown) } @@ -288,5 +288,8 @@ class GearpumpMaterializer(override val system: ActorSystem, case Ignore => () } + + + } diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test16.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test16.scala index eb0b5c7c0..c4c8d074d 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test16.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test16.scala @@ -18,12 +18,11 @@ package org.apache.gearpump.akkastream.example -import akka.actor.{Actor, ActorSystem, Props} +import akka.actor.ActorSystem import org.apache.gearpump.akkastream.GearpumpMaterializer import org.apache.gearpump.akkastream.scaladsl.{GearSink, GearSource} -import akka.stream.scaladsl.Sink import org.apache.gearpump.cluster.main.ArgumentsParser -import org.apache.gearpump.streaming.dsl.{CollectionDataSource, LoggerSink} +import org.apache.gearpump.streaming.dsl.scalaapi.{CollectionDataSource, LoggerSink} import org.apache.gearpump.util.AkkaApp import scala.concurrent.Await diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test3.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test3.scala index 0a51078c8..a75bcb298 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test3.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test3.scala @@ -24,7 +24,7 @@ import org.apache.gearpump.akkastream.GearpumpMaterializer import org.apache.gearpump.akkastream.scaladsl.GearSource import akka.stream.scaladsl.Sink import org.apache.gearpump.cluster.main.{ArgumentsParser, CLIOption} -import org.apache.gearpump.streaming.dsl.CollectionDataSource +import org.apache.gearpump.streaming.dsl.scalaapi.CollectionDataSource import org.apache.gearpump.util.AkkaApp import scala.concurrent.Await diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test4.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test4.scala index 3cb69ced2..7eb1cbfb2 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test4.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test4.scala @@ -23,7 +23,7 @@ import akka.stream.scaladsl.Source import org.apache.gearpump.akkastream.GearpumpMaterializer import org.apache.gearpump.akkastream.scaladsl.GearSink import org.apache.gearpump.cluster.main.ArgumentsParser -import org.apache.gearpump.streaming.dsl.LoggerSink +import org.apache.gearpump.streaming.dsl.scalaapi.LoggerSink import org.apache.gearpump.util.AkkaApp import scala.concurrent.Await diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test6.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test6.scala index 6f5493358..0158482f1 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test6.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test6.scala @@ -24,7 +24,7 @@ import akka.stream.{ActorMaterializer, ActorMaterializerSettings} import org.apache.gearpump.akkastream.GearpumpMaterializer import org.apache.gearpump.akkastream.scaladsl.GearSource import org.apache.gearpump.cluster.main.{ArgumentsParser, CLIOption} -import org.apache.gearpump.streaming.dsl.CollectionDataSource +import org.apache.gearpump.streaming.dsl.scalaapi.CollectionDataSource import org.apache.gearpump.util.AkkaApp import scala.concurrent.Await diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/RemoteGraph.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/RemoteGraph.scala index 8fbe78531..b5a21f474 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/RemoteGraph.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/RemoteGraph.scala @@ -79,7 +79,7 @@ object RemoteGraph { val materializer = new RemoteMaterializerImpl(graph, system) val (app, matValues) = materializer.materialize - val appId = context.submit(app) + val appId = context.submit(app).appId // scalastyle:off println println("sleep 5 second until the application is ready on cluster") // scalastyle:on println diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/materializer/LocalMaterializerImpl.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/materializer/LocalMaterializerImpl.scala index cbafcf56b..d14bee409 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/materializer/LocalMaterializerImpl.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/materializer/LocalMaterializerImpl.scala @@ -21,18 +21,16 @@ package org.apache.gearpump.akkastream.materializer import java.util.concurrent.atomic.AtomicBoolean import java.{util => ju} -import _root_.org.apache.gearpump.util.{Graph => GGraph} -import akka.NotUsed +import org.apache.gearpump.util.{Graph => GGraph} import akka.actor.{ActorRef, ActorSystem, Cancellable, Deploy, PoisonPill} import akka.dispatch.Dispatchers import akka.event.{Logging, LoggingAdapter} import akka.stream.impl.StreamLayout._ import akka.stream.impl._ import akka.stream.impl.fusing.GraphInterpreter.GraphAssembly +import akka.stream.impl.fusing.{ActorGraphInterpreter, Fold, GraphInterpreterShell, GraphModule, GraphStageModule} import akka.stream.impl.fusing.GraphStages.MaterializedValueSource -import akka.stream.impl.fusing.{Map => _, _} -import akka.stream.impl.io.{TLSActor, TlsModule} -import akka.stream.scaladsl.{GraphDSL, Keep, ModuleExtractor, RunnableGraph} +import akka.stream.scaladsl.ModuleExtractor import akka.stream.{ClosedShape, Graph => AkkaGraph, _} import org.apache.gearpump.akkastream.GearpumpMaterializer.Edge import org.apache.gearpump.akkastream.module.ReduceModule @@ -121,25 +119,26 @@ case class LocalMaterializerImpl ( assignPort(stage.inPort, processor) assignPort(stage.outPort, processor.asInstanceOf[Publisher[Any]]) matVal.put(atomic, mat) - case tls: TlsModule => // TODO solve this so TlsModule doesn't need special treatment here - val es = effectiveSettings(effectiveAttributes) - val props = - TLSActor.props(es, tls.sslContext, tls.sslConfig, - tls.firstSession, tls.role, tls.closing, tls.hostInfo) - val impl = actorOf(props, stageName(effectiveAttributes), es.dispatcher) - def factory(id: Int) = new ActorPublisher[Any](impl) { - override val wakeUpMsg = FanOut.SubstreamSubscribePending(id) - } - val publishers = Vector.tabulate(2)(factory) - impl ! FanOut.ExposedPublishers(publishers) - - assignPort(tls.plainOut, publishers(TLSActor.UserOut)) - assignPort(tls.cipherOut, publishers(TLSActor.TransportOut)) - - assignPort(tls.plainIn, FanIn.SubInput[Any](impl, TLSActor.UserIn)) - assignPort(tls.cipherIn, FanIn.SubInput[Any](impl, TLSActor.TransportIn)) - - matVal.put(atomic, NotUsed) + // FIXME +// case tls: TlsModule => // TODO solve this so TlsModule doesn't need special treatment here +// val es = effectiveSettings(effectiveAttributes) +// val props = +// TLSActor.props(es, tls.sslContext, tls.sslConfig, +// tls.firstSession, tls.role, tls.closing, tls.hostInfo) +// val impl = actorOf(props, stageName(effectiveAttributes), es.dispatcher) +// def factory(id: Int) = new ActorPublisher[Any](impl) { +// override val wakeUpMsg = FanOut.SubstreamSubscribePending(id) +// } +// val publishers = Vector.tabulate(2)(factory) +// impl ! FanOut.ExposedPublishers(publishers) +// +// assignPort(tls.plainOut, publishers(TLSActor.UserOut)) +// assignPort(tls.cipherOut, publishers(TLSActor.TransportOut)) +// +// assignPort(tls.plainIn, FanIn.SubInput[Any](impl, TLSActor.UserIn)) +// assignPort(tls.cipherIn, FanIn.SubInput[Any](impl, TLSActor.TransportIn)) +// +// matVal.put(atomic, NotUsed) case graph: GraphModule => matGraph(graph, effectiveAttributes, matVal) case stage: GraphStageModule => @@ -186,6 +185,11 @@ case class LocalMaterializerImpl ( } + override def materialize[Mat](runnableGraph: AkkaGraph[ClosedShape, Mat], + initialAttributes: Attributes): Mat = { + materialize(runnableGraph) + } + override def materialize[Mat](runnableGraph: AkkaGraph[ClosedShape, Mat], subflowFuser: GraphInterpreterShell => ActorRef): Mat = { @@ -194,6 +198,15 @@ case class LocalMaterializerImpl ( } + override def materialize[Mat](runnableGraph: AkkaGraph[ClosedShape, Mat], + subflowFuser: (GraphInterpreterShell) => ActorRef, initialAttributes: Attributes): Mat = { + materialize(runnableGraph) + } + + override def makeLogger(logSource: Class[_]): LoggingAdapter = { + logger + } + def buildToplevelModule(graph: GGraph[Module, Edge]): Module = { var moduleInProgress: Module = EmptyModule graph.vertices.foreach(module => { diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/materializer/RemoteMaterializerImpl.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/materializer/RemoteMaterializerImpl.scala index 936ac2918..33e87e6ca 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/materializer/RemoteMaterializerImpl.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/materializer/RemoteMaterializerImpl.scala @@ -20,22 +20,24 @@ package org.apache.gearpump.akkastream.materializer import akka.actor.ActorSystem import akka.stream.impl.StreamLayout.Module -import akka.stream.impl.Timers._ +import akka.stream.impl.Timers.{Completion, DelayInitial, Idle, IdleInject, IdleTimeoutBidi, Initial} +import akka.stream.impl.fusing.{Batch, Collect, Delay, Drop, DropWhile, DropWithin, Filter, FlattenMerge, Fold, GraphStageModule, GroupBy, GroupedWithin, Intersperse, LimitWeighted, Log, MapAsync, MapAsyncUnordered, PrefixAndTail, Recover, Reduce, Scan, Split, StatefulMapConcat, SubSink, SubSource, Take, TakeWhile, TakeWithin, Map => FMap} import akka.stream.impl.fusing.GraphStages.{MaterializedValueSource, SimpleLinearGraphStage, SingleSource, TickSource} -import akka.stream.impl.fusing.{Map => FMap, _} import akka.stream.impl.io.IncomingConnectionStage -import akka.stream.impl.{HeadOptionStage, Stages, Throttle} -import akka.stream.scaladsl._ +import akka.stream.impl.{HeadOptionStage, Stages, Throttle, Unfold, UnfoldAsync} +import akka.stream.scaladsl.{Balance, Broadcast, Concat, Interleave, Merge, MergePreferred, MergeSorted, ModuleExtractor, Unzip, Zip, ZipWith2} import akka.stream.stage.AbstractStage.PushPullGraphStageWithMaterializedValue import akka.stream.stage.GraphStage import org.apache.gearpump.akkastream.GearAttributes import org.apache.gearpump.akkastream.GearpumpMaterializer.Edge -import org.apache.gearpump.akkastream.module._ -import org.apache.gearpump.akkastream.task._ +import org.apache.gearpump.akkastream.module.{GroupByModule, ProcessorModule, ReduceModule, SinkBridgeModule, SinkTaskModule, SourceBridgeModule, SourceTaskModule} +import org.apache.gearpump.akkastream.task.{BalanceTask, BatchTask, BroadcastTask, ConcatTask, DelayInitialTask, DropWithinTask, FlattenMergeTask, FoldTask, GraphTask, GroupedWithinTask, InterleaveTask, MapAsyncTask, MergeTask, SingleSourceTask, SinkBridgeTask, SourceBridgeTask, StatefulMapConcatTask, TakeWithinTask, ThrottleTask, TickSourceTask, Zip2Task} +import org.apache.gearpump.akkastream.task.TickSourceTask.{INITIAL_DELAY, INTERVAL, TICK} import org.apache.gearpump.cluster.UserConfig -import org.apache.gearpump.streaming.dsl.StreamApp -import org.apache.gearpump.streaming.dsl.plan._ -import org.apache.gearpump.streaming.dsl.plan.functions.FlatMapFunction +import org.apache.gearpump.streaming.dsl.plan.functions.FlatMapper +import org.apache.gearpump.streaming.dsl.plan.{ChainableOp, DataSinkOp, DataSourceOp, Direct, GroupByOp, MergeOp, Op, OpEdge, ProcessorOp, Shuffle} +import org.apache.gearpump.streaming.dsl.scalaapi.StreamApp +import org.apache.gearpump.streaming.dsl.scalaapi.functions.FlatMapFunction import org.apache.gearpump.streaming.dsl.window.api.CountWindow import org.apache.gearpump.streaming.dsl.window.impl.GroupAlsoByWindow import org.apache.gearpump.streaming.{ProcessorId, StreamApplication} @@ -162,7 +164,8 @@ class RemoteMaterializerImpl(graph: Graph[Module, Edge], system: ActorSystem) { case sinkBridge: SinkBridgeModule[_, _] => ProcessorOp(classOf[SinkBridgeTask], parallelism, conf, "sink") case groupBy: GroupByModule[_, _] => - GroupByOp(groupBy.groupBy, parallelism, "groupBy", conf) + GroupByOp(GroupAlsoByWindow(groupBy.groupBy, CountWindow.apply(1).accumulating), + parallelism, "groupBy", conf) case reduce: ReduceModule[_] => reduceOp(reduce.f, conf) case graphStage: GraphStageModule => @@ -192,7 +195,6 @@ class RemoteMaterializerImpl(graph: Graph[Module, Edge], system: ActorSystem) { parallelism: Int, conf: UserConfig): Op = { module.stage match { case tickSource: TickSource[_] => - import TickSourceTask._ val tick: AnyRef = tickSource.tick.asInstanceOf[AnyRef] val tiConf = conf.withValue[FiniteDuration](INITIAL_DELAY, tickSource.initialDelay). withValue[FiniteDuration](INTERVAL, tickSource.interval). @@ -388,9 +390,9 @@ class RemoteMaterializerImpl(graph: Graph[Module, Edge], system: ActorSystem) { private def translateSymbolic(stage: PushPullGraphStageWithMaterializedValue[_, _, _, _], conf: UserConfig): Op = { stage match { - case symbolicGraphStage: Stages.SymbolicGraphStage[_, _, _] => - symbolicGraphStage.symbolicStage match { - case buffer: Stages.Buffer[_] => + case symbolicGraphStage: Stages.SymbolicGraphStage[_, _, _] + if symbolicGraphStage.symbolicStage.attributes.equals( + Stages.DefaultAttributes.buffer) => { // ignore the buffering operation identity("buffer", conf) } @@ -478,7 +480,7 @@ object RemoteMaterializerImpl { def flatMapOp[In, Out](fun: In => TraversableOnce[Out], description: String, conf: UserConfig): Op = { - ChainableOp(new FlatMapFunction[In, Out](fun, description), conf) + ChainableOp(new FlatMapper(FlatMapFunction[In, Out](fun), description), conf) } def conflateOp[In, Out](seed: In => Out, aggregate: (Out, In) => Out, diff --git a/project/BuildDashboard.scala b/project/BuildDashboard.scala index c14b9d6ba..cfa6aae9f 100644 --- a/project/BuildDashboard.scala +++ b/project/BuildDashboard.scala @@ -46,11 +46,11 @@ object BuildDashboard extends sbt.Build { private lazy val serviceJvmSettings = commonSettings ++ noPublish ++ Seq( libraryDependencies ++= Seq( - "com.typesafe.akka" %% "akka-http-testkit" % akkaVersion % "test", + "com.typesafe.akka" %% "akka-http-testkit" % akkaHttpVersion % "test", "org.scalatest" %% "scalatest" % scalaTestVersion % "test", "com.lihaoyi" %% "upickle" % upickleVersion, - "com.softwaremill.akka-http-session" %% "core" % "0.2.5", - "com.typesafe.akka" %% "akka-http-spray-json-experimental" % akkaVersion, + "com.softwaremill.akka-http-session" %% "core" % "0.3.0", + "com.typesafe.akka" %% "akka-http-spray-json" % akkaHttpVersion, "com.github.scribejava" % "scribejava-apis" % "2.4.0", "com.ning" % "async-http-client" % "1.9.33", "org.webjars" % "angularjs" % "1.4.9", diff --git a/project/BuildExperiments.scala b/project/BuildExperiments.scala index e07b6889a..eb5f9e196 100644 --- a/project/BuildExperiments.scala +++ b/project/BuildExperiments.scala @@ -25,7 +25,7 @@ import sbt.Keys._ object BuildExperiments extends sbt.Build { lazy val experiments: Seq[ProjectReference] = Seq( - // akkastream, + akkastream, cgroup, redis, storm, diff --git a/project/Dependencies.scala b/project/Dependencies.scala index 69494972c..4e30d3ffa 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -23,7 +23,8 @@ object Dependencies { val crossScalaVersionNumbers = Seq("2.11.8") val scalaVersionNumber = crossScalaVersionNumbers.last - val akkaVersion = "2.4.3" + val akkaVersion = "2.4.16" + val akkaHttpVersion = "10.0.1" val hadoopVersion = "2.6.0" val hbaseVersion = "1.0.0" val commonsHttpVersion = "3.1" @@ -82,10 +83,9 @@ object Dependencies { "com.typesafe.akka" %% "akka-agent" % akkaVersion, "com.typesafe.akka" %% "akka-slf4j" % akkaVersion, "com.typesafe.akka" %% "akka-kernel" % akkaVersion, - "com.typesafe.akka" %% "akka-http-experimental" % akkaVersion, - "com.typesafe.akka" %% "akka-http-spray-json-experimental" % akkaVersion, + "com.typesafe.akka" %% "akka-http" % akkaHttpVersion, + "com.typesafe.akka" %% "akka-http-spray-json" % akkaHttpVersion, "org.scala-lang" % "scala-reflect" % scalaVersionNumber, - "org.scala-lang.modules" %% "scala-parser-combinators" % "1.0.4", "com.github.romix.akka" %% "akka-kryo-serialization" % kryoVersion, "com.google.guava" % "guava" % guavaVersion, "com.codahale.metrics" % "metrics-graphite" % codahaleVersion diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/StreamApplication.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/StreamApplication.scala index ca8d89eb3..d4b3719e6 100644 --- a/streaming/src/main/scala/org/apache/gearpump/streaming/StreamApplication.scala +++ b/streaming/src/main/scala/org/apache/gearpump/streaming/StreamApplication.scala @@ -123,7 +123,7 @@ object LifeTime { */ class StreamApplication( override val name: String, val inputUserConfig: UserConfig, - dag: Graph[ProcessorDescription, PartitionerDescription]) + val dag: Graph[ProcessorDescription, PartitionerDescription]) extends Application { require(!dag.hasDuplicatedEdge(), "Graph should not have duplicated edges") diff --git a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/OP.scala b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/OP.scala index 82ea7c7ad..5aaf2facd 100644 --- a/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/OP.scala +++ b/streaming/src/main/scala/org/apache/gearpump/streaming/dsl/plan/OP.scala @@ -124,12 +124,11 @@ case class DataSinkOp( * to another Op to be used */ case class ChainableOp[IN, OUT]( - fn: SingleInputFunction[IN, OUT]) extends Op { + fn: SingleInputFunction[IN, OUT], + userConfig: UserConfig = UserConfig.empty) extends Op { override def description: String = fn.description - override def userConfig: UserConfig = UserConfig.empty - override def chain(other: Op)(implicit system: ActorSystem): Op = { other match { case op: ChainableOp[OUT, _] => From bd2d5bb44cb762f7c93737bed9860e1eb33ccf81 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Fri, 20 Jan 2017 15:26:08 +0800 Subject: [PATCH 5/9] Add scalastyle-config.xml --- scalastyle-config.xml | 240 ++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 240 insertions(+) create mode 100644 scalastyle-config.xml diff --git a/scalastyle-config.xml b/scalastyle-config.xml new file mode 100644 index 000000000..1b0a8385b --- /dev/null +++ b/scalastyle-config.xml @@ -0,0 +1,240 @@ + + + + + Scalastyle standard configuration + + + + + + + + + + + + + + + + + + true + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ARROW, EQUALS, ELSE, TRY, CATCH, FINALLY, LARROW, RARROW + + + + + + ARROW, EQUALS, COMMA, COLON, IF, ELSE, DO, WHILE, FOR, MATCH, TRY, + CATCH, FINALLY, LARROW, RARROW + + + + + + + + + + ^println$ + + + + + + + Runtime\.getRuntime\.addShutdownHook + + + + + + + mutable\.SynchronizedBuffer + + + + + + + JavaConversions + + Instead of importing implicits in scala.collection.JavaConversions._, import + scala.collection.JavaConverters._ and use .asScala / .asJava methods + + + + + + COMMA + + + + + + + \)\{ + + + + + + + + + + + + + + + 2 + 4 + + + + + + + \ No newline at end of file From 5539150e187bb9271a2a7337857cfdedf0c3b827 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Fri, 20 Jan 2017 16:27:38 +0800 Subject: [PATCH 6/9] Fix scalastyle --- .../gearpump/akkastream/GearAttributes.scala | 2 +- .../akkastream/GearpumpMaterializer.scala | 4 +- .../GearpumpMaterializerSession.scala | 2 +- .../gearpump/akkastream/example/Test.scala | 2 +- .../gearpump/akkastream/example/Test10.scala | 2 +- .../gearpump/akkastream/example/Test11.scala | 6 +- .../gearpump/akkastream/example/Test12.scala | 8 +- .../gearpump/akkastream/example/Test13.scala | 2 +- .../gearpump/akkastream/example/Test14.scala | 2 +- .../gearpump/akkastream/example/Test15.scala | 2 +- .../gearpump/akkastream/example/Test16.scala | 2 +- .../gearpump/akkastream/example/Test2.scala | 2 +- .../gearpump/akkastream/example/Test3.scala | 2 +- .../gearpump/akkastream/example/Test4.scala | 2 +- .../gearpump/akkastream/example/Test5.scala | 2 +- .../gearpump/akkastream/example/Test6.scala | 2 +- .../gearpump/akkastream/example/Test7.scala | 2 +- .../gearpump/akkastream/example/Test8.scala | 2 +- .../gearpump/akkastream/example/Test9.scala | 2 +- .../akkastream/example/WikipediaApp.scala | 2 +- .../akkastream/graph/GraphPartitioner.scala | 2 +- .../akkastream/graph/LocalGraph.scala | 2 +- .../akkastream/graph/RemoteGraph.scala | 2 +- .../gearpump/akkastream/graph/SubGraph.scala | 2 +- .../materializer/LocalMaterializerImpl.scala | 43 ++-- .../materializer/RemoteMaterializerImpl.scala | 8 +- .../akkastream/module/BridgeModule.scala | 2 +- .../akkastream/module/DummyModule.scala | 2 +- .../module/GearpumpTaskModule.scala | 2 +- .../akkastream/module/GroupByModule.scala | 2 +- .../akkastream/module/ReduceModule.scala | 2 +- .../gearpump/akkastream/scaladsl/Api.scala | 2 +- .../akkastream/task/BalanceTask.scala | 2 +- .../gearpump/akkastream/task/BatchTask.scala | 2 +- .../akkastream/task/BroadcastTask.scala | 2 +- .../gearpump/akkastream/task/ConcatTask.scala | 2 +- .../akkastream/task/DelayInitialTask.scala | 2 +- .../akkastream/task/DropWithinTask.scala | 2 +- .../akkastream/task/FlattenMergeTask.scala | 2 +- .../gearpump/akkastream/task/FoldTask.scala | 2 +- .../gearpump/akkastream/task/GraphTask.scala | 2 +- .../akkastream/task/GroupedWithinTask.scala | 2 +- .../akkastream/task/InterleaveTask.scala | 2 +- .../akkastream/task/MapAsyncTask.scala | 2 +- .../gearpump/akkastream/task/MergeTask.scala | 4 +- .../akkastream/task/SingleSourceTask.scala | 2 +- .../akkastream/task/SinkBridgeTask.scala | 2 +- .../akkastream/task/SourceBridgeTask.scala | 2 +- .../task/StatefulMapConcatTask.scala | 2 +- .../akkastream/task/TakeWithinTask.scala | 2 +- .../akkastream/task/ThrottleTask.scala | 2 +- .../akkastream/task/TickSourceTask.scala | 2 +- .../gearpump/akkastream/task/Unzip2Task.scala | 2 +- .../gearpump/akkastream/task/Zip2Task.scala | 2 +- .../util/MaterializedValueOps.scala | 2 +- project/scalastyle_config.xml | 240 ------------------ .../gearpump/services/SecurityService.scala | 2 +- 57 files changed, 87 insertions(+), 326 deletions(-) delete mode 100644 project/scalastyle_config.xml diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearAttributes.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearAttributes.scala index 016a7b279..4384b3902 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearAttributes.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearAttributes.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearpumpMaterializer.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearpumpMaterializer.scala index c41f9c28f..07c95f83a 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearpumpMaterializer.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearpumpMaterializer.scala @@ -7,7 +7,7 @@ * "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 + * 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, @@ -92,7 +92,7 @@ object GearpumpMaterializer { case _ => throw new IllegalArgumentException( s""" - | context must be a ActorSystem or ActorContext, got [${context.getClass.getName}] + | context must be a ActorSystem or ActorContext, got [${context.getClass.getName}] """.stripMargin ) } diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearpumpMaterializerSession.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearpumpMaterializerSession.scala index 871dcf817..8a869d224 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearpumpMaterializerSession.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/GearpumpMaterializerSession.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test.scala index 40cd556e1..52a45d93f 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test10.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test10.scala index 71678c3a7..826cdcffd 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test10.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test10.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test11.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test11.scala index b80398c0c..087c57dcf 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test11.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test11.scala @@ -7,7 +7,7 @@ * "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 + * 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, @@ -38,8 +38,8 @@ object Test11 extends AkkaApp with ArgumentsParser { implicit val system = ActorSystem("Test11", akkaConfig) implicit val materializer = GearpumpMaterializer() -// implicit val materializer = -// ActorMaterializer(ActorMaterializerSettings(system).withAutoFusing(false)) + // implicit val materializer = + // ActorMaterializer(ActorMaterializerSettings(system).withAutoFusing(false)) implicit val ec = system.dispatcher val g = RunnableGraph.fromGraph(GraphDSL.create() { diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test12.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test12.scala index a9e8b0817..b4f4bce4c 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test12.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test12.scala @@ -7,7 +7,7 @@ * "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 + * 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, @@ -37,9 +37,9 @@ object Test12 extends AkkaApp with ArgumentsParser{ import scala.concurrent.duration._ implicit val system = ActorSystem("Test12", akkaConfig) -// implicit val materializer = ActorMaterializer( -// ActorMaterializerSettings(system).withAutoFusing(false) -// ) + // implicit val materializer = ActorMaterializer( + // ActorMaterializerSettings(system).withAutoFusing(false) + // ) implicit val materializer = GearpumpMaterializer() implicit val ec = system.dispatcher diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test13.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test13.scala index 984c861a6..2e036cbb7 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test13.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test13.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test14.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test14.scala index 0542f43f5..c436130be 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test14.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test14.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test15.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test15.scala index c2f8d5f6c..f4e4dbd11 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test15.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test15.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test16.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test16.scala index c4c8d074d..969149622 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test16.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test16.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test2.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test2.scala index 21f1b8c4a..a6049cd6d 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test2.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test2.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test3.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test3.scala index a75bcb298..24faeb325 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test3.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test3.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test4.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test4.scala index 7eb1cbfb2..6a44a3580 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test4.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test4.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test5.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test5.scala index 72e21c77f..ad87a971e 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test5.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test5.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test6.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test6.scala index 0158482f1..a525471f3 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test6.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test6.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test7.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test7.scala index be916103e..8c837afab 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test7.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test7.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test8.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test8.scala index 434aa3353..ad2ac6170 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test8.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test8.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test9.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test9.scala index 63f9e2dd0..66414e049 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test9.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/Test9.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/WikipediaApp.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/WikipediaApp.scala index 830f27810..2a1e7ff8f 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/WikipediaApp.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/example/WikipediaApp.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/GraphPartitioner.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/GraphPartitioner.scala index c1e95bb8e..f7919c022 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/GraphPartitioner.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/GraphPartitioner.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/LocalGraph.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/LocalGraph.scala index c03fce284..fe8695119 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/LocalGraph.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/LocalGraph.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/RemoteGraph.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/RemoteGraph.scala index b5a21f474..99ebe177f 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/RemoteGraph.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/RemoteGraph.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/SubGraph.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/SubGraph.scala index a0395de7a..a74143e4d 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/SubGraph.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/graph/SubGraph.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/materializer/LocalMaterializerImpl.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/materializer/LocalMaterializerImpl.scala index d14bee409..477f4d30e 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/materializer/LocalMaterializerImpl.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/materializer/LocalMaterializerImpl.scala @@ -7,7 +7,7 @@ * "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 + * 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, @@ -119,26 +119,27 @@ case class LocalMaterializerImpl ( assignPort(stage.inPort, processor) assignPort(stage.outPort, processor.asInstanceOf[Publisher[Any]]) matVal.put(atomic, mat) - // FIXME -// case tls: TlsModule => // TODO solve this so TlsModule doesn't need special treatment here -// val es = effectiveSettings(effectiveAttributes) -// val props = -// TLSActor.props(es, tls.sslContext, tls.sslConfig, -// tls.firstSession, tls.role, tls.closing, tls.hostInfo) -// val impl = actorOf(props, stageName(effectiveAttributes), es.dispatcher) -// def factory(id: Int) = new ActorPublisher[Any](impl) { -// override val wakeUpMsg = FanOut.SubstreamSubscribePending(id) -// } -// val publishers = Vector.tabulate(2)(factory) -// impl ! FanOut.ExposedPublishers(publishers) -// -// assignPort(tls.plainOut, publishers(TLSActor.UserOut)) -// assignPort(tls.cipherOut, publishers(TLSActor.TransportOut)) -// -// assignPort(tls.plainIn, FanIn.SubInput[Any](impl, TLSActor.UserIn)) -// assignPort(tls.cipherIn, FanIn.SubInput[Any](impl, TLSActor.TransportIn)) -// -// matVal.put(atomic, NotUsed) + // FIXME + // case tls: TlsModule => + // TODO solve this so TlsModule doesn't need special treatment here + // val es = effectiveSettings(effectiveAttributes) + // val props = + // TLSActor.props(es, tls.sslContext, tls.sslConfig, + // tls.firstSession, tls.role, tls.closing, tls.hostInfo) + // val impl = actorOf(props, stageName(effectiveAttributes), es.dispatcher) + // def factory(id: Int) = new ActorPublisher[Any](impl) { + // override val wakeUpMsg = FanOut.SubstreamSubscribePending(id) + // } + // val publishers = Vector.tabulate(2)(factory) + // impl ! FanOut.ExposedPublishers(publishers) + // + // assignPort(tls.plainOut, publishers(TLSActor.UserOut)) + // assignPort(tls.cipherOut, publishers(TLSActor.TransportOut)) + // + // assignPort(tls.plainIn, FanIn.SubInput[Any](impl, TLSActor.UserIn)) + // assignPort(tls.cipherIn, FanIn.SubInput[Any](impl, TLSActor.TransportIn)) + // + // matVal.put(atomic, NotUsed) case graph: GraphModule => matGraph(graph, effectiveAttributes, matVal) case stage: GraphStageModule => diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/materializer/RemoteMaterializerImpl.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/materializer/RemoteMaterializerImpl.scala index 33e87e6ca..e065c9019 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/materializer/RemoteMaterializerImpl.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/materializer/RemoteMaterializerImpl.scala @@ -7,7 +7,7 @@ * "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 + * 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, @@ -324,9 +324,9 @@ class RemoteMaterializerImpl(graph: Graph[Module, Edge], system: ActorSystem) { // TODO null case unzip: Unzip[_, _] => -// ProcessorOp(classOf[Unzip2Task[_, _, _]], parallelism, -// conf.withValue( -// Unzip2Task.UNZIP2_FUNCTION, Unzip2Task.UnZipFunction(unzip.unzipper)), "unzip") + // ProcessorOp(classOf[Unzip2Task[_, _, _]], parallelism, + // conf.withValue( + // Unzip2Task.UNZIP2_FUNCTION, Unzip2Task.UnZipFunction(unzip.unzipper)), "unzip") // TODO null case zip: Zip[_, _] => diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/BridgeModule.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/BridgeModule.scala index 35d0e88dc..5b8c71b5a 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/BridgeModule.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/BridgeModule.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/DummyModule.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/DummyModule.scala index 2c430d5b8..ea76bb0df 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/DummyModule.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/DummyModule.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/GearpumpTaskModule.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/GearpumpTaskModule.scala index 755524429..dfbbee961 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/GearpumpTaskModule.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/GearpumpTaskModule.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/GroupByModule.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/GroupByModule.scala index 4465886db..b06dd0e13 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/GroupByModule.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/GroupByModule.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/ReduceModule.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/ReduceModule.scala index 295556f50..462d967f4 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/ReduceModule.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/module/ReduceModule.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/scaladsl/Api.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/scaladsl/Api.scala index 80619ef51..8e43c16de 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/scaladsl/Api.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/scaladsl/Api.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/BalanceTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/BalanceTask.scala index 513911711..43f07c498 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/BalanceTask.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/BalanceTask.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/BatchTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/BatchTask.scala index 582327b60..5c2485ba3 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/BatchTask.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/BatchTask.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/BroadcastTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/BroadcastTask.scala index 9f1194fca..292468de5 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/BroadcastTask.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/BroadcastTask.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/ConcatTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/ConcatTask.scala index 241fa7699..b77b9bd92 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/ConcatTask.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/ConcatTask.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/DelayInitialTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/DelayInitialTask.scala index d6c347aee..7c335dcc5 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/DelayInitialTask.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/DelayInitialTask.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/DropWithinTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/DropWithinTask.scala index 9da26b1aa..0c5482971 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/DropWithinTask.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/DropWithinTask.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/FlattenMergeTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/FlattenMergeTask.scala index 512164d31..14ff53781 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/FlattenMergeTask.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/FlattenMergeTask.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/FoldTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/FoldTask.scala index e2f02d89b..d982ebd88 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/FoldTask.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/FoldTask.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/GraphTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/GraphTask.scala index 8e7a2df44..3310ab926 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/GraphTask.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/GraphTask.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/GroupedWithinTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/GroupedWithinTask.scala index 29d9c91a7..eaf2b3fe8 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/GroupedWithinTask.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/GroupedWithinTask.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/InterleaveTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/InterleaveTask.scala index 837de6b49..741ec43ca 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/InterleaveTask.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/InterleaveTask.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/MapAsyncTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/MapAsyncTask.scala index 387116db7..daa1afc45 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/MapAsyncTask.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/MapAsyncTask.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/MergeTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/MergeTask.scala index 2b1cd3320..ad18f72c7 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/MergeTask.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/MergeTask.scala @@ -7,12 +7,12 @@ * "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 + * 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. - * SeG the License for the specific language governing permissions and + * See the License for the specific language governing permissions and * limitations under the License. */ diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/SingleSourceTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/SingleSourceTask.scala index 1ff9ccd27..458bb4e93 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/SingleSourceTask.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/SingleSourceTask.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/SinkBridgeTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/SinkBridgeTask.scala index 05011e983..1b9c4e380 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/SinkBridgeTask.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/SinkBridgeTask.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/SourceBridgeTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/SourceBridgeTask.scala index b0eda1909..054b4830a 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/SourceBridgeTask.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/SourceBridgeTask.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/StatefulMapConcatTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/StatefulMapConcatTask.scala index bf2c14ffe..a0674bc0a 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/StatefulMapConcatTask.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/StatefulMapConcatTask.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/TakeWithinTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/TakeWithinTask.scala index ef43fbe41..9559d8f54 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/TakeWithinTask.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/TakeWithinTask.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/ThrottleTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/ThrottleTask.scala index 4e09bf2a0..3c7ad8706 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/ThrottleTask.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/ThrottleTask.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/TickSourceTask.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/TickSourceTask.scala index b3850caa2..d99d2dbc4 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/TickSourceTask.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/TickSourceTask.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/Unzip2Task.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/Unzip2Task.scala index 7dd91fcd9..005d01886 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/Unzip2Task.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/Unzip2Task.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/Zip2Task.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/Zip2Task.scala index a35b1332a..7e0c08208 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/Zip2Task.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/task/Zip2Task.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/util/MaterializedValueOps.scala b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/util/MaterializedValueOps.scala index c9fe67de7..6ad90dfc9 100644 --- a/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/util/MaterializedValueOps.scala +++ b/experiments/akkastream/src/main/scala/org/apache/gearpump/akkastream/util/MaterializedValueOps.scala @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/project/scalastyle_config.xml b/project/scalastyle_config.xml deleted file mode 100644 index 1b0a8385b..000000000 --- a/project/scalastyle_config.xml +++ /dev/null @@ -1,240 +0,0 @@ - - - - - Scalastyle standard configuration - - - - - - - - - - - - - - - - - - true - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - ARROW, EQUALS, ELSE, TRY, CATCH, FINALLY, LARROW, RARROW - - - - - - ARROW, EQUALS, COMMA, COLON, IF, ELSE, DO, WHILE, FOR, MATCH, TRY, - CATCH, FINALLY, LARROW, RARROW - - - - - - - - - - ^println$ - - - - - - - Runtime\.getRuntime\.addShutdownHook - - - - - - - mutable\.SynchronizedBuffer - - - - - - - JavaConversions - - Instead of importing implicits in scala.collection.JavaConversions._, import - scala.collection.JavaConverters._ and use .asScala / .asJava methods - - - - - - COMMA - - - - - - - \)\{ - - - - - - - - - - - - - - - 2 - 4 - - - - - - - \ No newline at end of file diff --git a/services/jvm/src/main/scala/org/apache/gearpump/services/SecurityService.scala b/services/jvm/src/main/scala/org/apache/gearpump/services/SecurityService.scala index 8ae8dbee0..498936418 100644 --- a/services/jvm/src/main/scala/org/apache/gearpump/services/SecurityService.scala +++ b/services/jvm/src/main/scala/org/apache/gearpump/services/SecurityService.scala @@ -61,7 +61,7 @@ class SecurityService(inner: RouteService, implicit val system: ActorSystem) ext // Use scheme "GearpumpBasic" to avoid popping up web browser native authentication box. private val challenge = HttpChallenge(scheme = "GearpumpBasic", realm = Some("gearpump"), - params = Map.empty[String,String]) + params = Map.empty[String, String]) val LOG = LogUtil.getLogger(getClass, "AUDIT") From a84d9560f5ba10b67ea672dc52786fde74c002b8 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Fri, 20 Jan 2017 19:15:24 +0800 Subject: [PATCH 7/9] Remove obsolete configuration --- experiments/akkastream/src/main/resources/geardefault.conf | 3 --- 1 file changed, 3 deletions(-) diff --git a/experiments/akkastream/src/main/resources/geardefault.conf b/experiments/akkastream/src/main/resources/geardefault.conf index 56524d455..858451111 100644 --- a/experiments/akkastream/src/main/resources/geardefault.conf +++ b/experiments/akkastream/src/main/resources/geardefault.conf @@ -3,6 +3,3 @@ gearpump.serializers { "scala.collection.immutable.Map$Map1" = "" "scala.collection.immutable.Map$Map2" = "" } -akka { - version = "2.4.11" -} From 37e97d9827032c089b8da45dd1f74f2c9ac134eb Mon Sep 17 00:00:00 2001 From: manuzhang Date: Fri, 20 Jan 2017 19:25:13 +0800 Subject: [PATCH 8/9] Revert unnecessary changes to avoid conflicts --- .../cluster/master/AppMasterLauncher.scala | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/gearpump/cluster/master/AppMasterLauncher.scala b/core/src/main/scala/org/apache/gearpump/cluster/master/AppMasterLauncher.scala index de104b9f1..9305d5c35 100644 --- a/core/src/main/scala/org/apache/gearpump/cluster/master/AppMasterLauncher.scala +++ b/core/src/main/scala/org/apache/gearpump/cluster/master/AppMasterLauncher.scala @@ -19,9 +19,16 @@ package org.apache.gearpump.cluster.master import java.util.concurrent.{TimeUnit, TimeoutException} +import org.apache.gearpump.cluster.worker.WorkerId + +import scala.collection.JavaConverters._ +import scala.concurrent.duration.Duration +import scala.util.{Failure, Success} import akka.actor.{Actor, ActorRef, Props, _} import com.typesafe.config.Config +import org.slf4j.Logger + import org.apache.gearpump.cluster.AppMasterToMaster.RequestResource import org.apache.gearpump.cluster.AppMasterToWorker.{LaunchExecutor, ShutdownExecutor} import org.apache.gearpump.cluster.MasterToAppMaster.ResourceAllocated @@ -29,17 +36,11 @@ import org.apache.gearpump.cluster.MasterToClient.SubmitApplicationResult import org.apache.gearpump.cluster.WorkerToAppMaster.ExecutorLaunchRejected import org.apache.gearpump.cluster.appmaster.{AppMasterRuntimeEnvironment, AppMasterRuntimeInfo, WorkerInfo} import org.apache.gearpump.cluster.scheduler.{Resource, ResourceAllocation, ResourceRequest} -import org.apache.gearpump.cluster.worker.WorkerId import org.apache.gearpump.cluster.{AppDescription, AppJar, _} import org.apache.gearpump.transport.HostPort import org.apache.gearpump.util.ActorSystemBooter._ import org.apache.gearpump.util.Constants._ import org.apache.gearpump.util.{ActorSystemBooter, ActorUtil, LogUtil, Util} -import org.slf4j.Logger - -import scala.collection.JavaConverters._ -import scala.concurrent.duration.Duration -import scala.util.{Failure, Success} /** * @@ -144,4 +145,4 @@ object AppMasterLauncher extends AppMasterLauncherFactory { trait AppMasterLauncherFactory { def props(appId: Int, executorId: Int, app: AppDescription, jar: Option[AppJar], username: String, master: ActorRef, client: Option[ActorRef]): Props -} +} \ No newline at end of file From b3d33741136615f3c8fc2b85fc3fd2431ffa2ff2 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Fri, 20 Jan 2017 19:45:28 +0800 Subject: [PATCH 9/9] Fix AttributesSpec license header --- .../scala/org/apache/gearpump/akkastream/AttributesSpec.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/experiments/akkastream/src/test/scala/org/apache/gearpump/akkastream/AttributesSpec.scala b/experiments/akkastream/src/test/scala/org/apache/gearpump/akkastream/AttributesSpec.scala index 3731d41a1..e1846eac6 100644 --- a/experiments/akkastream/src/test/scala/org/apache/gearpump/akkastream/AttributesSpec.scala +++ b/experiments/akkastream/src/test/scala/org/apache/gearpump/akkastream/AttributesSpec.scala @@ -7,7 +7,7 @@ * "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 + * 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,