├── .gitignore
├── LICENSE
├── NOTICE
├── README.md
├── core
├── README.md
├── build.sbt
└── src
│ ├── main
│ └── scala
│ │ └── com
│ │ └── monovore
│ │ ├── coast
│ │ ├── core
│ │ │ ├── Process.scala
│ │ │ └── elements.scala
│ │ ├── flow
│ │ │ ├── AnyGrouping.scala
│ │ │ ├── Context.scala
│ │ │ ├── Flow.scala
│ │ │ ├── PoolDef.scala
│ │ │ ├── StreamDef.scala
│ │ │ └── package.scala
│ │ ├── machine
│ │ │ ├── Machine.scala
│ │ │ └── System.scala
│ │ ├── package.scala
│ │ ├── viz
│ │ │ └── Dot.scala
│ │ └── wire
│ │ │ ├── Partitioner.scala
│ │ │ ├── Protocol.scala
│ │ │ ├── Serializer.scala
│ │ │ └── StreamFormat.scala
│ │ └── example
│ │ └── coast
│ │ ├── ConnectedComponents.scala
│ │ ├── CustomerTransactions.scala
│ │ ├── Denormalize.scala
│ │ ├── EntityResolution.scala
│ │ ├── ExampleMain.scala
│ │ ├── LinearRoad.scala
│ │ ├── Scheduler.scala
│ │ ├── TwitterReach.scala
│ │ └── WordCount.scala
│ └── test
│ └── scala
│ └── com
│ └── monovore
│ ├── coast
│ └── machine
│ │ ├── MachineSpec.scala
│ │ ├── Sample.scala
│ │ └── SystemSpec.scala
│ └── example
│ └── coast
│ ├── ConnectedComponentsSpec.scala
│ ├── DenormalizeSpec.scala
│ ├── EntityResolutionSpec.scala
│ └── LinearRoadSpec.scala
├── docs
├── _config.yaml
├── _layouts
│ └── default.html
├── flow.md
├── overview.md
├── samza.md
└── semantics.md
├── project
├── CoastBuild.scala
├── build.properties
└── plugins.sbt
└── samza
├── README.md
├── build.sbt
└── src
├── it
├── resources
│ └── logback-test.xml
└── scala
│ └── com
│ └── monovore
│ └── integration
│ └── coast
│ ├── CoastKafkaSystemSpec.scala
│ ├── IntegrationTest.scala
│ ├── SafeIntegrationSpec.scala
│ └── SimpleIntegrationSpec.scala
├── main
└── scala
│ └── com
│ └── monovore
│ └── coast
│ └── samza
│ ├── CoastSerdeFactory.scala
│ ├── CoastTask.scala
│ ├── ConfigGenerator.scala
│ ├── Path.scala
│ ├── SafeBackend.scala
│ ├── SamzaApp.scala
│ ├── SamzaBackend.scala
│ ├── SamzaConfig.scala
│ ├── SerializationUtil.scala
│ ├── SimpleBackend.scala
│ ├── package.scala
│ └── safe
│ ├── Checkpoint.scala
│ ├── CoastKafkaSystem.scala
│ ├── CoastStorageEngine.scala
│ ├── MergingChooser.scala
│ ├── Messages.scala
│ ├── StaticCheckpointManager.scala
│ └── TaskCompiler.scala
└── test
└── scala
└── com
└── monovore
└── coast
└── samza
├── SamzaSpec.scala
├── SerializationUtilSpec.scala
└── safe
└── TaskCompilerSpec.scala
/.gitignore:
--------------------------------------------------------------------------------
1 | target
2 |
--------------------------------------------------------------------------------
/LICENSE:
--------------------------------------------------------------------------------
1 |
2 | Apache License
3 | Version 2.0, January 2004
4 | http://www.apache.org/licenses/
5 |
6 | TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
7 |
8 | 1. Definitions.
9 |
10 | "License" shall mean the terms and conditions for use, reproduction,
11 | and distribution as defined by Sections 1 through 9 of this document.
12 |
13 | "Licensor" shall mean the copyright owner or entity authorized by
14 | the copyright owner that is granting the License.
15 |
16 | "Legal Entity" shall mean the union of the acting entity and all
17 | other entities that control, are controlled by, or are under common
18 | control with that entity. For the purposes of this definition,
19 | "control" means (i) the power, direct or indirect, to cause the
20 | direction or management of such entity, whether by contract or
21 | otherwise, or (ii) ownership of fifty percent (50%) or more of the
22 | outstanding shares, or (iii) beneficial ownership of such entity.
23 |
24 | "You" (or "Your") shall mean an individual or Legal Entity
25 | exercising permissions granted by this License.
26 |
27 | "Source" form shall mean the preferred form for making modifications,
28 | including but not limited to software source code, documentation
29 | source, and configuration files.
30 |
31 | "Object" form shall mean any form resulting from mechanical
32 | transformation or translation of a Source form, including but
33 | not limited to compiled object code, generated documentation,
34 | and conversions to other media types.
35 |
36 | "Work" shall mean the work of authorship, whether in Source or
37 | Object form, made available under the License, as indicated by a
38 | copyright notice that is included in or attached to the work
39 | (an example is provided in the Appendix below).
40 |
41 | "Derivative Works" shall mean any work, whether in Source or Object
42 | form, that is based on (or derived from) the Work and for which the
43 | editorial revisions, annotations, elaborations, or other modifications
44 | represent, as a whole, an original work of authorship. For the purposes
45 | of this License, Derivative Works shall not include works that remain
46 | separable from, or merely link (or bind by name) to the interfaces of,
47 | the Work and Derivative Works thereof.
48 |
49 | "Contribution" shall mean any work of authorship, including
50 | the original version of the Work and any modifications or additions
51 | to that Work or Derivative Works thereof, that is intentionally
52 | submitted to Licensor for inclusion in the Work by the copyright owner
53 | or by an individual or Legal Entity authorized to submit on behalf of
54 | the copyright owner. For the purposes of this definition, "submitted"
55 | means any form of electronic, verbal, or written communication sent
56 | to the Licensor or its representatives, including but not limited to
57 | communication on electronic mailing lists, source code control systems,
58 | and issue tracking systems that are managed by, or on behalf of, the
59 | Licensor for the purpose of discussing and improving the Work, but
60 | excluding communication that is conspicuously marked or otherwise
61 | designated in writing by the copyright owner as "Not a Contribution."
62 |
63 | "Contributor" shall mean Licensor and any individual or Legal Entity
64 | on behalf of whom a Contribution has been received by Licensor and
65 | subsequently incorporated within the Work.
66 |
67 | 2. Grant of Copyright License. Subject to the terms and conditions of
68 | this License, each Contributor hereby grants to You a perpetual,
69 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable
70 | copyright license to reproduce, prepare Derivative Works of,
71 | publicly display, publicly perform, sublicense, and distribute the
72 | Work and such Derivative Works in Source or Object form.
73 |
74 | 3. Grant of Patent License. Subject to the terms and conditions of
75 | this License, each Contributor hereby grants to You a perpetual,
76 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable
77 | (except as stated in this section) patent license to make, have made,
78 | use, offer to sell, sell, import, and otherwise transfer the Work,
79 | where such license applies only to those patent claims licensable
80 | by such Contributor that are necessarily infringed by their
81 | Contribution(s) alone or by combination of their Contribution(s)
82 | with the Work to which such Contribution(s) was submitted. If You
83 | institute patent litigation against any entity (including a
84 | cross-claim or counterclaim in a lawsuit) alleging that the Work
85 | or a Contribution incorporated within the Work constitutes direct
86 | or contributory patent infringement, then any patent licenses
87 | granted to You under this License for that Work shall terminate
88 | as of the date such litigation is filed.
89 |
90 | 4. Redistribution. You may reproduce and distribute copies of the
91 | Work or Derivative Works thereof in any medium, with or without
92 | modifications, and in Source or Object form, provided that You
93 | meet the following conditions:
94 |
95 | (a) You must give any other recipients of the Work or
96 | Derivative Works a copy of this License; and
97 |
98 | (b) You must cause any modified files to carry prominent notices
99 | stating that You changed the files; and
100 |
101 | (c) You must retain, in the Source form of any Derivative Works
102 | that You distribute, all copyright, patent, trademark, and
103 | attribution notices from the Source form of the Work,
104 | excluding those notices that do not pertain to any part of
105 | the Derivative Works; and
106 |
107 | (d) If the Work includes a "NOTICE" text file as part of its
108 | distribution, then any Derivative Works that You distribute must
109 | include a readable copy of the attribution notices contained
110 | within such NOTICE file, excluding those notices that do not
111 | pertain to any part of the Derivative Works, in at least one
112 | of the following places: within a NOTICE text file distributed
113 | as part of the Derivative Works; within the Source form or
114 | documentation, if provided along with the Derivative Works; or,
115 | within a display generated by the Derivative Works, if and
116 | wherever such third-party notices normally appear. The contents
117 | of the NOTICE file are for informational purposes only and
118 | do not modify the License. You may add Your own attribution
119 | notices within Derivative Works that You distribute, alongside
120 | or as an addendum to the NOTICE text from the Work, provided
121 | that such additional attribution notices cannot be construed
122 | as modifying the License.
123 |
124 | You may add Your own copyright statement to Your modifications and
125 | may provide additional or different license terms and conditions
126 | for use, reproduction, or distribution of Your modifications, or
127 | for any such Derivative Works as a whole, provided Your use,
128 | reproduction, and distribution of the Work otherwise complies with
129 | the conditions stated in this License.
130 |
131 | 5. Submission of Contributions. Unless You explicitly state otherwise,
132 | any Contribution intentionally submitted for inclusion in the Work
133 | by You to the Licensor shall be under the terms and conditions of
134 | this License, without any additional terms or conditions.
135 | Notwithstanding the above, nothing herein shall supersede or modify
136 | the terms of any separate license agreement you may have executed
137 | with Licensor regarding such Contributions.
138 |
139 | 6. Trademarks. This License does not grant permission to use the trade
140 | names, trademarks, service marks, or product names of the Licensor,
141 | except as required for reasonable and customary use in describing the
142 | origin of the Work and reproducing the content of the NOTICE file.
143 |
144 | 7. Disclaimer of Warranty. Unless required by applicable law or
145 | agreed to in writing, Licensor provides the Work (and each
146 | Contributor provides its Contributions) on an "AS IS" BASIS,
147 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
148 | implied, including, without limitation, any warranties or conditions
149 | of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
150 | PARTICULAR PURPOSE. You are solely responsible for determining the
151 | appropriateness of using or redistributing the Work and assume any
152 | risks associated with Your exercise of permissions under this License.
153 |
154 | 8. Limitation of Liability. In no event and under no legal theory,
155 | whether in tort (including negligence), contract, or otherwise,
156 | unless required by applicable law (such as deliberate and grossly
157 | negligent acts) or agreed to in writing, shall any Contributor be
158 | liable to You for damages, including any direct, indirect, special,
159 | incidental, or consequential damages of any character arising as a
160 | result of this License or out of the use or inability to use the
161 | Work (including but not limited to damages for loss of goodwill,
162 | work stoppage, computer failure or malfunction, or any and all
163 | other commercial damages or losses), even if such Contributor
164 | has been advised of the possibility of such damages.
165 |
166 | 9. Accepting Warranty or Additional Liability. While redistributing
167 | the Work or Derivative Works thereof, You may choose to offer,
168 | and charge a fee for, acceptance of support, warranty, indemnity,
169 | or other liability obligations and/or rights consistent with this
170 | License. However, in accepting such obligations, You may act only
171 | on Your own behalf and on Your sole responsibility, not on behalf
172 | of any other Contributor, and only if You agree to indemnify,
173 | defend, and hold each Contributor harmless for any liability
174 | incurred by, or claims asserted against, such Contributor by reason
175 | of your accepting any such warranty or additional liability.
176 |
177 | END OF TERMS AND CONDITIONS
178 |
179 | APPENDIX: How to apply the Apache License to your work.
180 |
181 | To apply the Apache License to your work, attach the following
182 | boilerplate notice, with the fields enclosed by brackets "[]"
183 | replaced with your own identifying information. (Don't include
184 | the brackets!) The text should be enclosed in the appropriate
185 | comment syntax for the file format. We also recommend that a
186 | file or class name and description of purpose be included on the
187 | same "printed page" as the copyright notice for easier
188 | identification within third-party archives.
189 |
190 | Copyright [yyyy] [name of copyright owner]
191 |
192 | Licensed under the Apache License, Version 2.0 (the "License");
193 | you may not use this file except in compliance with the License.
194 | You may obtain a copy of the License at
195 |
196 | http://www.apache.org/licenses/LICENSE-2.0
197 |
198 | Unless required by applicable law or agreed to in writing, software
199 | distributed under the License is distributed on an "AS IS" BASIS,
200 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
201 | See the License for the specific language governing permissions and
202 | limitations under the License.
203 |
--------------------------------------------------------------------------------
/NOTICE:
--------------------------------------------------------------------------------
1 | Coast
2 | =====
3 |
4 | Copyright 2014 Ben Kirwin
5 |
--------------------------------------------------------------------------------
/README.md:
--------------------------------------------------------------------------------
1 | # coast
2 |
3 | In this dark stream-processing landscape, `coast` is a ray of light.
4 |
5 | ## Why `coast`?
6 |
7 | - **Simple:** `coast` provides a simple streaming model with strong ordering and
8 | exactly-once semantics. This straightforward behaviour extends across multiple
9 | machines, state aggregations, and even between independent jobs, making it
10 | easier to reason about how your entire system behaves.
11 |
12 | - **Easy:** Streams are built up and wired together using a concise, idiomatic
13 | Scala API. These dataflow graphs can be as small or as large as you like:
14 | no need to cram all your logic in one big job, or to write a bunch
15 | of single-stage jobs and track their relationships by hand.
16 |
17 | - **Kafkaesque:** `coast`'s core abstractions are patterned after Kafka's
18 | data model, and it's designed to fit comfortably in the middle of a larger
19 | Kafka-based infrastructure. By taking advantage of Kafka's messaging
20 | guarantees, `coast` can implement [exactly-once semantics][impossible]
21 | for messages and state without a heavy coordination cost.
22 |
23 | ## Quick Introduction
24 |
25 | `coast`'s streams are closely patterned after Kafka's topics: a stream has
26 | multiple partitions, and each partition has an ordered series of values. A
27 | stream can have any number of partitions, each of which has a unique key.
28 | You can create a stream by pulling data from a topic, but `coast` also
29 | has a rich API for building derivative streams: applying transformations,
30 | merging streams together, regrouping, aggregating state, or performing joins.
31 | Once you've defined a stream you like, you can give it a name and publish it
32 | out to another topic.
33 |
34 | By defining streams and networking them together, it's possible to
35 | express arbitrarily-complex dataflow graphs, including cycles and joins. You can
36 | use the resulting graphs in multiple ways: print it out as a GraphViz image,
37 | unit-test your logic using a simple in-memory implementation, or compile the
38 | graph to multiple [Samza jobs][samza] and run it on a cluster.
39 |
40 | Sound promising? You might be interested in:
41 | - The [heavily-commented 'Twitter reach' example][twitter-reach],
42 | which walks through all the pieces of a real job.
43 | - A [fork of the `hello-samza` project][hello-coast] with setup and deployment instructions.
44 | - Some [wiki documentation][wiki] on [the core concepts][wiki-overview],
45 | nuances of the [graph-builder API][wiki-flow],
46 | or [details on the Samza integration][wiki-samza].
47 |
48 | [samza]: http://samza.apache.org/
49 | [hello-coast]: https://github.com/bkirwi/incubator-samza-hello-samza/tree/hello-coast
50 | [twitter-reach]: core/src/main/scala/com/monovore/example/coast/TwitterReach.scala
51 | [impossible]: http://ben.kirw.in/2014/11/28/kafka-patterns/
52 | [wiki]: https://github.com/bkirwi/coast/wiki
53 | [wiki-overview]: https://github.com/bkirwi/coast/wiki/Overview
54 | [wiki-flow]: https://github.com/bkirwi/coast/wiki/Flow
55 | [wiki-samza]: https://github.com/bkirwi/coast/wiki/Samza
56 |
57 | ## Getting Started
58 |
59 | The 0.2.0 release is published on Bintray.
60 | If you're using maven, you'll want to point your `pom.xml` at the repo:
61 |
62 | ```xml
63 |
64 | bintray-coast
65 | https://dl.bintray.com/bkirwi/maven
66 |
67 | ```
68 |
69 | ...and add `coast` to your dependencies:
70 |
71 | ```xml
72 |
73 | com.monovore
74 | coast-samza_2.10
75 | 0.2.0
76 |
77 | ```
78 |
79 | *Mutatis mutandis*, the same goes for SBT and Gradle.
80 |
81 | ## Mandatory Word Count Example
82 |
83 | ```scala
84 | val Sentences = Topic[Source, String]("sentences")
85 |
86 | val WordCounts = Topic[String, Int]("word-counts")
87 |
88 | val graph = Flow.build { implicit builder =>
89 |
90 | Sentences.asSource
91 | .flatMap { _.split("\\s+") }
92 | .map { _ -> 1 }
93 | .groupByKey
94 | .streamTo("words")
95 | .sum.updates
96 | .sinkTo(WordCounts)
97 | }
98 | ```
99 |
100 | ## Future Work
101 |
102 | If you're interested in what the future holds for `coast` --
103 | or have questions or bugs to report --
104 | come on over to the [issue tracker][issues].
105 |
106 | [issues]: https://github.com/bkirwi/coast/issues
107 |
--------------------------------------------------------------------------------
/core/README.md:
--------------------------------------------------------------------------------
1 | # `core`
2 |
3 | This module contains the core coast abstractions, along with visualization
4 | tools, an in-memory implementation, and a bunch of example jobs.
5 |
--------------------------------------------------------------------------------
/core/build.sbt:
--------------------------------------------------------------------------------
1 |
2 | libraryDependencies ++= Seq(
3 | "com.google.guava" % "guava" % "18.0",
4 | "com.google.code.findbugs" % "jsr305" % "1.3.9" % "provided",
5 | "com.twitter" %% "algebird-core" % "0.9.0"
6 | )
7 |
--------------------------------------------------------------------------------
/core/src/main/scala/com/monovore/coast/core/Process.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.coast
2 | package core
3 |
4 | import com.twitter.algebird.Aggregator
5 |
6 | /**
7 | * A trait that captures a stream transformation from A to B, with a state of S.
8 | *
9 | * Processes are very composable: it's possible to build up a complex
10 | * stream process by plugging simpler process together.
11 | */
12 | trait Process[S, -A, +B] {
13 |
14 | /**
15 | * Given the current state and a new input, it returns an updated state and a
16 | * sequence of outputs.
17 | */
18 | def apply(state: S, input: A): (S, Seq[B])
19 |
20 | def map[B0](mapFn: B => B0): Process[S, A, B0] =
21 | Process { (s, a) =>
22 | val (s0, bs) = apply(s, a)
23 | (s0, bs.map(mapFn))
24 | }
25 |
26 | def flatMap[A0 <: A, B0](flatMapFn: B => Process[S, A0, B0]): Process[S, A0, B0] =
27 | Process { (s: S, a: A0) =>
28 | val (s0, bs) = apply(s, a)
29 |
30 | bs.foldLeft(s0 -> Seq.empty[B0]) { (pair, b) =>
31 | val (s, b0s) = pair
32 | val (s0, moreBs) = flatMapFn(b).apply(s, a)
33 | s0 -> (b0s ++ moreBs)
34 | }
35 | }
36 |
37 | def andThen[A0 <: A, B0 >: B](other: Process[S, A0, B0]): Process[S, A0, B0] =
38 | Process { (s, a) =>
39 | val (s0, b0) = apply(s, a)
40 | val (s1, b1) = other.apply(s0, a)
41 | s1 -> (b0 ++ b1)
42 | }
43 |
44 | def chain[B0](other: Process[S, B, B0]): Process[S, A, B0] =
45 | Process { (s, a) =>
46 | val (s0, bs) = apply(s, a)
47 |
48 | bs.foldLeft(s0 -> Seq.empty[B0]) { (pair, b) =>
49 | val (s, b0s) = pair
50 | val (s0, moreBs) = other.apply(s, b)
51 | s0 -> (b0s ++ moreBs)
52 | }
53 | }
54 | }
55 |
56 | object Process {
57 |
58 | def apply[S, A, B](function: (S, A) => (S, Seq[B])): Process[S, A, B] =
59 | new Process[S, A, B] {
60 | override def apply(state: S, input: A): (S, Seq[B]) = function(state, input)
61 | }
62 |
63 | def skip[S, A, B]: Process[S, A, B] = Process { (s, a) => (s, Nil) }
64 |
65 | def output[S, A, B](bs: B*) = Process { (s: S, a: A) => (s, bs) }
66 |
67 | def outputEach[S, A, B](bs: Seq[B]) = Process { (s: S, a: A) => (s, bs) }
68 |
69 | def on[S, A, B](func: (S, A) => Process[S, A, B]) =
70 | Process { (s: S, a: A) => func(s, a).apply(s, a) }
71 |
72 | def onInput[S, A, B](func: A => Process[S, A, B]) =
73 | Process { (s: S, a: A) => func(a).apply(s, a) }
74 |
75 | def onState[S, A, B](func: S => Process[S, A, B]) =
76 | Process[S, A, B] { (s: S, a: A) => func(s).apply(s, a) }
77 |
78 | def setState[S](s: S) =
79 | Process[S, Any, Nothing] { (_: S, _: Any) => (s, Nil) }
80 |
81 | def fromAggregator[S, A, B](aggregator: Aggregator[A, S, B]) =
82 | Process { (s: S, a: A) =>
83 | val reduced = aggregator.append(s, a)
84 | (reduced -> Seq(aggregator.present(reduced)))
85 | }
86 | }
--------------------------------------------------------------------------------
/core/src/main/scala/com/monovore/coast/core/elements.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.coast
2 | package core
3 |
4 | import com.monovore.coast.wire.{Partitioner, Serializer}
5 |
6 | sealed trait Node[A, +B]
7 |
8 | case class Source[A, B](
9 | source: String
10 | )(
11 | implicit val keyFormat: Serializer[A],
12 | val valueFormat: Serializer[B]
13 | ) extends Node[A, B]
14 |
15 | case class Clock(seconds: Long) extends Node[Unit, Long]
16 |
17 | case class StatefulTransform[S, A, B0, +B](
18 | upstream: Node[A, B0],
19 | init: S,
20 | transformer: A => (S, B0) => (S, Seq[B])
21 | )(
22 | implicit val keyFormat: Serializer[A],
23 | val stateFormat: Serializer[S]
24 | ) extends Transform[S, A, B0, B]
25 |
26 | sealed trait Transform[S, A, B0, +B] extends Node[A, B] {
27 | def upstream: Node[A, B0]
28 | def init: S
29 | def transformer: A => (S, B0) => (S, Seq[B])
30 | }
31 |
32 | case class PureTransform[A, B0, B](
33 | upstream: Node[A, B0],
34 | function: A => B0 => Seq[B]
35 | ) extends Transform[Unit, A, B0, B] {
36 |
37 | override val init: Unit = ()
38 |
39 | override val transformer: (A) => (Unit, B0) => (Unit, Seq[B]) = {
40 | a => {
41 | val fn = function(a)
42 |
43 | (_, b) => { () -> fn(b) }
44 | }
45 | }
46 | }
47 |
48 | object Transform {
49 | def unapply[S, A, B0, B](t: Transform[S, A, B0, B]): Option[(Node[A, B0], S, A => (S, B0) => (S, Seq[B]))] =
50 | Some((t.upstream, t.init, t.transformer))
51 |
52 | def apply[S : Serializer, A: Serializer, B0, B](e: Node[A, B0], i: S, t: A => (S, B0) => (S, Seq[B])): Transform[S, A, B0, B] =
53 | StatefulTransform(e, i, t)
54 | }
55 |
56 | case class Merge[A, +B](upstreams: Seq[String -> Node[A, B]]) extends Node[A, B]
57 |
58 | case class GroupBy[A, B, A0](upstream: Node[A0, B], groupBy: A0 => B => A) extends Node[A, B]
59 |
60 | case class Sink[A, B](element: Node[A, B])(
61 | implicit val keyFormat: Serializer[A],
62 | val valueFormat: Serializer[B],
63 | val keyPartitioner: Partitioner[A]
64 | )
65 |
66 | trait Graph {
67 | def bindings: Seq[String -> Sink[_, _]]
68 | }
69 |
--------------------------------------------------------------------------------
/core/src/main/scala/com/monovore/coast/flow/AnyGrouping.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.coast
2 | package flow
3 |
4 | import scala.annotation.implicitNotFound
5 |
6 | class AnyGrouping
7 | class Grouped extends AnyGrouping
8 |
9 | @implicitNotFound("Can't prove that the stream you're working with is grouped.")
10 | trait IsGrouped[-G <: AnyGrouping] {
11 | def stream[A, B](s: StreamDef[G, A, B]): GroupedStream[A, B]
12 | def pool[A, B](p: PoolDef[G, A, B]): GroupedPool[A, B]
13 | }
14 |
15 | object IsGrouped {
16 |
17 | implicit object groupedGrouped extends IsGrouped[Grouped] {
18 |
19 | override def stream[A, B](s: StreamDef[Grouped, A, B]): GroupedStream[A, B] = s
20 |
21 | override def pool[A, B](p: PoolDef[Grouped, A, B]): GroupedPool[A, B] = p
22 | }
23 | }
--------------------------------------------------------------------------------
/core/src/main/scala/com/monovore/coast/flow/Context.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.coast
2 | package flow
3 |
4 | /**
5 | * In the Stream / Pool builders, we often want two versions of a method: one
6 | * that gives access to the key, and one that doesn't (for concision). Instead
7 | * of duplicating all that code, we parameterize the builders by a 'context'
8 | * which specifies whether or not a key is expected. This means we can do both:
9 | *
10 | * stream.map { value => fn(value) }
11 | * stream.withKeys.map { key => value => fn(key, value) }
12 | */
13 | sealed trait Context[K, X[+_]] {
14 | def unwrap[A](wrapped: X[A]): (K => A)
15 | def map[A, B](wrapped: X[A])(function: A => B): X[B]
16 | }
17 |
18 | class NoContext[K] extends Context[K, Id] {
19 | override def unwrap[A](wrapped: Id[A]): (K) => A = { _ => wrapped }
20 | override def map[A, B](wrapped: Id[A])(function: (A) => B): Id[B] = function(wrapped)
21 | }
22 |
23 | class FnContext[K] extends Context[K, From[K]#To] {
24 | override def unwrap[A](wrapped: From[K]#To[A]): (K) => A = wrapped
25 | override def map[A, B](wrapped: From[K]#To[A])(function: (A) => B): From[K]#To[B] = wrapped andThen function
26 | }
--------------------------------------------------------------------------------
/core/src/main/scala/com/monovore/coast/flow/Flow.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.coast
2 | package flow
3 |
4 | import com.monovore.coast.wire.{Partitioner, Serializer}
5 | import core._
6 |
7 | /**
8 | * A mechanism for maintaining name bindings.
9 | * @param bindings
10 | * @param value
11 | * @tparam A
12 | */
13 | case class Flow[+A](bindings: Seq[String -> Sink[_, _]], value: A) extends Graph with FlowLike[A] {
14 |
15 | def map[B](func: A => B): Flow[B] = copy(value = func(value))
16 |
17 | def flatMap[B](func: A => Flow[B]): Flow[B] = {
18 |
19 | val result = func(value)
20 |
21 | val duplicateName = bindings.exists { case (name, _) =>
22 | result.bindings.exists { case (other, _) => name == other }
23 | }
24 |
25 | if (duplicateName) throw new IllegalArgumentException("Reused name binding!")
26 |
27 | Flow(bindings ++ result.bindings, result.value)
28 | }
29 |
30 | override def toFlow: Flow[A] = this
31 | }
32 |
33 | object Flow {
34 |
35 | /**
36 | * Creates a graph without any name bindings.
37 | */
38 | def apply[A](a: A): Flow[A] = Flow(Seq.empty, a)
39 |
40 |
41 | // Builder methods
42 |
43 | def merge[G <: AnyGrouping, A, B](upstreams: (String -> StreamDef[G, A, B])*): StreamDef[G, A, B] = {
44 |
45 | for ((branch -> streams) <- upstreams.groupByKey) {
46 | require(streams.size == 1, s"merged branches must be unique ($branch is specified ${streams.size} times)")
47 | }
48 |
49 | new StreamDef[G, A, B](Merge(upstreams.map { case (name, stream) => name -> stream.element}))
50 | }
51 |
52 | def source[A : Serializer, B : Serializer](topic: Topic[A,B]): GroupedStream[A, B] =
53 | new StreamDef[Grouped, A, B](Source[A, B](topic.name))
54 |
55 | def clock(seconds: Long) = new StreamDef[Grouped, Unit, Long](Clock(seconds))
56 |
57 | def sink[A : Serializer : Partitioner, B : Serializer](topic: Topic[A, B])(flow: FlowLike[GroupedStream[A, B]]): Flow[Unit] = {
58 | flow.toFlow.flatMap { stream => Flow(Seq(topic.name -> Sink(stream.element)), ()) }
59 | }
60 |
61 | def stream[A : Serializer : Partitioner, B : Serializer](label: String)(stream: FlowLike[AnyStream[A, B]]): Flow[GroupedStream[A, B]] =
62 | stream.toFlow.flatMap { stream =>
63 | Flow(Seq(label -> Sink(stream.element)), new StreamDef[Grouped, A, B](Source[A, B](label)))
64 | }
65 |
66 | def pool[A : Serializer : Partitioner, B : Serializer](label: String)(pool: FlowLike[AnyPool[A, B]]): Flow[GroupedPool[A, B]] =
67 | pool.toFlow.flatMap { pool =>
68 | Flow(Seq(label -> Sink(pool.element)), new PoolDef[Grouped, A, B](pool.initial, Source[A, B](label)))
69 | }
70 |
71 | def cycle[A : Serializer : Partitioner, B : Serializer](label: String)(cycle: GroupedStream[A, B] => FlowLike[AnyStream[A, B]]): Flow[GroupedStream[A, B]] = {
72 |
73 | val stream = new StreamDef[Grouped, A, B](Source[A, B](label))
74 |
75 | cycle(stream).toFlow.flatMap { cycled =>
76 | Flow(Seq(label -> Sink(cycled.element)), stream)
77 | }
78 | }
79 |
80 | def builder(): Builder = new Builder()
81 |
82 | class Builder private[Flow](private[Flow] var _bindings: Seq[(String, Sink[_, _])] = Nil) extends Graph {
83 |
84 | def add[A](flow: Flow[A]): A = {
85 | val updated = Flow(_bindings, ()).flatMap { _ => flow }
86 | _bindings = updated.bindings
87 | updated.value
88 | }
89 |
90 | def addCycle[A:Serializer:Partitioner, B:Serializer](name: String)(function: GroupedStream[A, B] => AnyStream[A, B]): GroupedStream[A, B] = {
91 | add(Flow.cycle(name)(function))
92 | }
93 |
94 | override def bindings: Seq[(String, Sink[_, _])] = _bindings
95 |
96 | def toFlow: Flow[Unit] = Flow(bindings, ())
97 | }
98 |
99 | def build[A](fn: Builder => A): Flow[A] = {
100 | val builder = new Builder()
101 | val out = fn(builder)
102 | Flow(builder.bindings, out)
103 | }
104 | }
105 |
106 | trait FlowLike[+A] {
107 | def toFlow: Flow[A]
108 | }
--------------------------------------------------------------------------------
/core/src/main/scala/com/monovore/coast/flow/PoolDef.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.coast
2 | package flow
3 |
4 | import com.monovore.coast.core._
5 | import com.monovore.coast.wire.Serializer
6 |
7 | class PoolDef[+G <: AnyGrouping, A, +B](
8 | private[coast] val initial: B,
9 | private[coast] val element: Node[A, B]
10 | ) { self =>
11 |
12 | def updates: StreamDef[G, A, B] = new StreamDef(element)
13 |
14 | def updatedPairs[B0 >: B](
15 | implicit isGrouped: IsGrouped[G], keyFormat: Serializer[A], valueFormat: Serializer[B0]
16 | ): GroupedStream[A, (B0, B0)] =
17 | updates.transform(initial: B0) { (last, current) =>
18 | current -> Seq(last -> current)
19 | }
20 |
21 | def map[B0](function: B => B0): PoolDef[G, A, B0] =
22 | updates.map(function).latestOr(function(initial))
23 |
24 | def join[B0 >: B, B1](other: GroupedPool[A, B1])(
25 | implicit isGrouped: IsGrouped[G], keyFormat: Serializer[A], pairFormat: Serializer[(B0, B1)]
26 | ): PoolDef[Grouped, A, (B0, B1)] = {
27 |
28 | val merged = Flow.merge(
29 | "left" -> isGrouped.pool(this).updates.map(Left(_)),
30 | "right" -> other.updates.map(Right(_))
31 | )
32 |
33 | merged
34 | .fold(initial: B0, other.initial) { (state, update) =>
35 | update.fold(
36 | { left => (left, state._2) },
37 | { right => (state._1, right) }
38 | )
39 | }
40 | }
41 | }
42 |
--------------------------------------------------------------------------------
/core/src/main/scala/com/monovore/coast/flow/StreamDef.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.coast
2 | package flow
3 |
4 | import com.monovore.coast.core._
5 | import com.monovore.coast.wire._
6 | import com.twitter.algebird.{Group, Monoid, MonoidAggregator, Semigroup}
7 |
8 | class StreamBuilder[WithKey[+_], +G <: AnyGrouping, A, +B](
9 | private[coast] val context: Context[A, WithKey],
10 | private[coast] val element: Node[A, B]
11 | ) { self =>
12 |
13 | def stream: StreamDef[G, A, B] = new StreamDef(element)
14 |
15 | def flatMap[B0](func: WithKey[B => Seq[B0]]): StreamDef[G, A, B0] = new StreamDef(
16 | PureTransform[A, B, B0](self.element, context.unwrap(func))
17 | )
18 |
19 | def filter(func: WithKey[B => Boolean]): StreamDef[G, A, B] = flatMap {
20 | context.map(func) { func =>
21 | { a => if (func(a)) Seq(a) else Seq.empty }
22 | }
23 | }
24 |
25 | def map[B0](func: WithKey[B => B0]): StreamDef[G, A, B0] =
26 | flatMap(context.map(func) { func => func andThen { b => Seq(b) } })
27 |
28 | def collect[B0](func: WithKey[PartialFunction[B, B0]]): StreamDef[G, A, B0] =
29 | flatMap(context.map(func) { func =>
30 | { b => if (func.isDefinedAt(b)) Seq(func(b)) else Seq.empty }
31 | })
32 |
33 | def transform[S, B0](init: S)(func: WithKey[(S, B) => (S, Seq[B0])])(
34 | implicit isGrouped: IsGrouped[G], keyFormat: Serializer[A], stateFormat: Serializer[S]
35 | ): GroupedStream[A, B0] = {
36 |
37 | new StreamDef(StatefulTransform[S, A, B, B0](self.element, init, context.unwrap(func)))
38 | }
39 |
40 | def process[S, B0](init: S)(trans: WithKey[Process[S, B, B0]])(
41 | implicit isGrouped: IsGrouped[G], keyFormat: Serializer[A], stateFormat: Serializer[S]
42 | ): GroupedStream[A, B0] = {
43 |
44 | transform(init)(context.map(trans) { _.apply })
45 | }
46 |
47 | def fold[B0](init: B0)(func: WithKey[(B0, B) => B0])(
48 | implicit isGrouped: IsGrouped[G], keyFormat: Serializer[A], stateFormat: Serializer[B0]
49 | ): GroupedPool[A, B0] = {
50 |
51 | val transformer = context.map(func) { fn =>
52 |
53 | (state: B0, next: B) => {
54 | val newState = fn(state, next)
55 | newState -> Seq(newState)
56 | }
57 | }
58 |
59 | this.transform(init)(transformer).latestOr(init)
60 | }
61 |
62 | def aggregate[S, B0](aggregator: MonoidAggregator[B, S, B0])(
63 | implicit isGrouped: IsGrouped[G], keyFormat: Serializer[A], stateFormat: Serializer[S]
64 | ): GroupedPool[A, B0] = {
65 |
66 | implicit val stateMonoid = aggregator.monoid
67 |
68 | this.stream
69 | .map(aggregator.prepare)
70 | .sum
71 | .map(aggregator.present)
72 | }
73 |
74 | def grouped[B0 >: B](size: Int)(
75 | implicit isGrouped: IsGrouped[G], keyFormat: Serializer[A], stateFormat: Serializer[Seq[B0]]
76 | ): GroupedStream[A, Seq[B0]] = {
77 |
78 | require(size > 0, "Expected a positive group size")
79 |
80 | stream.transform(Vector.empty[B0]: Seq[B0]) { (buffer, next) =>
81 |
82 | if (buffer.size >= size) Vector.empty[B0] -> Seq(buffer)
83 | else (buffer :+ (next: B0)) -> Seq.empty[Seq[B0]]
84 | }
85 | }
86 |
87 | def latestOr[B0 >: B](init: B0): PoolDef[G, A, B0] =
88 | new PoolDef(init, element)
89 |
90 | def latestOption: PoolDef[G, A, Option[B]] =
91 | stream.map { b => Some(b) }.latestOr(None)
92 |
93 | def groupBy[A0](func: WithKey[B => A0]): StreamDef[AnyGrouping, A0, B] =
94 | new StreamDef[G, A0, B](GroupBy(self.element, context.unwrap(func)))
95 |
96 | def groupByKey[A0, B0](implicit asPair: B <:< (A0, B0)) =
97 | stream.groupBy { _._1 }.map { _._2 }
98 |
99 | def invert[A0, B0](implicit asPair: B <:< (A0, B0)): StreamDef[AnyGrouping, A0, (A, B0)] = {
100 | stream
101 | .withKeys.map { key => value => key -> (value: (A0, B0)) }
102 | .groupBy { case (_, (k, _)) => k }
103 | .map { case (k, (_, v)) => k -> v }
104 | }
105 |
106 | def flatten[B0](implicit func: B <:< Seq[B0]) = stream.flatMap(func)
107 |
108 | def flattenOption[B0](implicit func: B <:< Option[B0]) = stream.flatMap(func andThen { _.toSeq })
109 |
110 | def sum[B0 >: B](
111 | implicit monoid: Monoid[B0], isGrouped: IsGrouped[G], keyFormat: Serializer[A], valueFormat: Serializer[B0]
112 | ): GroupedPool[A, B0] = {
113 | stream.fold(monoid.zero)(monoid.plus)
114 | }
115 |
116 | def join[B0](pool: GroupedPool[A, B0])(
117 | implicit isGrouped: IsGrouped[G], keyFormat: Serializer[A], b0Format: Serializer[B0]
118 | ): GroupedStream[A, (B, B0)] = {
119 |
120 | Flow.merge("stream" -> isGrouped.stream(this.stream).map(Right(_)), "pool" -> pool.updates.map(Left(_)))
121 | .transform(pool.initial) { (state: B0, msg: Either[B0, B]) =>
122 | msg match {
123 | case Left(newState) => newState -> Seq.empty
124 | case Right(msg) => state -> Seq(msg -> state)
125 | }
126 | }
127 | }
128 |
129 | def zipWithKey: StreamDef[G, A, (A, B)] =
130 | stream.withKeys.map { k => v => (k, v) }
131 |
132 |
133 | // Builder-related methods
134 |
135 | def streamTo[B0 >: B](name: String)(
136 | implicit keyFormat: Serializer[A], partitioner: Partitioner[A], valueFormat: Serializer[B0], ctx: Flow.Builder
137 | ): GroupedStream[A, B0] = {
138 | ctx.add(Flow.stream[A, B0](name)(stream))
139 | }
140 |
141 | def sinkTo[B0 >: B](topic: Topic[A, B0])(
142 | implicit keyFormat: Serializer[A], partitioner: Partitioner[A], valueFormat: Serializer[B0], grouped: IsGrouped[G], ctx: Flow.Builder
143 | ): Unit = {
144 | ctx.add(Flow.sink(topic)(grouped.stream(stream)))
145 | }
146 |
147 | def sumByKey[K, V](
148 | name: String
149 | )(implicit
150 | isMap: B <:< Map[K, V],
151 | ctx: Flow.Builder,
152 | partitioner: Partitioner[K],
153 | ordering: Ordering[K],
154 | vGroup: Group[V],
155 | isGrouped: IsGrouped[G],
156 | keyFormat: Serializer[A],
157 | newKeyFormat: Serializer[K],
158 | messageFormat: Serializer[V]
159 | ): GroupedPool[K, V] = {
160 |
161 | implicit val c = StreamFormat.fromSerializer(keyFormat)
162 | implicit val a = StreamFormat.fromSerializer(newKeyFormat)
163 | implicit val b = StreamFormat.fromSerializer(messageFormat)
164 |
165 | import Protocol.common._
166 |
167 | stream
168 | .transform(Map.empty[K, V]) { (undoPrevious, next) =>
169 | val asMap = isMap(next)
170 | val messages = Semigroup.plus(undoPrevious, asMap).toSeq.sortBy { _._1 }
171 | Group.negate(asMap) -> messages
172 | }
173 | .groupByKey
174 | .streamTo(name)
175 | .sum
176 | }
177 |
178 | def latestByKey[K, V](
179 | name: String
180 | )(implicit
181 | isMap: B <:< Map[K, V],
182 | ctx: Flow.Builder,
183 | partitioner: Partitioner[K],
184 | ordering: Ordering[K],
185 | isGrouped: IsGrouped[G],
186 | keyFormat: Serializer[A],
187 | newKeyFormat: Serializer[K],
188 | messageFormat: Serializer[V]
189 | ): GroupedPool[K, Map[A, V]] = {
190 |
191 | implicit val c = StreamFormat.fromSerializer(keyFormat)
192 | implicit val a = StreamFormat.fromSerializer(newKeyFormat)
193 | implicit val b = StreamFormat.fromSerializer(messageFormat)
194 |
195 | import Protocol.common._
196 |
197 | stream
198 | .transform(Seq.empty[K]) { (last, next) =>
199 | val asMap = isMap(next)
200 | val remove = last.filterNot(asMap.contains).map { _ -> None }
201 | val add = asMap.mapValues(Some(_)).toSeq.sortBy {_._1 }
202 | add.map { _._1 } -> (remove ++ add)
203 | }
204 | .invert
205 | .streamTo(name)
206 | .fold(Map.empty[A, V]) { (map, update) =>
207 | update match {
208 | case (k, None) => map - k
209 | case (k, Some(v)) => map.updated(k, v)
210 | }
211 | }
212 | }
213 | }
214 |
215 | class StreamDef[+G <: AnyGrouping, A, +B](element: Node[A, B])
216 | extends StreamBuilder[Id, G, A, B](new NoContext[A], element) with FlowLike[StreamDef[G, A, B]] {
217 |
218 | def withKeys: StreamBuilder[From[A]#To, G, A, B] =
219 | new StreamBuilder[From[A]#To, G, A, B](new FnContext[A], element)
220 |
221 | override def toFlow: Flow[StreamDef[G, A, B]] = Flow(this)
222 | }
223 |
--------------------------------------------------------------------------------
/core/src/main/scala/com/monovore/coast/flow/package.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.coast
2 |
3 | import com.monovore.coast.wire.{Serializer, Partitioner}
4 |
5 | package object flow {
6 |
7 | type AnyStream[A, +B] = StreamDef[AnyGrouping, A, B]
8 | type GroupedStream[A, +B] = StreamDef[Grouped, A, B]
9 |
10 | type AnyPool[A, +B] = PoolDef[AnyGrouping, A, B]
11 | type GroupedPool[A, +B] = PoolDef[Grouped, A, B]
12 |
13 | case class Topic[A, B](name: String) {
14 | def asSource(implicit af: Serializer[A], bf: Serializer[B]): GroupedStream[A, B] = Flow.source(this)
15 | }
16 |
17 | private[coast] type Id[+A] = A
18 |
19 | private[coast] type From[A] = { type To[+B] = (A => B) }
20 | }
21 |
--------------------------------------------------------------------------------
/core/src/main/scala/com/monovore/coast/machine/Machine.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.coast
2 | package machine
3 |
4 | import com.monovore.coast.flow._
5 | import com.monovore.coast.core._
6 |
7 | import com.twitter.algebird.Semigroup
8 |
9 | object Machine {
10 |
11 | sealed trait Label
12 | case class Named(name: String) extends Label
13 | case class Anonymous(index: Int) extends Label
14 |
15 | def compile(graph: Graph): Machine = {
16 |
17 | val newID = {
18 |
19 | var count = 0
20 |
21 | () => {
22 | count += 1
23 | Anonymous(count): Label
24 | }
25 | }
26 |
27 | def compile[A, B](
28 | downstream: Label,
29 | flow: Node[A, B]
30 | ): (Map[Label, Actor] -> Seq[Label -> Label]) = flow match {
31 | case Source(name) => {
32 | Map.empty[Label, Actor] -> Seq(Named(name) -> downstream)
33 | }
34 | case Transform(upstream, init, transformer) => {
35 |
36 | val id = newID()
37 |
38 | val (nodes -> edges) = compile(id, upstream)
39 |
40 | val node = Actor(State(init), { case (s, k, blob) =>
41 | val (newS, messages) = transformer(k.cast)(s.cast, blob.cast)
42 | State(newS) -> Map(k -> messages.map(Message(_)))
43 | })
44 |
45 | nodes.updated(id, node) -> (edges ++ Seq(id -> downstream))
46 | }
47 | case Merge(upstreams) => {
48 |
49 | val id = newID()
50 |
51 | val (nodes, edges) = upstreams
52 | .foldLeft(Map.empty[Label, Actor] -> Seq.empty[Label -> Label]) { (soFar, upstreamPair) =>
53 |
54 | val (_ -> upstream) = upstreamPair
55 |
56 | val (nodes, edges) = soFar
57 |
58 | val (newNodes, newEdges) = compile(id, upstream)
59 |
60 | (nodes ++ newNodes) -> (edges ++ newEdges)
61 | }
62 |
63 | nodes.updated(id, Actor.passthrough) -> (edges ++ Seq(id -> downstream))
64 | }
65 | case GroupBy(upstream, groupBy) => {
66 |
67 | val id = newID()
68 |
69 | val (nodes, edges) = compile(id, upstream)
70 |
71 | val actor = Actor(State(unit), { case (s, key, input) =>
72 | val group = groupBy(key.cast)(input.cast)
73 | (s, Map(Key(group) -> Seq(input)))
74 | })
75 |
76 | nodes.updated(id, actor) -> (edges ++ Seq(id -> downstream))
77 | }
78 | }
79 |
80 | val (nodes, edges) = graph.bindings
81 | .map { case (key -> Sink(flow)) =>
82 |
83 | val (nodes, edges) = compile(Named(key), flow)
84 |
85 | nodes -> edges
86 | }
87 | .unzip
88 |
89 |
90 | val edgeMap = edges.flatten.groupByKey
91 |
92 | val nodeMap = nodes.flatten.toMap
93 |
94 | Machine(System(nodes = nodeMap, edges = edgeMap), System.State())
95 | }
96 | }
97 |
98 | case class Machine(system: System[Machine.Label], state: System.State[Machine.Label]) {
99 |
100 | def push(messages: Messages): Machine = {
101 |
102 | val toSend = for {
103 | (name, data) <- messages.messageMap
104 | label = Machine.Named(name)
105 | (key, messages) <- data
106 | message <- messages
107 | } yield System.Send[Machine.Label](label, key, message)
108 |
109 | val newState = toSend.foldLeft(state) { (state, send) =>
110 | system.update(state, send)._1
111 | }
112 |
113 | Machine(system, newState)
114 | }
115 |
116 | def next: Seq[() => (Machine, Messages)] = {
117 |
118 | system.commands(state).map { command =>
119 | () => {
120 |
121 | val (newState, messageMap) = system.update(state, command)
122 |
123 | val messages = messageMap
124 | .flatMap {
125 | case (Machine.Named(name) -> value) => Some(name -> value)
126 | case _ => None
127 | }
128 |
129 | Machine(system, newState) -> Messages(messages)
130 | }
131 | }
132 | }
133 | }
134 |
135 |
136 | case class Messages(messageMap: Map[String, Map[Key, Seq[Message]]]) {
137 |
138 | def apply[A, B](name: Topic[A, B]): Map[A, Seq[B]] = {
139 |
140 | val keyed = messageMap.getOrElse(name.name, Map.empty)
141 |
142 | keyed.map { case (k -> v) => k.cast[A] -> v.map { _.cast[B] } }.withDefaultValue(Nil)
143 | }
144 |
145 | def ++(other: Messages): Messages = Messages(Semigroup.plus(messageMap, other.messageMap))
146 |
147 | def isEmpty: Boolean = messageMap.values.forall { _.values.forall { _.isEmpty } }
148 | }
149 |
150 | object Messages {
151 |
152 | val empty: Messages = Messages(Map.empty)
153 |
154 | def from[A, B](name: Topic[A, B], messages: Map[A, Seq[B]]): Messages = {
155 |
156 | Messages(Map(
157 | name.name -> messages
158 | .filter { case (_ -> v) => v.nonEmpty }
159 | .map { case (k -> v) => Key(k) -> v.map(Message) }
160 | ))
161 | }
162 | }
--------------------------------------------------------------------------------
/core/src/main/scala/com/monovore/coast/machine/System.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.coast
2 | package machine
3 |
4 | import com.twitter.algebird.Semigroup
5 |
6 | private[machine] case class Message(get: Any) { def cast[T]: T = get.asInstanceOf[T] }
7 | private[machine] case class State(get: Any) { def cast[T]: T = get.asInstanceOf[T] }
8 | private[machine] case class Key(get: Any) { def cast[T]: T = get.asInstanceOf[T] }
9 |
10 | case class Actor(
11 | initialState: State,
12 | push: (State, Key, Message) => (State, Map[Key, Seq[Message]])
13 | )
14 |
15 | object Actor {
16 |
17 | val passthrough = Actor(State(unit), { case (s, k, m) => s -> Map(k -> Seq(m)) } )
18 |
19 | case class Data[Label](state: State, input: Map[Label, Seq[Message]] = Map.empty[Label, Seq[Message]])
20 | }
21 |
22 | object System {
23 |
24 | case class State[Label](stateMap: Map[Label, Map[Key, Actor.Data[Label]]] = Map.empty[Label, Map[Key, Actor.Data[Label]]])
25 |
26 | sealed trait Command[Label]
27 | case class Send[Label](from: Label, partition: Key, message: Message) extends Command[Label]
28 | case class Process[Label](actor: Label, from: Label, key: Key) extends Command[Label]
29 | }
30 |
31 | case class System[Label](
32 | nodes: Map[Label, Actor] = Map.empty[Label, Actor],
33 | edges: Map[Label, Seq[Label]] = Map.empty[Label, Seq[Label]]
34 | ) {
35 |
36 | def update(state: System.State[Label], command: System.Command[Label]): (System.State[Label], Map[Label, Map[Key, Seq[Message]]]) = {
37 |
38 | command match {
39 |
40 | case System.Process(actor, from, key) => {
41 |
42 | val updated = {
43 |
44 | val node = nodes.getOrElse(actor, Actor.passthrough)
45 | val actorState = state.stateMap.getOrElse(actor, Map.empty)
46 | val keyState = actorState.getOrElse(key, Actor.Data(node.initialState))
47 | val messages = keyState.input.getOrElse(from, Seq.empty)
48 |
49 | assuming(messages.nonEmpty) {
50 |
51 | val (newState, output) = node.push(keyState.state, key, messages.head)
52 |
53 | val newPartitionState = Actor.Data(newState, keyState.input.updated(from, messages.tail))
54 |
55 | val tidied = System.State(state.stateMap.updated(actor, actorState.updated(key, newPartitionState)))
56 |
57 | val steps = for {
58 | (key, messages) <- output
59 | message <- messages
60 | } yield System.Send(actor, key, message)
61 |
62 | steps.foldLeft(tidied -> Map.empty[Label, Map[Key, Seq[Message]]]) { (stateMessages, send) =>
63 | val (state, messages) = stateMessages
64 | val (newState, newMessages) = update(state, send)
65 | newState -> Semigroup.plus(messages, newMessages)
66 | }
67 | }
68 | }
69 |
70 | updated getOrElse (state -> Map.empty)
71 | }
72 |
73 | case System.Send(from, partition, message) => {
74 |
75 | val newState = edges.getOrElse(from, Seq.empty)
76 | .foldLeft(state) { (state, to) =>
77 |
78 | val targetState = state.stateMap.getOrElse(to, Map.empty)
79 | val partitioned = targetState.getOrElse(partition, Actor.Data(state = nodes.getOrElse(to, Actor.passthrough).initialState))
80 | val pushed = partitioned.copy(input = Semigroup.plus(partitioned.input, Map(from -> Seq(message))))
81 | System.State(state.stateMap.updated(to, targetState.updated(partition, pushed)))
82 | }
83 |
84 | newState -> Map(from -> Map(partition -> Seq(message)))
85 | }
86 | }
87 | }
88 |
89 | def commands(state: System.State[Label]): Seq[System.Command[Label]] = {
90 |
91 | for {
92 | (label -> partitions) <- state.stateMap.toSeq
93 | (key -> partitionState) <- partitions
94 | (from -> messages) <- partitionState.input
95 | if messages.nonEmpty
96 | } yield System.Process(label, from, key)
97 | }
98 | }
99 |
--------------------------------------------------------------------------------
/core/src/main/scala/com/monovore/coast/package.scala:
--------------------------------------------------------------------------------
1 | package com.monovore
2 |
3 | package object coast {
4 |
5 | // IMPLEMENTATION
6 | // always-visible utilities; should be hidden within the coast package
7 |
8 | private[coast] val unit: Unit = ()
9 |
10 | private[coast] type ->[A, B] = (A, B)
11 |
12 | private[coast] object -> {
13 | def unapply[A, B](pair: (A, B)) = Some(pair)
14 | }
15 |
16 | private[coast] implicit class SeqOps[A](underlying: Seq[A]) {
17 | def groupByKey[B,C](implicit proof: A <:< (B, C)): Map[B, Seq[C]] =
18 | underlying.groupBy { _._1 }.mapValues { _.unzip._2 }
19 | }
20 |
21 | private[coast] def assuming[A](cond: Boolean)(action: => A): Option[A] =
22 | if (cond) Some(action) else None
23 | }
24 |
--------------------------------------------------------------------------------
/core/src/main/scala/com/monovore/coast/viz/Dot.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.coast
2 | package viz
3 |
4 | import java.util.concurrent.atomic.AtomicInteger
5 |
6 | import com.monovore.coast.core._
7 |
8 | object Dot {
9 |
10 | private[this] sealed trait LabelType
11 | private[this] case class Public(name: String) extends LabelType { override def toString = "\"public-" + name + "\"" }
12 | private[this] case class Private(count: Int) extends LabelType { override def toString = "\"internal-" + count + "\"" }
13 |
14 | private[this] case class Label(global: LabelType, pretty: String)
15 | private[this] case class Edge(from: Label, to: Label, tag: Option[String] = None)
16 |
17 | /**
18 | * Write out a graph description in the
19 | * [[http://en.wikipedia.org/wiki/DOT_(graph_description_language) DOT graph
20 | * description language]]. You'll probably want to hand this off to another
21 | * tool for visualization or further processing.
22 | */
23 | def describe(graph: Graph): String = {
24 |
25 | val newID: String => Label = {
26 | val atomic = new AtomicInteger()
27 | (pretty) => { Label(Private(atomic.getAndIncrement), pretty) }
28 | }
29 |
30 | def sources[A, B](downstream: Label, flow: Node[A, B]): Seq[Edge] = flow match {
31 | case Source(name) => {
32 | val label = Label(Public(name), name)
33 | Seq(Edge(label, downstream))
34 | }
35 | case Clock(seconds) => {
36 | val id = newID(s"clock [$seconds seconds]")
37 | Seq(Edge(id, downstream))
38 | }
39 | case PureTransform(upstream, _) => {
40 | val id = newID("transform")
41 | sources(id, upstream) ++ Seq(Edge(id, downstream))
42 | }
43 | case StatefulTransform(upstream, _, _) => {
44 | val id = newID("aggregate")
45 | sources(id, upstream) ++ Seq(Edge(id, downstream))
46 | }
47 | case Merge(upstreams) => {
48 | val id = newID("merge")
49 | upstreams.map { case (name -> up) => sources(id, up) }.flatten ++ Seq(Edge(id, downstream))
50 | }
51 | case GroupBy(upstream, _) => {
52 | val id = newID("groupBy")
53 | sources(id, upstream) ++ Seq(Edge(id, downstream))
54 | }
55 | }
56 |
57 | val chains = graph.bindings
58 | .map { case (name -> Sink(flow)) =>
59 | name -> sources(Label(Public(name), name), flow)
60 | }
61 | .toMap
62 |
63 | val subgraphs = for ((label, chain) <- chains) yield {
64 |
65 | val nodes = chain
66 | .flatMap { case Edge(k, v, _) => Seq(k, v) }.toSet
67 | .map { label: Label =>
68 |
69 | val Label(global, pretty) = label
70 |
71 | val shape = global match {
72 | case Public(_) => "rectangle"
73 | case Private(_) => "plaintext"
74 | }
75 |
76 |
77 | s"""$global [shape=$shape, label="${pretty}"];"""
78 | }
79 |
80 | val edges = chain.map { case Edge(Label(k, _), Label(v, _), tag) =>
81 | val options = tag.map { t => s"""[label="${t}"]""" }.getOrElse("")
82 | s"$k -> $v; $options"
83 | }
84 |
85 | s""" subgraph "cluster-$label" {
86 | |
87 | | label="$label"
88 | | labeljust=l
89 | |
90 | | // nodes
91 | | ${ nodes.mkString("\n ") }
92 | |
93 | | // edges
94 | | ${ edges.mkString("\n ") }
95 | | }
96 | """.stripMargin
97 | }
98 |
99 | s"""digraph flow {
100 | |
101 | |${subgraphs.mkString("\n")}
102 | |}
103 | |""".stripMargin
104 | }
105 | }
106 |
--------------------------------------------------------------------------------
/core/src/main/scala/com/monovore/coast/wire/Partitioner.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.coast.wire
2 |
3 | import com.google.common.hash.{Funnel, HashCode, HashFunction}
4 | import com.google.common.primitives.UnsignedInts
5 |
6 | import scala.annotation.implicitNotFound
7 | import scala.language.existentials
8 |
9 | /**
10 | * Hashes the value A to a partition in the range [0, numPartitions). This is
11 | * analogous to Kafka's partitioner class, but meant to be used as a typeclass.
12 | * This makes it easier to configure partitioning strategies per-topic, instead
13 | * of per-producer-instance.
14 | */
15 | @implicitNotFound("No partitioner for in scope for key type ${A}")
16 | trait Partitioner[-A] extends Serializable {
17 | def partition(a: A, numPartitions: Int): Int
18 | }
19 |
20 | object Partitioner {
21 |
22 | /**
23 | * Our default partitioner should behave the same as Kafka's default partitioner.
24 | */
25 | val byHashCode = new Partitioner[Any] {
26 | override def partition(a: Any, numPartitions: Int): Int = {
27 | // Kafka uses bitwise ops instead of [[scala.math.abs]] to avoid strange behaviour at Int.MinValue
28 | (a.hashCode & 0x7fffffff) % numPartitions
29 | }
30 | }
31 | }
32 |
--------------------------------------------------------------------------------
/core/src/main/scala/com/monovore/coast/wire/Protocol.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.coast.wire
2 |
3 | import java.io.{ByteArrayInputStream, DataInputStream, DataOutputStream, ByteArrayOutputStream}
4 |
5 | import com.google.common.base.Charsets
6 |
7 | object Protocol {
8 |
9 | object simple {
10 |
11 | implicit val utf8: Serializer[String] = new Serializer[String] {
12 | override def toArray(value: String): Array[Byte] = value.getBytes(Charsets.UTF_8)
13 | override def fromArray(bytes: Array[Byte]): String = new String(bytes, Charsets.UTF_8)
14 | }
15 |
16 | implicit val longFormat: Serializer[Long] = new Serializer[Long] {
17 | override def toArray(value: Long): Array[Byte] = value.toString.getBytes(Charsets.US_ASCII)
18 | override def fromArray(bytes: Array[Byte]): Long = new String(bytes, Charsets.US_ASCII).toLong
19 | }
20 |
21 | implicit val intFormat: Serializer[Int] = new Serializer[Int] {
22 | override def toArray(value: Int): Array[Byte] = value.toString.getBytes(Charsets.US_ASCII)
23 | override def fromArray(bytes: Array[Byte]): Int = new String(bytes, Charsets.US_ASCII).toInt
24 | }
25 |
26 | implicit val unit: Serializer[Unit] = new Serializer[Unit] {
27 | private[this] val empty: Array[Byte] = Array.ofDim(0)
28 | override def toArray(value: Unit): Array[Byte] = empty
29 | override def fromArray(bytes: Array[Byte]): Unit = {
30 | require(bytes.length == 0, "Expecting empty byte array.")
31 | }
32 | }
33 |
34 | implicit val intPartitioner: Partitioner[Int] = Partitioner.byHashCode
35 |
36 | implicit val stringPartitioner: Partitioner[String] = Partitioner.byHashCode
37 |
38 | implicit val longPartitioner: Partitioner[Long] = Partitioner.byHashCode
39 |
40 | implicit object unitPartitioner extends Partitioner[Unit] {
41 | override def partition(a: Unit, numPartitions: Int): Int = 0
42 | }
43 | }
44 |
45 | object common {
46 |
47 | implicit def streamFormatSerializer[A](implicit format: StreamFormat[A]): Serializer[A] = new Serializer[A] {
48 | override def toArray(value: A): Array[Byte] = {
49 | val baos = new ByteArrayOutputStream()
50 | format.write(new DataOutputStream(baos), value)
51 | baos.toByteArray
52 | }
53 | override def fromArray(bytes: Array[Byte]): A = {
54 | format.read(new DataInputStream(new ByteArrayInputStream(bytes)))
55 | }
56 | }
57 |
58 | implicit val intPartitioner: Partitioner[Int] = Partitioner.byHashCode
59 |
60 | implicit val stringPartitioner: Partitioner[String] = Partitioner.byHashCode
61 |
62 | implicit val longPartitioner: Partitioner[Long] = Partitioner.byHashCode
63 |
64 | implicit object unitPartitioner extends Partitioner[Unit] {
65 | override def partition(a: Unit, numPartitions: Int): Int = 0
66 | }
67 | }
68 |
69 | object native {
70 |
71 | implicit def anyPartitioner[A]: Partitioner[A] = Partitioner.byHashCode
72 |
73 | implicit def anyFormat[A]: Serializer[A] = Serializer.fromJavaSerialization[A]
74 | }
75 | }
--------------------------------------------------------------------------------
/core/src/main/scala/com/monovore/coast/wire/Serializer.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.coast.wire
2 |
3 | import java.io._
4 |
5 | import scala.language.implicitConversions
6 |
7 | /**
8 | * Manages reading and writing data to Java's standard Data{Input,Output} classes.
9 | */
10 | trait Serializer[A] extends Serializable {
11 |
12 | def toArray(value: A): Array[Byte]
13 |
14 | def fromArray(bytes: Array[Byte]): A
15 | }
16 |
17 | object Serializer {
18 |
19 | def fromArray[A](input: Array[Byte])(implicit reader: Serializer[A]): A = reader.fromArray(input)
20 |
21 | def toArray[A](value: A)(implicit writer: Serializer[A]): Array[Byte] = writer.toArray(value)
22 |
23 | def fromJavaSerialization[A] = new Serializer[A] {
24 | override def toArray(value: A): Array[Byte] = {
25 | val baos = new ByteArrayOutputStream()
26 | val oos = new ObjectOutputStream(baos)
27 | oos.writeObject(value)
28 | oos.close()
29 | baos.toByteArray
30 | }
31 | override def fromArray(bytes: Array[Byte]): A = {
32 | val bais = new ByteArrayInputStream(bytes)
33 | val ois = new ObjectInputStream(bais)
34 | ois.readObject().asInstanceOf[A]
35 | }
36 | }
37 | }
38 |
--------------------------------------------------------------------------------
/core/src/main/scala/com/monovore/coast/wire/StreamFormat.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.coast.wire
2 |
3 | import java.io.{DataInputStream, DataOutputStream}
4 |
5 | trait StreamFormat[A] extends Serializable {
6 | def write(output: DataOutputStream, value: A): Unit
7 | def read(input: DataInputStream): A
8 | }
9 |
10 | object StreamFormat {
11 |
12 | def read[A](input: DataInputStream)(implicit reader: StreamFormat[A]): A = reader.read(input)
13 |
14 | def write[A](output: DataOutputStream, value: A)(implicit writer: StreamFormat[A]) = writer.write(output, value)
15 |
16 | def fromSerializer[A](serializer: Serializer[A]) = new StreamFormat[A] {
17 | override def write(output: DataOutputStream, value: A): Unit = {
18 | val array = serializer.toArray(value)
19 | output.writeInt(array.length)
20 | output.write(array)
21 | }
22 | override def read(input: DataInputStream): A = {
23 | val size = input.readInt()
24 | val array = Array.ofDim[Byte](size)
25 | input.readFully(array)
26 | serializer.fromArray(array)
27 | }
28 | }
29 |
30 | implicit object longFormat extends StreamFormat[Long] {
31 | override def write(output: DataOutputStream, value: Long): Unit = output.writeLong(value)
32 | override def read(input: DataInputStream): Long = input.readLong()
33 | }
34 |
35 | implicit object intFormat extends StreamFormat[Int] {
36 | override def write(output: DataOutputStream, value: Int): Unit = output.writeInt(value)
37 | override def read(input: DataInputStream): Int = input.readInt()
38 | }
39 |
40 | implicit object stringFormat extends StreamFormat[String] {
41 | override def write(output: DataOutputStream, value: String): Unit = output.writeUTF(value)
42 | override def read(input: DataInputStream): String = input.readUTF()
43 | }
44 |
45 | implicit object unitFormat extends StreamFormat[Unit] {
46 | override def write(output: DataOutputStream, value: Unit): Unit = {}
47 | override def read(input: DataInputStream): Unit = {}
48 | }
49 |
50 | implicit val bytesFormat: StreamFormat[Array[Byte]] = new StreamFormat[Array[Byte]] {
51 |
52 | override def write(output: DataOutputStream, value: Array[Byte]): Unit = {
53 | output.writeInt(value.length)
54 | output.write(value)
55 | }
56 |
57 | override def read(input: DataInputStream): Array[Byte] = {
58 | val size = input.readInt()
59 | val bytes = Array.ofDim[Byte](size)
60 | input.readFully(bytes)
61 | bytes
62 | }
63 | }
64 |
65 | implicit def optionFormat[A](implicit format: StreamFormat[A]): StreamFormat[Option[A]] = new StreamFormat[Option[A]] {
66 |
67 | override def write(output: DataOutputStream, value: Option[A]): Unit = value match{
68 | case Some(a) => {
69 | output.writeBoolean(true)
70 | format.write(output, a)
71 | }
72 | case None => output.writeBoolean(false)
73 | }
74 |
75 | override def read(input: DataInputStream): Option[A] = {
76 | if (input.readBoolean()) Some(format.read(input))
77 | else None
78 | }
79 | }
80 |
81 | implicit def tuple2Format[A : StreamFormat, B : StreamFormat] = new StreamFormat[(A, B)] {
82 |
83 | override def write(output: DataOutputStream, value: (A, B)): Unit = {
84 | StreamFormat.write(output, value._1)
85 | StreamFormat.write(output, value._2)
86 | }
87 |
88 | override def read(input: DataInputStream): (A, B) = {
89 | val a = StreamFormat.read[A](input)
90 | val b = StreamFormat.read[B](input)
91 | (a, b)
92 | }
93 | }
94 |
95 | implicit def tuple3Format[A : StreamFormat, B : StreamFormat, C : StreamFormat] = new StreamFormat[(A, B, C)] {
96 |
97 | override def write(output: DataOutputStream, value: (A, B, C)): Unit = {
98 | StreamFormat.write(output, value._1)
99 | StreamFormat.write(output, value._2)
100 | StreamFormat.write(output, value._3)
101 | }
102 |
103 | override def read(input: DataInputStream): (A, B, C) = {
104 | val a = StreamFormat.read[A](input)
105 | val b = StreamFormat.read[B](input)
106 | val c = StreamFormat.read[C](input)
107 | (a, b, c)
108 | }
109 | }
110 |
111 | implicit def seqFormat[A : StreamFormat] = new StreamFormat[Seq[A]] {
112 |
113 | override def write(output: DataOutputStream, value: Seq[A]): Unit = {
114 | output.writeInt(value.size)
115 | value.foreach(StreamFormat.write(output, _))
116 | }
117 |
118 | override def read(input: DataInputStream): Seq[A] = {
119 | val size = input.readInt()
120 | Seq.fill(size)(StreamFormat.read[A](input))
121 | }
122 | }
123 |
124 | implicit def mapFormat[A : StreamFormat, B : StreamFormat] = new StreamFormat[Map[A, B]] {
125 |
126 | override def write(output: DataOutputStream, value: Map[A, B]): Unit = {
127 | output.writeInt(value.size)
128 | value.foreach { case (k, v) =>
129 | StreamFormat.write(output, k)
130 | StreamFormat.write(output, v)
131 | }
132 | }
133 |
134 | override def read(input: DataInputStream): Map[A, B] = {
135 | val size = input.readInt()
136 | Iterator.fill(size)(StreamFormat.read[A](input) -> StreamFormat.read[B](input))
137 | .toMap
138 | }
139 | }
140 | }
--------------------------------------------------------------------------------
/core/src/main/scala/com/monovore/example/coast/ConnectedComponents.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.example.coast
2 |
3 | import com.monovore.coast.core.Process
4 | import com.monovore.coast.flow._
5 | import com.monovore.coast.wire.{Protocol, Serializer}
6 |
7 | import scala.collection.immutable.SortedSet
8 |
9 | /**
10 | * An implementation of connected components: given a stream of new edges, we
11 | * incrementally maintain a mapping of node id to component id -- where the id
12 | * for the component is the smallest id of any node in that component.
13 | *
14 | * This cribs heavily off the MR-based implementation presented here:
15 | *
16 | * http://mmds-data.org/presentations/2014_/vassilvitskii_mmds14.pdf
17 | */
18 | object ConnectedComponents extends ExampleMain {
19 |
20 | import Protocol.common._
21 |
22 | type NodeID = Long
23 |
24 | implicit val eventFormat = Serializer.fromJavaSerialization[SortedSet[NodeID]]
25 |
26 | val Edges = Topic[Long, Long]("edges")
27 |
28 | val Components = Topic[Long, Long]("components")
29 |
30 | def connect(a: NodeID, b: NodeID) = Seq(a -> b, b -> a)
31 |
32 | implicit val graph = Flow.builder()
33 |
34 | val connected =
35 | Edges.asSource
36 | .zipWithKey
37 | .flatMap { case (one, other) => connect(one, other) }
38 | .groupByKey
39 | .streamTo("connected-input")
40 |
41 | val largeStar = graph.addCycle[NodeID, NodeID]("large-star") { largeStar =>
42 |
43 | val smallStar =
44 | Flow.merge("large" -> largeStar, "input" -> connected)
45 | .withKeys.process(SortedSet.empty[NodeID]) { node =>
46 |
47 | Process.on { (neighbours, newEdge) =>
48 |
49 | val all = (neighbours + node)
50 | val least = all.min
51 |
52 | if (node < newEdge || all.contains(newEdge)) Process.skip
53 | else {
54 | Process.setState(SortedSet(newEdge)) andThen {
55 | if (least < newEdge) Process.outputEach(connect(newEdge, least))
56 | else Process.outputEach(all.toSeq.flatMap(connect(_, newEdge)))
57 | }
58 | }
59 | }
60 | }
61 | .groupByKey
62 | .streamTo("small-star")
63 |
64 | smallStar
65 | .withKeys.process(SortedSet.empty[NodeID]) { node =>
66 |
67 | Process.on { (neighbours, newEdge) =>
68 | val all = neighbours + node
69 | val least = all.min
70 |
71 | Process.setState(neighbours + newEdge) andThen {
72 |
73 | if (newEdge < least) {
74 | val larger = neighbours.toSeq.filter {_ > node}
75 | Process.outputEach(larger.flatMap {connect(_, newEdge)})
76 | }
77 | else if (newEdge < node || all.contains(newEdge)) Process.skip
78 | else Process.outputEach(connect(newEdge, least))
79 | }
80 | }
81 | }
82 | .groupByKey
83 | }
84 |
85 | largeStar
86 | .withKeys.transform(Long.MaxValue) { node => (currentOrMax, next) =>
87 | val current = currentOrMax min node
88 | val min = current min next
89 | if (min < current) min -> Seq(min)
90 | else current -> Nil
91 | }
92 | .sinkTo(Components)
93 | }
94 |
--------------------------------------------------------------------------------
/core/src/main/scala/com/monovore/example/coast/CustomerTransactions.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.example.coast
2 |
3 | import com.monovore.coast
4 | import coast.flow
5 | import com.monovore.coast.flow.{Flow, Topic}
6 | import com.monovore.coast.wire.Protocol
7 |
8 | /**
9 | * Based on the discussion in this thread:
10 | *
11 | * http://mail-archives.apache.org/mod_mbox/incubator-samza-dev/201411.mbox/%3CCAFhxiSQ4V3KTt2L4CcRVHrKDRi-oS26LGCGvhSemKVPH-SW_RA@mail.gmail.com%3E
12 | */
13 | object CustomerTransactions extends ExampleMain {
14 |
15 | import Protocol.native._
16 |
17 | type CustomerID = String
18 | type TransactionID = String
19 |
20 | case class Customer()
21 | case class Transaction()
22 |
23 | val Customers = Topic[CustomerID, Customer]("customers")
24 | val CustomerTransactions = Topic[TransactionID, CustomerID]("customer-transactions")
25 | val Transactions = Topic[TransactionID, Transaction]("transactions")
26 |
27 | val CustomerInfo = Topic[CustomerID, (Customer, Seq[Transaction])]("customer-info")
28 |
29 | override def graph: Flow[Unit] = for {
30 |
31 | transactionsByCustomer <- Flow.stream("transactions-by-customer") {
32 |
33 | (Flow.source(Transactions).latestOption join Flow.source(CustomerTransactions).latestOption)
34 | .updates
35 | .flatMap { case (latestTransaction, allCustomers) =>
36 |
37 | val both = for {
38 | transaction <- latestTransaction.toSeq
39 | customer <- allCustomers
40 | } yield customer -> transaction
41 |
42 | both.toSeq
43 | }
44 | .groupByKey
45 | }
46 |
47 | _ <- Flow.sink(CustomerInfo) {
48 |
49 | val allCustomerTransactions = transactionsByCustomer.fold(Seq.empty[Transaction]) { _ :+ _ }
50 |
51 | val latestCustomerInfo = Flow.source(Customers).latestOption
52 |
53 | (latestCustomerInfo join allCustomerTransactions)
54 | .updates
55 | .flatMap { case (customerOption, transactions) =>
56 |
57 | customerOption
58 | .map { _ -> transactions }
59 | .toSeq
60 | }
61 | }
62 | } yield ()
63 | }
64 |
--------------------------------------------------------------------------------
/core/src/main/scala/com/monovore/example/coast/Denormalize.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.example.coast
2 |
3 | import com.monovore.coast.flow.{Flow, Topic}
4 | import com.monovore.coast.wire.Protocol
5 |
6 | import scala.collection.immutable.SortedSet
7 |
8 | /**
9 | * A sketch of a denormalization flow -- normalized models come in at the top,
10 | * and denormalized versions appear at the bottom.
11 | */
12 | object Denormalize extends ExampleMain {
13 |
14 | import Protocol.native._
15 |
16 | case class ID(value: Long)
17 | type GroupID = ID
18 | type UserID = ID
19 |
20 | implicit val IDOrdering = Ordering.by { id: ID => id.value }
21 |
22 | case class Group(name: String)
23 | case class User(name: String, groupIDs: SortedSet[GroupID])
24 | case class DenormalizedGroup(name: String, memberNames: Set[String])
25 |
26 | // 'Changelog' for users and groups
27 | // We expect None when the data is missing or deleted, and Some(user) otherwise
28 | val Users = Topic[UserID, Option[User]]("users")
29 | val Groups = Topic[GroupID, Option[Group]]("groups")
30 |
31 | val Denormalized = Topic[GroupID, Option[DenormalizedGroup]]("denormalized-groups")
32 |
33 | val graph = Flow.build { implicit builder =>
34 |
35 | val usersPool =
36 | Flow.source(Users)
37 | .map { userOpt =>
38 | userOpt
39 | .map { user =>
40 | user.groupIDs.map { _ -> user.name }.toMap
41 | }
42 | .getOrElse(Map.empty)
43 | }
44 | .latestByKey("users-changes")
45 |
46 | val groups = Flow.source(Groups).latestOr(None)
47 |
48 | (groups join usersPool)
49 | .map { case (groupOption, members) =>
50 | for (group <- groupOption) yield {
51 | DenormalizedGroup(group.name, members.values.toSet)
52 | }
53 | }
54 | .updates
55 | .sinkTo(Denormalized)
56 | }
57 | }
58 |
--------------------------------------------------------------------------------
/core/src/main/scala/com/monovore/example/coast/EntityResolution.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.example.coast
2 |
3 | import com.monovore.coast.flow.{AnyStream, Flow, Topic}
4 | import com.monovore.coast.wire.Protocol
5 |
6 | import scala.annotation.tailrec
7 |
8 | /**
9 | * Based on the D-Swoosh algorithm by Stanford's Entity Resolution group.
10 | */
11 | object EntityResolution extends ExampleMain {
12 |
13 | import Protocol.native._
14 |
15 | // Basic entity resolution / 'swoosh' setup
16 |
17 | case class Name(name: String)
18 |
19 | case class Category(categoryName: String)
20 |
21 | implicit val categoryOrdering: Ordering[Category] = Ordering.by { _.categoryName }
22 |
23 | case class Product(names: Set[Name], minPrice: Int, categories: Set[Category])
24 |
25 | def matches(one: Product, other: Product): Boolean = {
26 | (one.names intersect other.names).nonEmpty &&
27 | (one.categories intersect other.categories).nonEmpty
28 | }
29 |
30 | def merge(one: Product, other: Product): Product = Product(
31 | names = one.names ++ other.names,
32 | minPrice = math.min(one.minPrice, other.minPrice),
33 | categories = one.categories ++ other.categories
34 | )
35 |
36 | def scope(product: Product): Seq[Category] =
37 | product.categories.toSeq.sorted
38 |
39 | @tailrec
40 | def mergeAll(set: Set[Product], next: Product): (Set[Product], Option[Product]) = {
41 |
42 | set.find(matches(_, next)) match {
43 | case Some(found) => {
44 | val merged = merge(found, next)
45 |
46 | if (merged == found) set -> None
47 | else mergeAll(set - found, merged)
48 | }
49 | case None => (set + next) -> Some(next)
50 | }
51 | }
52 |
53 | // D-Swoosh job definition
54 |
55 | // New product info, arbitrarily partitioned
56 | val RawProducts = Topic[Int, Product]("raw-products")
57 |
58 | // For each category, a stream of all products in that category
59 | val AllProducts = Topic[Category, Product]("all-products")
60 |
61 | val graph = for {
62 |
63 | // This stream holds the results of the merge within each category
64 | // Since a merge in one category could trigger a merge in another,
65 | // we need to pass the output back in as input, so this definition is cyclic.
66 | // To help avoid redundant work, the Boolean tracks whether or not the product
67 | // is the result of a merge.
68 | allProducts <- Flow.cycle[Category, (Product, Boolean)]("all-products-merged") { allProducts =>
69 |
70 | for {
71 |
72 | // This stream takes both the raw products and the merge output,
73 | // broadcasting them to all the categories in scope
74 | scoped <- Flow.stream("scoped-products") {
75 |
76 | def groupByScope[A](stream: AnyStream[A, Product]) =
77 | stream
78 | .flatMap { e => scope(e).map { _ -> e } }
79 | .groupByKey
80 |
81 | Flow.merge(
82 | "all" -> groupByScope(Flow.source(RawProducts)),
83 | "raw" -> groupByScope(allProducts.collect { case (product, true) => product })
84 | )
85 | }
86 |
87 | } yield {
88 |
89 | scoped
90 | .transform(Set.empty[Product]) { (set, next) =>
91 |
92 | val (newSet, output) = mergeAll(set, next)
93 |
94 | newSet -> {
95 | output
96 | .map { product => product -> (product != next) }
97 | .toSeq
98 | }
99 | }
100 | }
101 | }
102 |
103 | // This just copies the output of the merge to the AllProducts stream
104 | _ <- Flow.sink(AllProducts) {
105 | allProducts.map { case (product, _) => product }
106 | }
107 | } yield ()
108 | }
109 |
--------------------------------------------------------------------------------
/core/src/main/scala/com/monovore/example/coast/ExampleMain.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.example.coast
2 |
3 | import java.io.File
4 |
5 | import com.google.common.base.Charsets
6 | import com.google.common.io.Files
7 | import com.monovore.coast.viz.Dot
8 | import com.monovore.coast.core.Graph
9 |
10 | /**
11 | * A simple main method for running the example jobs. At the moment, it just
12 | * pretty-prints the flows in GraphViz format.
13 | */
14 | trait ExampleMain {
15 |
16 | def graph: Graph
17 |
18 | def main(args: Array[String]): Unit = {
19 |
20 | args.toList match {
21 | case List("dot") => println(Dot.describe(graph))
22 | case List("dot", path) => {
23 | Files.asCharSink(new File(path), Charsets.UTF_8).write(Dot.describe(graph))
24 | }
25 | }
26 | }
27 | }
28 |
--------------------------------------------------------------------------------
/core/src/main/scala/com/monovore/example/coast/LinearRoad.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.example.coast
2 |
3 | import com.monovore.coast.flow.{Flow, Topic}
4 | import com.monovore.coast.wire.Protocol
5 | import com.twitter.algebird.{AveragedValue, Monoid}
6 |
7 | /**
8 | * Starting to stub out the standard linear road example, based on the
9 | * description here:
10 | *
11 | * https://cs.uwaterloo.ca/~david/cs848/stream-cql.pdf
12 | *
13 | * I'm not convinced this iss particularly faithful; the non-overlapping time
14 | * windows and road 'segments' seem particularly fishy. It would be better to
15 | * work from the official spec.
16 | */
17 | object LinearRoad extends ExampleMain {
18 |
19 | import Protocol.native._
20 |
21 | type VehicleID = Long
22 |
23 | // To simplify the calculation, let's discretize by space and time
24 | type Segment = Int
25 | type TimeWindow = Long
26 |
27 | val BaseToll = 1.0 // ???
28 |
29 | case class PlaceAndTime(segment: Segment, time: TimeWindow)
30 | case class Summary(vehicles: Set[VehicleID] = Set.empty, averageSpeed: AveragedValue = Monoid.zero[AveragedValue])
31 |
32 | implicit val SummaryMonoid = Monoid(Summary.apply _, Summary.unapply _)
33 |
34 | val PositionReports = Topic[VehicleID, (PlaceAndTime, Double)]("position-reports")
35 | val TotalTolls = Topic[VehicleID, Double]("total-tolls")
36 |
37 | val graph = Flow.build { implicit builder =>
38 |
39 | PositionReports.asSource
40 | .invert
41 | .streamTo("reports-by-position")
42 | .map { case (vehicle, speed) => Summary(Set(vehicle), AveragedValue(speed)) }
43 | .sum
44 | .updates
45 | .map { summary =>
46 | if (summary.averageSpeed.value < 40) {
47 | val toll = BaseToll * math.pow(summary.vehicles.size - 150, 2)
48 | summary.vehicles.map { _ -> toll }.toMap
49 | } else Map.empty[VehicleID, Double]
50 | }
51 | .sumByKey("summaries")
52 | .updates
53 | .sinkTo(TotalTolls)
54 | }
55 | }
56 |
--------------------------------------------------------------------------------
/core/src/main/scala/com/monovore/example/coast/Scheduler.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.example.coast
2 |
3 | import com.monovore.coast.core.Graph
4 | import com.monovore.coast.flow.{Topic, Flow}
5 | import com.monovore.coast.wire.Protocol
6 |
7 | object Scheduler extends ExampleMain {
8 |
9 | import Protocol.native._
10 |
11 | val Requests = Topic[Long, String]("requests")
12 |
13 | val Triggered = Topic[Long, Set[String]]("triggered")
14 |
15 | override def graph: Graph = for {
16 |
17 | ticks <- Flow.stream("whatever") {
18 |
19 | Flow.clock(seconds = 60)
20 | .map { _ -> "tick!" }
21 | .groupByKey
22 | }
23 |
24 | _ <- Flow.sink(Triggered) {
25 |
26 | val allEvents = Requests.asSource.map(Set(_)).sum
27 |
28 | (ticks join allEvents)
29 | .map { case (_, events) => events }
30 | }
31 |
32 | } yield ()
33 | }
34 |
--------------------------------------------------------------------------------
/core/src/main/scala/com/monovore/example/coast/TwitterReach.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.example.coast
2 |
3 | import java.net.URI
4 |
5 | import com.monovore.coast.flow.{Flow, Topic}
6 | import com.monovore.coast.wire.Protocol
7 |
8 | import scala.util.Try
9 |
10 | /* This streaming job is loosely based on Trident's 'Twitter reach' example
11 | * here:
12 | *
13 | * https://storm.apache.org/documentation/Trident-tutorial.html#toc_2
14 | *
15 | * Both jobs calculate the 'reach' of a URI on Twitter, which is the total
16 | * number of Twitter users that have seen a particular link, by joining a user's
17 | * links against their set of followers. However, there are a few major
18 | * differences between the two:
19 | *
20 | * - The Trident job is RPC-style: it only calculates the reach for a particular
21 | * tweet when requested by a user. This example job continuously maintains the
22 | * reach count for all links.
23 | *
24 | * - The Trident job relies on external tables for the id -> followers and link
25 | * -> tweeted_by joins. The job here maintains that state itself, by subscribing
26 | * to the stream of changes.
27 | */
28 |
29 | object TwitterReach extends ExampleMain {
30 |
31 | /* We'll start this file with some top-level definitions. If you have multiple
32 | * `coast` jobs in the same codebase that share data formats or streams, you'd
33 | * probably want to pull these out to a common file so all the jobs can access
34 | * them.
35 | *
36 | * First, we'll define a few `Topic`s. A `Topic` corresponds closely to a topic
37 | * in Kafka; it groups together the topic name and the types of the
38 | * partition keys and messages.
39 | *
40 | * Both our inputs are partitioned by user ID. Every time the user sends a
41 | * tweet, we get the tweet's text on the `tweets` input stream. Every time a
42 | * user gets a new follower, we get the follower's ID on the `followers`
43 | * stream. `reach` is the job's output stream. Every time a user tweets a
44 | * link, our job will write out the URI with the updated count on this stream.
45 | *
46 | * I'll be lazy here and use Strings for IDs. You can probably think of
47 | * something more appropriate.
48 | */
49 |
50 | type UserID = String
51 | type FollowerID = String
52 |
53 | val Tweets = Topic[UserID, String]("tweets")
54 | val Followers = Topic[UserID, FollowerID]("followers")
55 |
56 | val Reach = Topic[URI, Int]("reach")
57 |
58 | /* `coast` uses implicit parameters to decide how to partition and serialize
59 | * your data. Don't be frightened! It's both safer and less verbose than using
60 | * configuration or a global registry, and the error messages are better.
61 | *
62 | * For now, I'm importing `Protocol.native._`, which uses `Object.hashCode`
63 | * for partitioning and java serialization on the wire. I suggest not doing
64 | * this in production, but it's handy for experimenting.
65 | */
66 |
67 | import Protocol.native._
68 |
69 | /* Now we come to the job logic. We're building up a `Flow` object here; this
70 | * defines the stream-processing graph and associates topic names with the
71 | * actual processing logic. The `for` block here may seem a bit odd, but it
72 | * makes it possible to track the stream names and types without cluttering
73 | * the job itself.
74 | *
75 | * This job is split into two stages. The first gathers up the followers that
76 | * saw a particular link in a particular tweet; the second accumulates all the
77 | * followers that have ever seen a link, writing the total count to the output
78 | * stream.
79 | */
80 |
81 | val graph = for {
82 |
83 | /* This next line defines an internal stream. The string in the middle gives
84 | * a name to the stream -- in the Samza backend, for example, this
85 | * is the name of both the output Kafka topic and the Samza job that
86 | * produces it. On the left, we bind the output to a variable, so we can use
87 | * it as an input to other stages. On the right, we're opening a block: this
88 | * holds the 'definition' of the stream.
89 | */
90 |
91 | followersByURI <- Flow.stream("followers-by-uri") {
92 |
93 | /* This little definition has two parts. The first,
94 | * `Flow.source(Followers)`, subscribes us to the `followers` stream we
95 | * defined above. This gives us a `GroupedStream[UserID, FollowerID]` --
96 | * it's an ongoing stream of events, and the types match up with the types
97 | * we defined for `Followers` above.
98 | *
99 | * The second part has the form `.fold(initial)(update function)`. Here, it
100 | * accumulates all the followers for a given user into a single set: the
101 | * return type here is `GroupedPool[UserID, Set[FollowerID]]`. A pool is like a
102 | * stream with a current value for each key; or, if you prefer, like a
103 | * table with a changelog stream. In this case, calculating the current
104 | * value requires keeping state; `coast` will take care of this,
105 | * serializing it when necessary using the formatters we defined above.
106 | *
107 | * Streams and pools are `coast`'s two main abstractions for streaming data;
108 | * all transformations and joins we do below result in one of these two
109 | * types.
110 | */
111 |
112 | val followersByUser =
113 | Flow.source(Followers)
114 | .fold(Set.empty[FollowerID]) { _ + _ }
115 |
116 | /* This defines a stream that extracts URIs from tweets. If a tweet
117 | * contains multiple URIs, we'll get multiple events in the stream.
118 | *
119 | * There's not much to say here, except to note the similarity between
120 | * this and the usual Scala collections API, and the poor quality of my
121 | * URI validation code.
122 | */
123 |
124 | val tweetedLinks =
125 | Flow.source(Tweets)
126 | .flatMap { _.split("\\s+") }
127 | .filter { _.startsWith("http") }
128 | .map { maybeURI => Try(new URI(maybeURI)).toOption }
129 | .flattenOption
130 |
131 | /* After that, we join the two together to make a new stream, then regroup
132 | * by key. Each of these steps could probably use a little more explanation.
133 | *
134 | * Recall that tweetedLinks is a stream, and followersByUser is a pool.
135 | * When we join the two, it returns the type `GroupedStream[UserID, (URI,
136 | * Set[FollowerID])]`. It's handy to think of the 'stream-pool join' as a
137 | * 'lookup' operation: every time there's a new event, we look up the
138 | * current state for the matching key and pair the event and state
139 | * together. In our case, we have both the link and all of the user's
140 | * followers at the time -- which is exactly what we needed.
141 | *
142 | * To get the total reach, though, we'll need to get the set of all
143 | * followers together on a single node -- but our data is still grouped
144 | * by user ID. When running on a cluster, we'll have to shuffle data
145 | * across the network to get the proper grouping before we can accumulate.
146 | * That's why this job is split in two: this first part writes the data
147 | * grouped by link URI, so the second part can get all the followers that
148 | * have ever seen a particular link in the same partition.
149 | *
150 | * We'll use the `.groupByKey` convenience method, which takes a stream of
151 | * key/value pairs and makes a new stream where the values are grouped
152 | * under the keys.
153 | */
154 |
155 | (tweetedLinks join followersByUser).groupByKey
156 | }
157 |
158 | /* In the second part of the job, we're calculating the final counts and
159 | * writing them to the output stream. Since this is our actual output, and
160 | * not just an implementation detail, we write it to the named output stream
161 | * we defined above. Since this stream may have any number of consumers,
162 | * perhaps using other languages or frameworks, `coast` is careful to keep
163 | * public outputs like this free of duplicates or metadata.
164 | */
165 |
166 | _ <- Flow.sink(Reach) {
167 |
168 | /* This last bit's pretty minimal. We use another fold to accumulate all
169 | * the followers in one big set. Every time we see a new follower, we
170 | * recalculate the size and write it to our output stream.
171 | */
172 |
173 | followersByURI
174 | .fold(Set.empty[FollowerID]) { _ ++ _ }
175 | .map { _.size }
176 | .updates
177 | }
178 |
179 | } yield ()
180 |
181 | /* And that's it! Looking back, the job logic seems dwarfed by the wall of
182 | * text; hope everything made it through okay.
183 | *
184 | * If you have this code checked out, you can run this code with the `dot`
185 | * argument to print the job in graphviz format; you'll see the two main
186 | * stages, plus the structure inside each stage that defines the dataflow.
187 | * Otherwise, if you create the Kafka topics with the right names,
188 | * this code is ready to run on the cluster.
189 | */
190 |
191 | }
192 |
--------------------------------------------------------------------------------
/core/src/main/scala/com/monovore/example/coast/WordCount.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.example.coast
2 |
3 | import com.monovore.coast.flow._
4 | import com.monovore.coast.wire.Protocol
5 |
6 | object WordCount extends ExampleMain {
7 |
8 | import Protocol.simple._
9 |
10 | type Source = Long
11 |
12 | val Sentences = Topic[Source, String]("sentences")
13 |
14 | val WordCounts = Topic[String, Int]("word-counts")
15 |
16 | val graph = Flow.build { implicit builder =>
17 |
18 | Sentences.asSource
19 | .flatMap { _.split("\\s+") }
20 | .map { _ -> 1 }
21 | .groupByKey
22 | .streamTo("words")
23 | .sum.updates
24 | .sinkTo(WordCounts)
25 | }
26 | }
27 |
--------------------------------------------------------------------------------
/core/src/test/scala/com/monovore/coast/machine/MachineSpec.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.coast
2 | package machine
3 |
4 | import com.monovore.coast
5 | import coast.flow
6 | import com.monovore.coast.flow.{Flow, Topic}
7 | import com.monovore.coast.core.Graph
8 | import com.monovore.coast.wire.Protocol
9 |
10 | import com.twitter.algebird.Semigroup
11 | import org.scalacheck.Prop
12 | import org.specs2.ScalaCheck
13 | import org.specs2.mutable._
14 |
15 | class MachineSpec extends Specification with ScalaCheck {
16 |
17 | // Tweak me
18 | val InputSize = 10
19 |
20 | import Protocol.common._
21 |
22 | "a compiled flow" should {
23 |
24 | val integers = Topic[String, Int]("integers")
25 |
26 | val output = Topic[String, Int]("output")
27 |
28 | "do a basic deterministic transformation" in {
29 |
30 | val doubled = Topic[String, Int]("doubled")
31 |
32 | val graph = Flow.stream("doubled") {
33 | Flow.source(integers).map { _ * 2 }
34 | }
35 |
36 | val input = Messages.from(integers, Map(
37 | "foo" -> Seq(1, 3), "bar" -> Seq(2)
38 | ))
39 |
40 | val compiled = Machine.compile(graph).push(input)
41 |
42 | Prop.forAll(Sample.complete(compiled)) { output =>
43 | output(doubled) must_== Map(
44 | "foo" -> Seq(2, 6),
45 | "bar" -> Seq(4)
46 | )
47 | }
48 | }
49 |
50 | "support all operations" in {
51 |
52 | "pool" in {
53 |
54 | val graph = Flow.sink(output) {
55 | Flow.source(integers).fold(0) { _ + _ }.updates
56 | }
57 |
58 | prop { input: Map[String, Seq[Int]] =>
59 |
60 | val expected = input
61 | .mapValues { _.scanLeft(0)(_ + _).tail }
62 | .filter { case (_ -> v) => v.nonEmpty }
63 |
64 | val compiled = Machine.compile(graph).push(Messages.from(integers, input))
65 |
66 | Prop.forAll(Sample.complete(compiled)) { messages =>
67 | messages(output) must_== expected
68 | }
69 | } set (maxSize = InputSize)
70 | }
71 |
72 | "merge" in {
73 |
74 | val integers2 = Topic[String, Int]("integers-2")
75 |
76 | val graph = Flow.sink(output) {
77 |
78 | Flow.merge(
79 | "ints" -> Flow.source(integers),
80 | "more" -> Flow.source(integers2)
81 | )
82 | }
83 |
84 | prop { (input: Map[String, Seq[Int]], input2: Map[String, Seq[Int]]) =>
85 |
86 | val expected = Semigroup.plus(input, input2)
87 | .filter { case (_ -> v) => v.nonEmpty }
88 | .mapValues { _.sorted }
89 |
90 | val compiled = Machine.compile(graph)
91 | .push(Messages.from(integers, input))
92 | .push(Messages.from(integers2, input2))
93 |
94 | Prop.forAll(Sample.complete(compiled)) { messages =>
95 | messages(output).mapValues { _.sorted } must_== expected
96 | }
97 | } set (maxSize = InputSize)
98 | }
99 |
100 | "groupBy" in {
101 |
102 | val graph = for {
103 |
104 | grouped <- Flow.stream("grouped") {
105 | Flow.source(integers).groupBy { n => (n % 2 == 0).toString}
106 | }
107 |
108 | _ <- Flow.sink(output) { grouped }
109 | } yield ()
110 |
111 | prop { input: Map[String, Seq[Int]] =>
112 |
113 | val expected = input.values.toSeq.flatten
114 | .groupBy { n => (n % 2 == 0).toString }
115 | .mapValues { _.sorted }
116 |
117 | val compiled = Machine.compile(graph)
118 | .push(Messages.from(integers, input))
119 |
120 | Prop.forAll(Sample.complete(compiled)) { messages =>
121 | messages(output).mapValues { _.sorted } must_== expected
122 | }
123 | } set (maxSize = InputSize)
124 | }
125 | }
126 |
127 | "obey some functor / monad-type laws" in {
128 |
129 | "x.map(identity) === x" in {
130 |
131 | val original = Flow.sink(output) { Flow.source(integers) }
132 | val mapped = Flow.sink(output) { Flow.source(integers).map(identity) }
133 |
134 | prop { (pairs: Map[String, Seq[Int]]) =>
135 |
136 | equivalent(Messages.from(integers, pairs), original, mapped)
137 |
138 | } set (maxSize = InputSize)
139 | }
140 |
141 | "x.map(f).map(g) === x.map(f andThen g)" in {
142 |
143 | val f: (Int => Int) = { _ * 2 }
144 | val g: (Int => Int) = { _ + 6 }
145 |
146 | val original = Flow.sink(output) { Flow.source(integers).map(f andThen g) }
147 | val mapped = Flow.sink(output) { Flow.source(integers).map(f).map(g) }
148 |
149 | prop { (pairs: Map[String, Seq[Int]]) =>
150 |
151 | equivalent(Messages.from(integers, pairs), original, mapped)
152 | } set (maxSize = InputSize)
153 | }
154 |
155 | "stream.flatMap(f).flatMap(g) === stream.flatMap(f andThen { _.flatMap(g) })" in {
156 |
157 | val f: (Int => Seq[Int]) = { x => Seq(x, x) }
158 | val g: (Int => Seq[Int]) = { x => Seq(x + 6) }
159 |
160 | val nested = Flow.sink(output) { Flow.source(integers).flatMap(f andThen { _.flatMap(g) }) }
161 | val chained = Flow.sink(output) { Flow.source(integers).flatMap(f).flatMap(g) }
162 |
163 | prop { (pairs: Map[String, Seq[Int]]) =>
164 |
165 | equivalent(Messages.from(integers, pairs), nested, chained)
166 | } set (maxSize = InputSize)
167 | }
168 |
169 | "stream.flatMap(lift) === stream" in {
170 |
171 | val noop = Flow.sink(output) { Flow.source(integers) }
172 | val mapped = Flow.sink(output) { Flow.source(integers).flatMap { x => List(x) } }
173 |
174 | prop { (pairs: Map[String, Seq[Int]]) =>
175 |
176 | equivalent(Messages.from(integers, pairs), noop, mapped)
177 | } set (maxSize = InputSize)
178 | }
179 |
180 | "pool.map(identity) === pool" in {
181 |
182 | val pool = Flow.source(integers).latestOr(0)
183 |
184 | val original = Flow.sink(output) { pool.updates }
185 | val mapped = Flow.sink(output) { pool.map(identity).updates }
186 |
187 | prop { (pairs: Map[String, Seq[Int]]) =>
188 |
189 | equivalent(Messages.from(integers, pairs), original, mapped)
190 |
191 | } set (maxSize = InputSize)
192 | }
193 |
194 | "pool.map(f).map(g) === pool.map(f andThen g)" in {
195 |
196 | val f: (Int => Int) = { _ * 2 }
197 | val g: (Int => Int) = { _ + 6 }
198 |
199 | val pool = Flow.source(integers).latestOr(0)
200 |
201 | val original = Flow.sink(output) { pool.map(f andThen g).updates }
202 | val mapped = Flow.sink(output) { pool.map(f).map(g).updates }
203 |
204 | prop { (pairs: Map[String, Seq[Int]]) =>
205 |
206 | equivalent(Messages.from(integers, pairs), original, mapped)
207 | } set (maxSize = InputSize)
208 | }
209 | }
210 | }
211 |
212 | def equivalent(messages: Messages, one: Graph, two: Graph): Prop = {
213 |
214 | prop { swap: Boolean =>
215 |
216 | val (sample, prove) = {
217 |
218 | val oneM = Machine.compile(one)
219 | val twoM = Machine.compile(two)
220 |
221 | if (swap) (twoM, oneM) else (oneM, twoM)
222 | }
223 |
224 | Prop.forAll(Sample.complete(sample.push(messages))) { output =>
225 | Sample.canProduce(prove.push(messages), output)
226 | }
227 | }
228 | }
229 | }
230 |
--------------------------------------------------------------------------------
/core/src/test/scala/com/monovore/coast/machine/Sample.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.coast
2 | package machine
3 |
4 | import org.scalacheck.Gen
5 |
6 | import scala.annotation.tailrec
7 | import scala.util.Random
8 |
9 | object Sample {
10 |
11 | def complete(machine: Machine): Gen[Messages] = {
12 |
13 | @tailrec def doComplete(machine: Machine, messages: Messages): Messages = {
14 |
15 | val next = Random.shuffle(machine.next).headOption
16 |
17 | next match {
18 | case Some(shit) => {
19 | val (newMachine, newMessages) = shit()
20 | doComplete(newMachine, messages ++ newMessages)
21 | }
22 | case None => messages
23 | }
24 | }
25 |
26 | Gen.wrap { doComplete(machine, Messages.empty) }
27 | }
28 |
29 | def canProduce(machine: Machine, output: Messages): Boolean = {
30 |
31 | def dropSeq[A](prefix: Seq[A], other: Seq[A]): Option[Seq[A]] = {
32 | val (left, right) = other.splitAt(prefix.length)
33 | assuming (left == prefix) { right }
34 | }
35 |
36 | def dropMap[A, B](dropValuePrefix: (B, B) => Option[B])(prefix: Map[A, B], other: Map[A, B]): Option[Map[A, B]] = {
37 |
38 | prefix.foldLeft(Some(other): Option[Map[A, B]]) { (acc, kv) =>
39 |
40 | val (key, valuePrefix) = kv
41 |
42 | for {
43 | accMap <- acc
44 | value <- accMap.get(key)
45 | dropped <- dropValuePrefix(valuePrefix, value)
46 | } yield accMap.updated(key, dropped)
47 | }
48 | }
49 |
50 |
51 | val next = machine.next
52 |
53 | val result =
54 | if (next.isEmpty) output.isEmpty
55 | else {
56 | next.exists { case nextFn =>
57 |
58 | val (machin, soFar) = nextFn()
59 |
60 | val dropped = dropMap(dropMap[Key, Seq[Message]](dropSeq[Message]))(soFar.messageMap, output.messageMap)
61 |
62 | dropped.exists { rest =>
63 | canProduce(machin, Messages(rest))
64 | }
65 | }
66 | }
67 |
68 | result
69 | }
70 | }
71 |
--------------------------------------------------------------------------------
/core/src/test/scala/com/monovore/coast/machine/SystemSpec.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.coast
2 | package machine
3 |
4 | import org.specs2.mutable._
5 |
6 | class SystemSpec extends Specification {
7 |
8 | "an actor system" should {
9 |
10 | "send a message" in {
11 |
12 | val machine = System[String](
13 | edges = Map("source" -> Seq("target"))
14 | )
15 |
16 | val (state, _) = machine.update(
17 | System.State(),
18 | System.Send("source", Key(0), Message("message"))
19 | )
20 |
21 | state.stateMap("target")(Key(0)).input must_== Map("source" -> Seq(Message("message")))
22 | }
23 |
24 | "process a message" in {
25 |
26 | val machine = System[String]()
27 |
28 | val state = System.State(Map("target" -> Map(Key(0) -> Actor.Data(
29 | state = State(unit),
30 | input = Map("source" -> Seq(Message("payload")))
31 | ))))
32 |
33 | machine.commands(state) must haveSize(1)
34 |
35 | val (updated, output) = machine.update(state, machine.commands(state).head)
36 |
37 | output("target") must_== Map(Key(0) -> Seq(Message("payload")))
38 |
39 | updated.stateMap("target")(Key(0)).input("source") must beEmpty
40 | }
41 | }
42 | }
43 |
--------------------------------------------------------------------------------
/core/src/test/scala/com/monovore/example/coast/ConnectedComponentsSpec.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.example.coast
2 |
3 | import com.monovore.coast.flow.{Flow, Topic}
4 | import com.monovore.coast.machine.{Machine, Messages, Sample}
5 | import org.scalacheck.{Gen, Prop}
6 | import org.specs2.ScalaCheck
7 | import org.specs2.matcher.Parameters
8 | import org.specs2.mutable._
9 |
10 | class ConnectedComponentsSpec extends Specification with ScalaCheck {
11 |
12 | implicit val scalacheck = Parameters(maxSize = 20)
13 |
14 | "connected-components finder" should {
15 |
16 | import ConnectedComponents._
17 |
18 | "find correct label for linear graph" in {
19 |
20 | val input = Messages.from(Edges, Map(
21 | 1L -> Seq(0L), 2L -> Seq(1L), 3L -> Seq(2L)
22 | ))
23 |
24 | val testCase = Machine.compile(graph).push(input)
25 |
26 | Prop.forAll(Sample.complete(testCase)) { output =>
27 |
28 | output(Components)(2).last must_== 0
29 | output(Components)(1).last must_== 0
30 | }
31 | }
32 |
33 | "label a small random graph correctly" in {
34 |
35 | val gen = for {
36 | x <- Gen.choose(0L, 9L)
37 | n <- Gen.choose(1, 5)
38 | ys <- Gen.listOfN(n, Gen.choose(0L, 9L))
39 | } yield (x -> ys)
40 |
41 | Prop.forAll(Gen.mapOf(gen)) { inputs =>
42 |
43 | val input = Messages.from(Edges, inputs)
44 | val testCase = Machine.compile(graph).push(input)
45 |
46 | Prop.forAll(Sample.complete(testCase)) { output =>
47 |
48 | def label(id: Long) =
49 | output(Components).get(id)
50 | .flatMap { _.lastOption }
51 | .getOrElse(id)
52 |
53 | // if two nodes were connected, they should get the same label
54 | foreach(inputs) { case (source, targets) =>
55 | foreach(targets) { target =>
56 | label(source) must_== label(target)
57 | }
58 | }
59 |
60 | foreach(output(Components)) { case (source, labels) =>
61 | labels.reverse must beSorted
62 | }
63 | }
64 | }
65 | }
66 | }
67 | }
68 |
--------------------------------------------------------------------------------
/core/src/test/scala/com/monovore/example/coast/DenormalizeSpec.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.example.coast
2 |
3 | import com.monovore.coast.machine.{Sample, Messages, Machine}
4 | import org.scalacheck.{Prop, Arbitrary, Gen}
5 | import org.specs2.ScalaCheck
6 | import org.specs2.matcher.Parameters
7 | import org.specs2.mutable._
8 | import org.scalacheck.Arbitrary.arbitrary
9 |
10 | import scala.collection.immutable.SortedSet
11 |
12 | class DenormalizeSpec extends Specification with ScalaCheck {
13 |
14 | import Denormalize._
15 |
16 | // we have a lot of nested collections, so let's keep things reasonable here
17 | implicit val scalacheck = Parameters(maxSize = 15)
18 |
19 | "Denormalize example" should {
20 |
21 | implicit val idGen = Arbitrary {
22 | Gen.choose(0L, 32L).map(Denormalize.ID)
23 | }
24 |
25 | implicit val usersGen = Arbitrary {
26 | for {
27 | name <- Gen.oneOf("Miguel", "Allie", "Spencer", "StarFox")
28 | ids <- arbitrary[Set[Denormalize.ID]].map { _.to[SortedSet] }
29 | } yield Denormalize.User(name, ids)
30 | }
31 |
32 | implicit val groupsGen = Arbitrary {
33 | for {
34 | name <- Gen.oneOf("Robbers", "Colts Fans", "Breadwinners")
35 | } yield Denormalize.Group(name)
36 | }
37 |
38 |
39 | "never output a group if no groups are added" in {
40 |
41 | val compiled = Machine.compile(Denormalize.graph)
42 |
43 | prop { input: Map[ID, Seq[Option[User]]] =>
44 |
45 | val messages = Messages.from(Users, input)
46 |
47 | val thing = compiled.push(messages)
48 |
49 | Prop.forAll(Sample.complete(thing)) { output =>
50 |
51 | forall(output(Denormalized)) { case (id, values) =>
52 | values.flatten must beEmpty
53 | }
54 | }
55 | }
56 | }
57 | }
58 | }
59 |
--------------------------------------------------------------------------------
/core/src/test/scala/com/monovore/example/coast/EntityResolutionSpec.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.example.coast
2 |
3 | import com.monovore.coast.machine.{Sample, Messages, Machine}
4 | import org.scalacheck.{Shrink, Arbitrary, Prop, Gen}
5 | import org.specs2.matcher.Parameters
6 | import org.specs2.mutable._
7 | import org.specs2.ScalaCheck
8 |
9 | class EntityResolutionSpec extends Specification with ScalaCheck {
10 |
11 | implicit val scalacheck = Parameters(maxSize = 5)
12 |
13 | "EntityResolutionSpec" should {
14 |
15 | import EntityResolution._
16 |
17 | val products = for {
18 | names <- Gen.nonEmptyContainerOf[Set, Name](
19 | Gen.choose(1, 20).map { n => Name(s"Item #$n") }
20 | )
21 | minPrice <- Gen.choose(5, 25)
22 | category <- Gen.nonEmptyContainerOf[Set, Category](
23 | Gen.oneOf("electronics", "spice", "hardware", "candy", "boxes", "shoes").map(Category)
24 | )
25 | } yield Product(names, minPrice, category)
26 |
27 | implicit val arbProduct = Arbitrary(products)
28 |
29 | implicit val shrinkProduct = Shrink[Product] { case Product(names, price, categories) =>
30 |
31 | Shrink.shrink(names).map { Product(_, price, categories) } append
32 | Shrink.shrink(categories).map { Product(names, price, _) }
33 | }
34 |
35 | "handle a simple example" in {
36 |
37 | val electrons = Category("electronics")
38 | val fooBar = Product(Set(Name("Foo"), Name("Bar")), 12, Set(electrons))
39 | val fooBaz = Product(Set(Name("Foo"), Name("Baz")), 12, Set(electrons))
40 |
41 | val machine = Machine.compile(graph)
42 | .push(Messages.from(RawProducts, Map(7 -> Seq(fooBar, fooBaz))))
43 |
44 | Prop.forAll(Sample.complete(machine)) { output =>
45 |
46 | output(AllProducts)(electrons) must_== Seq(fooBar, merge(fooBar, fooBaz))
47 | }
48 | }
49 |
50 | "merge all mergeable products" in {
51 |
52 | propNoShrink { products: Map[Int, Seq[Product]] =>
53 |
54 | val machine = Machine.compile(graph)
55 | .push(Messages.from(RawProducts, products))
56 |
57 | Prop.forAll(Sample.complete(machine)) { output =>
58 |
59 | forall(output(AllProducts)) { case (scope, values) =>
60 |
61 | values.tails
62 | .collect { case head :: tail => (head, tail) }
63 | .forall { case (head, tail) =>
64 | tail.forall { x => !matches(head, x) || merge(head, x) == x }
65 | }
66 | }
67 | }
68 | }
69 | }
70 |
71 | "not throw away any information" in {
72 |
73 | propNoShrink { products: Map[Int, Seq[Product]] =>
74 |
75 | val machine =
76 | Machine.compile(graph)
77 | .push(Messages.from(RawProducts, products))
78 |
79 | Prop.forAll(Sample.complete(machine)) { output =>
80 |
81 | val inputProducts: Seq[Product] = products.values.flatten.toSeq
82 |
83 | val allProducts = output(AllProducts)
84 |
85 | forall(inputProducts) { product =>
86 |
87 | product.categories forall { category =>
88 | allProducts(category) exists { other =>
89 | matches(product, other) && merge(product, other) == other
90 | }
91 | }
92 | }
93 | }
94 | }
95 | }
96 |
97 | "match the results of a simple swoosh run" in {
98 |
99 | propNoShrink { products: Map[Int, Seq[Product]] =>
100 |
101 | val machine =
102 | Machine.compile(graph)
103 | .push(Messages.from(RawProducts, products))
104 |
105 | Prop.forAll(Sample.complete(machine)) { output =>
106 |
107 | val swooshed = products.values.flatten
108 | .foldLeft(Set.empty[Product]) { (set, next) =>
109 | mergeAll(set, next)._1
110 | }
111 |
112 | val allProducts = output(AllProducts)
113 |
114 | forall(swooshed) { product =>
115 |
116 | product.categories forall { category =>
117 | allProducts(category).contains(product)
118 | }
119 | }
120 | }
121 | }
122 | }
123 | }
124 | }
125 |
--------------------------------------------------------------------------------
/core/src/test/scala/com/monovore/example/coast/LinearRoadSpec.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.example.coast
2 |
3 | import com.monovore.coast.machine.{Messages, Sample, Machine}
4 | import com.monovore.example.coast.LinearRoad.PlaceAndTime
5 | import org.scalacheck.Prop
6 | import org.specs2.matcher.Parameters
7 | import org.specs2.mutable._
8 | import org.specs2.ScalaCheck
9 |
10 | class LinearRoadSpec extends Specification with ScalaCheck {
11 |
12 | implicit val scalacheck = Parameters(maxSize = 20)
13 |
14 | "The linear road example" should {
15 |
16 | "not charge at times when cars are fast" in {
17 |
18 | val vehicle: LinearRoad.VehicleID = 7L
19 |
20 | val input = Messages.from(LinearRoad.PositionReports, Map(vehicle -> Seq(PlaceAndTime(10, 20) -> 50.5)))
21 |
22 | val testCase = Machine.compile(LinearRoad.graph).push(input)
23 |
24 | Prop.forAll(Sample.complete(testCase)) { output =>
25 |
26 | output(LinearRoad.TotalTolls)(vehicle).lastOption.getOrElse(0L) must_== 0
27 | }
28 | }
29 |
30 | "charge at times when cars are slow" in {
31 |
32 | val vehicle: LinearRoad.VehicleID = 7L
33 |
34 | val input = Messages.from(LinearRoad.PositionReports, Map(vehicle -> Seq(PlaceAndTime(10, 20) -> 20.5)))
35 |
36 | val testCase = Machine.compile(LinearRoad.graph).push(input)
37 |
38 | Prop.forAll(Sample.complete(testCase)) { output =>
39 | output(LinearRoad.TotalTolls)(vehicle).last must_!= 0
40 | }
41 | }
42 | }
43 | }
44 |
--------------------------------------------------------------------------------
/docs/_config.yaml:
--------------------------------------------------------------------------------
1 | defaults:
2 | - scope:
3 | path: "" # everybody gets the same layout
4 | values:
5 | layout: "default"
6 |
--------------------------------------------------------------------------------
/docs/_layouts/default.html:
--------------------------------------------------------------------------------
1 |
2 |
3 |
4 |
5 |
6 |
7 |
8 |
15 |
16 |
17 |
18 | {{ page.title }}
19 | {{ content }}
20 |
21 |
22 |
23 |
24 |
25 |
--------------------------------------------------------------------------------
/docs/flow.md:
--------------------------------------------------------------------------------
1 | ---
2 | title: Flow API
3 | ---
4 |
5 | This page covers the Scala API used to build up a streaming graph from scratch.
6 |
7 | ## Learning by Example
8 |
9 | At the moment, the best way to learn the basics is to read through the [Twitter
10 | Reach example][twitter-reach]: it reviews everything from defining topics to
11 | implementing streaming joins.
12 |
13 | [twitter-reach]: https://github.com/bkirwi/coast/blob/master/core/src/main/scala/com/monovore/example/coast/TwitterReach.scala
14 |
15 | This page collects a little extra information:
16 | we'll go into more detail on some finer points of the API,
17 | and look at some unusual features.
18 |
19 | ## Partitioning and Serialization
20 |
21 | In the `wire` package,
22 | `coast` defines a couple abstractions
23 | that allow you to configure serialization and partitioning on a per-type basis.
24 |
25 | - `BinaryFormat`: This provides a basic serialization interface. `coast`
26 | requires an implementation of this anywhere data is getting sent over the
27 | network -- whether updating state or writing back to Kafka.
28 |
29 | - `Partitioner`: This is used to assign a 'logical' partition (indexed by an
30 | arbitrary key) to a 'physical' partition (indexed by a small integer). This is
31 | very similar to Kafka's partitioning interface.
32 |
33 |
34 |
35 | ## Named Streams and Pools
36 |
37 | In the flow API, you can give a specific name to a stream:
38 |
39 | ```scala
40 | val namedStream = Flow.stream("stream-name") {
41 | // stream definition
42 | }
43 | ```
44 |
45 | On a cluster, this ensures the data is written back out to Kafka.
46 |
47 | There are a couple of reasons you might want to do this:
48 |
49 | - Sharing: if an intermediate stream is expensive to compute, sending it over
50 | the network may be cheaper than recomputing it.
51 |
52 | - Determinism: when you merge two streams, a variety of interleavings are
53 | possible. By naming a stream, you ensure that all downstream processing sees
54 | the same ordering.
55 |
56 | - Regrouping: one of the advantages of partitioning is that all state for
57 | messages with the same key can be maintained on a single node. This means
58 | that, when you change the partitioning of the stream, you need to shuffle data
59 | over the network. If you try to do some stateful processing before naming the
60 | stream, the compiler will warn you about it.
61 |
62 | ## Cycles
63 |
64 | Most streaming graphs in the wild are 'acyclic' -- messages flow in one direction
65 | from input to output. This is a good thing! Graphs without cycles have a simpler
66 | data flow, so they're often easier to build and understand.
67 |
68 | Unfortunately, some streaming problems are inherently cyclic. Suppose you have a stream of pages
69 | partitioned by URL, and you want to run a continuous page-rank calculation over the
70 | stream; updating the rank for one page may change the rank of all the pages
71 | it links to, so the output of one partition's calculation is the
72 | input for another. A lot of calculations that have an *iterative* structure when run
73 | in batch have a cyclic *structure* when translated to the streaming world, so
74 | handling cycles is an important task for a streaming framework.
75 |
76 | You introduce a cycle like this:
77 |
78 | ```scala
79 | val cyclicFlow = Flow.cycle[Key, Message]("stream-name") { cyclicStream =>
80 | // stream definition goes here
81 | }
82 | ```
83 |
84 | This looks like a regular stream definition, but there are a few key differences:
85 |
86 | - The `cyclicStream` is passed in as a parameter, so you can use it as part of the
87 | stream's definition. (Sound self-referential? That's the cycle!)
88 |
89 | - You have to write the key and value types explicitly... there's not enough context
90 | here for the compiler to infer it.
91 |
--------------------------------------------------------------------------------
/docs/overview.md:
--------------------------------------------------------------------------------
1 | ---
2 | title: Overview
3 | ---
4 |
5 | `coast` borrows a number of terms from Kafka and other streaming projects,
6 | and introduces a few new ideas of its own.
7 | In this page,
8 | we'll walk through some of the most important pieces
9 | and look at how they fit together.
10 |
11 | ## Topics, Streams, and Graphs
12 |
13 | Think of a stream processing job as a black box: you have
14 | messages flowing in at the top, some processing happens, and some new data flows
15 | out at the bottom. `coast` refers to these inputs and outputs as **topics**. You
16 | can think of the set of input and output topics as your job's "public API" --
17 | other parts of the system only interact with it by providing it input or
18 | consuming the output.
19 |
20 | Of course, your job is not really a black box! With `coast`, you describe the
21 | way data flows from input to output by defining **streams**. You can create a
22 | stream my reading data from a topic, by transforming an existing stream, or by
23 | combining multiple streams together. This basic vocabulary is expressive enough
24 | to cover a wide variety of stream processing tasks. Once you've built up the
25 | output you want, you can sink it directly to an output topic.
26 |
27 | A complete stream processing job -- with all its inputs, outputs, and arbitrary
28 | topology of internal streams -- is referred to as a **graph**. `coast` includes
29 | several utilities for testing and visualizing these graphs -- and, of course,
30 | executing them on a cluster.
31 |
32 | ## Partitions and Ordering
33 |
34 | Every topic -- and every stream -- is divided up into an arbitrary number of
35 | **partitions**. Each partition in a stream is identified by a unique key and
36 | holds an unbounded series of messages.
37 |
38 | Most of the basic operations in `coast` are defined 'partitionwise':
39 |
40 | - Messages within a partition have a well-defined order, but `coast` defines no
41 | particular ordering between messages in different partitions or different
42 | streams.
43 |
44 | - All state is local to a partition. (You can think of this as a key-value
45 | store, where `coast` stores the current state for each partition key.)
46 |
47 | - When you merge multiple streams, partitions with the same key are merged
48 | together.
49 |
50 | Partitioning is very useful for parallelism: different partitions can be
51 | assigned to different nodes, so processing can scale easily across a cluster.
52 | By default, processing for each partition happens independently, but it's easy
53 | to repartition a stream when you want to bring data from multiple partitions
54 | together.
55 | This lends itself to a map/reduce style of computation --
56 | you do as much as you can in parallel,
57 | reshuffling the data when you need to aggregate across multiple messages.
58 |
--------------------------------------------------------------------------------
/docs/samza.md:
--------------------------------------------------------------------------------
1 | ---
2 | title: Samza Backend
3 | ---
4 |
5 | `coast` can compile a flow down to a set of Samza jobs, which can be run
6 | locally or on the cluster.
7 |
8 | ## Overview
9 |
10 | `coast` leverages a lot of Samza's infrastructure, from its deployment model to
11 | its state management approach. We'll touch briefly on a few key points of Samza's
12 | design here -- if anything is unfamiliar, you'll want to follow the links through
13 | to Samza's documentation.
14 |
15 | Samza's job definitions are configuration-based: a single config file specifies
16 | all the classes and metadata necessary to run the job.
17 | `coast` integrates with Samza by compiling its own graph representation down to a set of Samza configs,
18 | each of which defines an independent Samza job.
19 | You can write these configs back out to the file system,
20 | to take advantage of the normal Samza deployment flow...
21 | or just run them immediately to launch the whole data flow graph on the cluster.
22 |
23 | The generated jobs use Samza's usual approach to state management:
24 | state can be stored in any of Samza's `KeyValueStore`s,
25 | and a Kafka commit log is used for persistence.
26 |
27 | ## Backends
28 |
29 | There are currently two distinct Samza backends,
30 | aimed at somewhat different use cases.
31 | `SimpleBackend` provides the most direct mapping between `coast`'s streaming model and Samza's primitives.
32 | This gives you roughly the same flexibility and performance as a raw Samza job,
33 | along with the same guarantees --
34 | in particular,
35 | message sends and state updates may happen more than once,
36 | and a restarted task may see an inconsistent message ordering.
37 | If you're already happy with Samza's reliability story,
38 | this backend might be a good choice.
39 |
40 | As the name suggests, `SafeBackend` provides stronger guarantees --
41 | it supports exactly-once state updates and message publishing,
42 | even in the presence of failure.
43 | This requires tracking some extra metadata and taking particular care with message ordering,
44 | which does have a small performance cost.
45 |
46 | > **NB:** The 'safe' backend is relatively new --
47 | > it depends on some new features in Samza 0.9 --
48 | > and the current version has a couple extra limitations:
49 | > Kafka is the only supported system,
50 | > and all topics are expected to have the same number of partitions.
51 | > These issues should be resolved in the next release.
52 |
53 | These
54 |
55 | ## Configuration
56 |
57 | `coast` respects Samza's existing config when possible, and defines a few properties
58 | of its own:
59 |
60 | - `coast.system.name`: At the moment, `coast` expects all inputs and outputs to
61 | be part of the same Samza 'system'. The default value is `kafka`.
62 |
63 | - `coast.default.stores.*`: `coast` may generate multiple storage instances per Samza job,
64 | one for every join or stateful transformation. These config keys are used to set the
65 | default configuration for each
66 |
67 | You probably want to specify a
68 | storage factory here -- the in-memory store or the RocksDB backed storage are
69 | both good choices.
70 |
71 | ## `SamzaApp`
72 |
73 | `coast-samza` also provides a simple `SamzaApp` template, which provides a basic
74 | command-line interface.
75 |
76 | ```scala
77 | object MyApp extends SamzaApp(backend = SimpleBackend) {
78 |
79 | val graph = ??? // coast graph goes here
80 |
81 | }
82 | ```
83 |
84 | The resulting application can be run with the usual Samza class runner:
85 |
86 | ```bash
87 | bin/run-class.sh org.whatever.MyApp
88 | ```
89 |
90 | Supported commands include:
91 |
92 | - `run`: Runs the job on the cluster.
93 |
94 | ```bash
95 | bin/run-class.sh org.whatever.MyApp run --config-file config/base-config.properties
96 | ```
97 |
98 | - `gen-config`: Instead of running the jobs directly, this command writes the generated configs
99 | back out to a file. The generated files define 'vanilla' Samza jobs: each generated file is
100 | completely self-contained, and can be run with Samza's standard `bin/run-job.sh` script. This
101 | is a bit more work than having the application launch the jobs directly, of course, but it's also a
102 | bit more flexible -- for example, you might run the config-generation step on a build server
103 | but use an existing Samza deployment process to execute each job on the cluster.
104 |
105 | - `print-dot`: Print out a description of the graph in GraphViz's `dot` format. By default, this
106 | writes to standard out, but it's often more useful to write it to a file:
107 |
108 | ```bash
109 | bin/run-class.sh org.whatever.MyApp print-dot --to-file /tmp/my-app.dot && dot -Tjpg /tmp/my-app.dot
110 | ```
111 |
112 | - `info`: Lists a bunch of information about the job: inputs, outputs, and changelogs, as well
113 | as the merge and checkpoint streams for the 'safe' backend. This information may be useful if you
114 | configure your Kafka topics manually -- you can confirm that all the listed topics exist and have
115 | the proper settings before launching the job for the first time.
116 |
117 |
--------------------------------------------------------------------------------
/docs/semantics.md:
--------------------------------------------------------------------------------
1 | ---
2 | title: Semantics
3 | ---
4 |
5 | Notes! Caveat lector.
6 |
7 | In the spirit of denotational semantics, this sketches out a formal underpinning
8 | for `coast`'s streaming operations.
9 |
10 | # Streams are sequences
11 |
12 | In computer science, _stream_ is often used to refer to an infinite sequence of
13 | elements -- following one after the other, in order, and continuing indefinitely.
14 |
15 | `coast` basically adopts this model. In practice, it's useful to think of
16 | streams as "potentially" infinite; while we're never sitting around holding an
17 | infinity of events, it's always possible that a new one might show up, no matter
18 | how many we have.
19 |
20 | This gives an easy model for most stream transformations: we're just mapping
21 | streams to other streams.
22 |
23 | It also gives a workable model for a single element of state that changes over
24 | time: given an initial state, the stream can capture the evolution of that state
25 | over time.
26 |
27 | # Partitions as functions
28 |
29 | While streams are convenient, they imposa an impractical total order on events
30 | -- we still need a way to express 'independence' or 'concurrency'. As a model
31 | for this, we use plain-old-functions. Our functions take a _partition key_, and
32 | return a stream. This gives us our notion of independance -- the streams
33 | returned by `f(a)` and `f(b)` have no implied ordering relationship. In Scala, a
34 | naive type for a stream with `Int` keys and `String` values might be `Int =>
35 | Stream[String]`.
36 |
37 | We can still apply our usual transformations to a function-to-streams; we just
38 | need to do it 'value-wise'
39 |
40 | # Adding and removing concurrency
41 |
42 | There are two operations that allow us to go from streams to
43 | functions-of-streams and back again.
44 |
45 | _Splitting_ a stream 'destroys' ordering information -- it takes a single stream
46 | and splits it up into many streams. Given a function from stream values to keys,
47 | we can get a function from keys to streams; this works a little bit like the
48 | standard `.groupBy` function on `Scala` sequences.
49 |
50 | _Merging_ a stream, on the other hand, 'creates' ordering information: given a
51 | function-to-streams, it collapses all the elements from the entire range of the
52 | function to a single stream. As mentioned above, there's no implied ordering
53 | between the different keys... and this makes the result of a merge hopelessly
54 | undefined. As far as this semantics goes, _any_ merging of the results is
55 | allowed. This is an unfortunate nondeterminism, but it does an okay job of
56 | modelling how messaging works in a world with unbounded delays.
57 |
58 | You can get pretty far with this -- a standard regrouping operation can be
59 | modelled as a split followed by a merge.
60 |
--------------------------------------------------------------------------------
/project/CoastBuild.scala:
--------------------------------------------------------------------------------
1 | import sbt._
2 | import Keys._
3 |
4 | object CoastBuild extends Build {
5 |
6 | import bintray.Plugin._
7 |
8 | lazy val coast = Project(
9 | id = "coast",
10 | base = file(".")
11 | ) aggregate (
12 | core, samza
13 | ) settings(
14 |
15 | // global project settings
16 | scalaVersion in ThisBuild := "2.10.4",
17 | scalacOptions in ThisBuild := Seq("-feature", "-language:higherKinds"),
18 |
19 | organization in ThisBuild := "com.monovore",
20 | version in ThisBuild := "0.3.0-SNAPSHOT",
21 |
22 | licenses in ThisBuild += ("Apache-2.0", url("https://www.apache.org/licenses/LICENSE-2.0.html")),
23 |
24 | // make it possible to cancel forked processes with ctrl-c
25 | cancelable in Global := true,
26 |
27 | // No tests in aggregate project
28 | test := (),
29 | publish := (),
30 |
31 | libraryDependencies in ThisBuild ++= Seq(
32 | "org.specs2" %% "specs2" % "2.4.15" % "test",
33 | "org.scalacheck" %% "scalacheck" % "1.12.1" % "test"
34 | ),
35 |
36 | publishMavenStyle := false
37 | )
38 |
39 | lazy val core = Project(
40 | id = "coast-core",
41 | base = file("core")
42 | ) settings (
43 | bintrayPublishSettings: _*
44 | )
45 |
46 | lazy val samza = Project(
47 | id = "coast-samza",
48 | base = file("samza")
49 | ) dependsOn (
50 | core
51 | ) configs(
52 | IntegrationTest
53 | ) settings (
54 | bintrayPublishSettings ++ Defaults.itSettings : _*
55 | )
56 | }
57 |
--------------------------------------------------------------------------------
/project/build.properties:
--------------------------------------------------------------------------------
1 | sbt.version = 0.13.8
2 |
--------------------------------------------------------------------------------
/project/plugins.sbt:
--------------------------------------------------------------------------------
1 | addSbtPlugin("me.lessis" % "bintray-sbt" % "0.2.1")
2 |
--------------------------------------------------------------------------------
/samza/README.md:
--------------------------------------------------------------------------------
1 | # `samza`
2 |
3 | This module contains the full Samza integration. Most of the work here is
4 | converting between the `coast` and Samza streaming models to get the right job
5 | logic and message ordering.
6 |
7 |
--------------------------------------------------------------------------------
/samza/build.sbt:
--------------------------------------------------------------------------------
1 | def samzaDep(name: String) = "org.apache.samza" %% name % "0.9.0"
2 |
3 | resolvers += "Local Maven Repository" at s"file://${Path.userHome.absolutePath}/.m2/repository"
4 |
5 | libraryDependencies ++= Seq(
6 | "com.google.guava" % "guava" % "18.0",
7 | "com.google.code.findbugs" % "jsr305" % "1.3.9" % "provided",
8 | samzaDep("samza-core"),
9 | samzaDep("samza-kv"),
10 | "ch.qos.logback" % "logback-classic" % "1.1.2",
11 | // TODO: integration tests only
12 | samzaDep("samza-kv-inmemory") exclude ("com.google.guava", "guava"),
13 | samzaDep("samza-kafka") exclude("org.slf4j", "slf4j-log4j12") exclude("org.apache.zookeeper", "zookeeper"),
14 | "org.slf4j" % "log4j-over-slf4j" % "1.7.6",
15 | "org.apache.kafka" %% "kafka" % "0.8.2.1"
16 | )
17 |
18 | libraryDependencies ++= Seq(
19 | "org.apache.kafka" %% "kafka" % "0.8.2.1" classifier "test", // TODO: scope under IntegrationTest
20 | "org.specs2" %% "specs2" % "2.4.15" % "it",
21 | "org.scalacheck" %% "scalacheck" % "1.12.1" % "it"
22 | )
23 |
24 | fork in run := true
25 |
26 | fork in IntegrationTest := true
27 |
28 | parallelExecution in IntegrationTest := false
29 |
30 | baseDirectory in (IntegrationTest, test) := { baseDirectory.value / "target" }
--------------------------------------------------------------------------------
/samza/src/it/resources/logback-test.xml:
--------------------------------------------------------------------------------
1 |
2 |
3 |
4 |
6 |
7 | %d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n
8 |
9 |
10 |
11 |
12 |
13 |
14 |
15 |
16 |
17 |
18 |
19 |
20 |
21 |
22 |
23 |
24 |
--------------------------------------------------------------------------------
/samza/src/it/scala/com/monovore/integration/coast/CoastKafkaSystemSpec.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.integration.coast
2 |
3 | import com.monovore.coast.flow.Topic
4 | import com.monovore.coast.samza.safe.CoastKafkaSystem
5 | import com.monovore.coast.wire.{Protocol, Serializer}
6 | import kafka.producer.{Producer, ProducerConfig}
7 | import org.apache.samza.system.{OutgoingMessageEnvelope, SystemStream}
8 | import org.specs2.ScalaCheck
9 | import org.specs2.mutable._
10 |
11 | class CoastKafkaSystemSpec extends Specification with ScalaCheck {
12 |
13 | val BigNumber = 5000
14 |
15 | sequential
16 |
17 | "a ratcheted kafka producer" should {
18 |
19 | import Protocol.common._
20 |
21 | def stream(name: String) = Topic[String, Int](name)
22 |
23 | "eventually write all the data" in {
24 |
25 | IntegrationTest.withKafkaCluster { props =>
26 |
27 | IntegrationTest.expect(Set("test"), props)
28 |
29 | val producer = new CoastKafkaSystem.Producer(
30 | new Producer(new ProducerConfig(props)),
31 | {
32 | case "odd" => 1
33 | case "even" => 2
34 | case _ => 0
35 | }
36 | )
37 |
38 | producer.register("producer-test")
39 |
40 | producer.start()
41 |
42 | val messages = for (i <- 1 to BigNumber) yield {
43 | new OutgoingMessageEnvelope(
44 | new SystemStream("producer-test", "test"),
45 | "empty".##,
46 | Serializer.toArray("empty"),
47 | Serializer.toArray(i)
48 | )
49 | }
50 |
51 | for (message <- messages) {
52 | producer.send("producer-test", message)
53 | }
54 |
55 | producer.flush("producer-test")
56 |
57 | Thread.sleep(5000)
58 |
59 | producer.stop()
60 |
61 | val returned = IntegrationTest.slurp(Set("test"), props)
62 |
63 | returned.get(stream("test")).apply("empty") must_== (1 to BigNumber)
64 | }
65 | }
66 |
67 | "maintain some ordering invariants" in {
68 |
69 | IntegrationTest.withKafkaCluster { clientProps =>
70 |
71 | IntegrationTest.expect(Set("even", "odd"), clientProps)
72 |
73 | val producer = new CoastKafkaSystem.Producer(
74 | new Producer(new ProducerConfig(clientProps)),
75 | {
76 | case "odd" => 1
77 | case "even" => 2
78 | case _ => 0
79 | }
80 | )
81 |
82 | producer.register("producer-test")
83 |
84 | producer.start()
85 |
86 | val messages = for (i <- 1 to BigNumber) yield {
87 | new OutgoingMessageEnvelope(
88 | new SystemStream("producer-test", if (i % 2 == 0) "even" else "odd"),
89 | "empty".##,
90 | Serializer.toArray("empty"),
91 | Serializer.toArray(i)
92 | )
93 | }
94 |
95 | for (message <- messages) {
96 | producer.send("producer-test", message)
97 | Thread.sleep(0, 500)
98 | }
99 |
100 | producer.stop()
101 |
102 | val persisted = IntegrationTest.slurp(Set("even", "odd"), clientProps)
103 |
104 | def contiguous(seq: Seq[Int]) = seq.grouped(2).foreach {
105 | case Seq(a, b) => { (b - a) must_== 2 }
106 | case _ => {}
107 | }
108 |
109 | val evens = persisted.get(stream("even")).apply("empty")
110 | val odds = persisted.get(stream("odd")).apply("empty")
111 |
112 | val allOdds = odds.toSet
113 |
114 | evens.foreach { even => allOdds.contains(even - 1) must_== true }
115 |
116 | contiguous { evens }
117 | contiguous { odds }
118 |
119 | evens must not be empty
120 | odds must not be empty
121 | }
122 | }
123 | }
124 | }
125 |
--------------------------------------------------------------------------------
/samza/src/it/scala/com/monovore/integration/coast/IntegrationTest.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.integration.coast
2 |
3 | import java.lang.Thread.UncaughtExceptionHandler
4 | import java.nio.ByteBuffer
5 | import java.util.Properties
6 |
7 | import com.monovore.coast
8 | import coast.samza.{SamzaConfig, SafeBackend, SimpleBackend}
9 | import com.monovore.coast.flow.Topic
10 | import com.monovore.coast.core.Graph
11 | import com.monovore.coast.wire.{Serializer, Partitioner}
12 | import kafka.api.{FetchRequest, PartitionFetchInfo, TopicMetadataRequest}
13 | import kafka.common.TopicAndPartition
14 | import kafka.consumer.{ConsumerConfig, SimpleConsumer}
15 | import kafka.producer.{KeyedMessage, Producer, ProducerConfig}
16 | import kafka.server.{KafkaConfig, KafkaServer}
17 | import kafka.utils.{TestUtils, TestZKUtils}
18 | import kafka.zk.EmbeddedZookeeper
19 | import org.apache.samza.job.ApplicationStatus
20 | import org.apache.samza.job.local.ThreadJobFactory
21 | import org.apache.samza.storage.kv.inmemory.InMemoryKeyValueStorageEngineFactory
22 | import org.slf4j.LoggerFactory
23 | import org.specs2.execute.{SkipException, Skipped}
24 |
25 | import scala.util.Random
26 |
27 | object IntegrationTest {
28 |
29 | val logger = LoggerFactory.getLogger("coast.IntegrationTest")
30 |
31 | def withKafkaCluster[A](withProps: java.util.Properties => A): A = {
32 |
33 | val Seq(port0) = TestUtils.choosePorts(1)
34 |
35 | val broker0 = TestUtils.createBrokerConfig(0, port0)
36 | broker0.setProperty("auto.create.topics.enable", "true")
37 | broker0.setProperty("num.partitions", "3")
38 |
39 | val config = new java.util.Properties()
40 |
41 | val brokers = s"localhost:$port0"
42 | val zkString = TestZKUtils.zookeeperConnect
43 |
44 | config.setProperty("metadata.broker.list", brokers)
45 | config.setProperty("producer.type", "sync")
46 | config.setProperty("request.required.acks", "1")
47 | config.setProperty("message.send.max.retries", "0")
48 |
49 | config.setProperty("zookeeper.connect", zkString)
50 | config.setProperty("group.id", "input-producer")
51 | config.setProperty("auto.offset.reset", "smallest")
52 |
53 | var zookeeper: EmbeddedZookeeper = null
54 | var server0: KafkaServer = null
55 |
56 | try {
57 | zookeeper = new EmbeddedZookeeper(zkString)
58 |
59 | server0 = TestUtils.createServer(new KafkaConfig(broker0))
60 |
61 | withProps(config)
62 |
63 | } finally {
64 |
65 | if (server0 != null) {
66 | server0.shutdown()
67 | server0.awaitShutdown()
68 | }
69 |
70 | if (zookeeper != null) {
71 | zookeeper.shutdown()
72 | }
73 | }
74 | }
75 |
76 | def fuzz(graph: Graph, input: Messages, simple: Boolean = false): Messages = {
77 |
78 | Thread.setDefaultUncaughtExceptionHandler(new UncaughtExceptionHandler {
79 | override def uncaughtException(thread: Thread, throwable: Throwable): Unit = {
80 | logger.error(thread.getName, throwable)
81 | }
82 | })
83 |
84 | val factory = new ThreadJobFactory
85 |
86 | var producer: Producer[Array[Byte], Array[Byte]] = null
87 | var consumer: SimpleConsumer = null
88 |
89 | IntegrationTest.withKafkaCluster { config =>
90 |
91 | val producerConfig = new ProducerConfig(config)
92 |
93 | try {
94 |
95 | IntegrationTest.expect(input.messages.keySet, config)
96 |
97 | val port = config.getProperty("metadata.broker.list")
98 | .split(",")(0)
99 | .split(":")(1)
100 | .toInt
101 |
102 | producer = new Producer(producerConfig)
103 | consumer = new SimpleConsumer("localhost", port, ConsumerConfig.SocketTimeout, ConsumerConfig.SocketBufferSize, ConsumerConfig.DefaultClientId)
104 |
105 | for {
106 | (name, messages) <- input.messages
107 | numPartitions = {
108 | val meta = consumer.send(new TopicMetadataRequest(Seq(name), 913))
109 | meta.topicsMetadata.find { _.topic == name }.get
110 | .partitionsMetadata.size
111 | }
112 | (key, (partitioner, values)) <- messages
113 | partitionId = partitioner(numPartitions)
114 | value <- values.grouped(100)
115 | } {
116 | producer.send(value.map { value => new KeyedMessage(name, key.toArray, partitionId, value.toArray)}: _*)
117 | }
118 |
119 | val baseConfig = SamzaConfig.from(
120 | // toy-problem config
121 | "task.window.ms" -> "30",
122 |
123 | "coast.system.name" -> "toy-kafka",
124 |
125 | // overridden in safe config generator
126 | "task.checkpoint.factory" -> "org.apache.samza.checkpoint.kafka.KafkaCheckpointManagerFactory",
127 | "task.checkpoint.system" -> "toy-kafka",
128 | "task.checkpoint.replication.factor" -> "1",
129 | "systems.toy-kafka.samza.factory" -> "org.apache.samza.system.kafka.KafkaSystemFactory",
130 | // point things at local kafka / zookeeper2
131 | "systems.toy-kafka.consumer.zookeeper.connect" -> config.getProperty("zookeeper.connect"),
132 | "systems.toy-kafka.producer.metadata.broker.list" -> config.getProperty("metadata.broker.list"),
133 | "systems.toy-kafka.producer.bootstrap.servers" -> config.getProperty("metadata.broker.list"),
134 | // config template for storage
135 | "coast.default.stores.changelog.replication.factor" -> "1"
136 | )
137 |
138 | val backend = if (simple) SimpleBackend else SafeBackend
139 |
140 | val configs = backend(baseConfig).configure(graph)
141 |
142 | // FLAIL!
143 |
144 | val sleeps =
145 | if (simple) Seq(8000)
146 | else (0 until 3).map { _ => Random.nextInt(1500) + 1500} ++ Seq(18000)
147 |
148 | for (sleepTime <- sleeps) {
149 |
150 | val jobs = configs.values.toSeq
151 | .map { config => factory.getJob(config)}
152 |
153 | jobs.foreach {
154 | _.submit()
155 | }
156 |
157 | Thread.sleep(sleepTime)
158 |
159 | jobs.foreach {
160 | _.kill()
161 | }
162 |
163 | jobs.foreach { job =>
164 | job.waitForFinish(2000) match {
165 | case ApplicationStatus.SuccessfulFinish => ()
166 | case ApplicationStatus.UnsuccessfulFinish => ()
167 | case status => {
168 | throw new SkipException(Skipped(s"TOO SLOW: $status"))
169 | }
170 | }
171 | }
172 | }
173 |
174 | val outputStreams = graph.bindings.map { case (name, _) => name}
175 |
176 | IntegrationTest.slurp(outputStreams.toSet, config)
177 |
178 | } finally {
179 |
180 | if (producer != null) {
181 | producer.close()
182 | }
183 |
184 | if (consumer != null) {
185 | consumer.close()
186 | }
187 | }
188 | }
189 | }
190 |
191 | def expect(topics: Set[String], config: Properties): Unit = {
192 | slurp(topics, config)
193 | Thread.sleep(300)
194 | }
195 |
196 | def slurp(topics: Set[String], config: Properties): Messages = {
197 |
198 | var simple: Map[Int, SimpleConsumer] = null
199 |
200 | try {
201 |
202 | val ports = config.getProperty("metadata.broker.list").split(",")
203 | .map { _.split(":")(1).toInt }
204 |
205 | simple = ports
206 | .map { port =>
207 | port -> new SimpleConsumer("localhost", port, ConsumerConfig.SocketTimeout, ConsumerConfig.SocketBufferSize, ConsumerConfig.DefaultClientId)
208 | }
209 | .toMap
210 |
211 | val meta = simple.values.head.send(new TopicMetadataRequest(topics.toSeq, 236))
212 |
213 | def toByteSeq(bb: ByteBuffer): Seq[Byte] = {
214 | val bytes = Array.ofDim[Byte](bb.remaining())
215 | bb.duplicate().get(bytes)
216 | bytes.toSeq
217 | }
218 |
219 | val outputMessages = meta.topicsMetadata
220 | .map { topic =>
221 | val messages = topic.partitionsMetadata
222 | .flatMap { partition =>
223 | val broker = partition.leader.get.port
224 |
225 | val consumer = simple(broker)
226 |
227 | val tp = TopicAndPartition(topic.topic, partition.partitionId)
228 |
229 | val correlationID = Random.nextInt()
230 | val minBytes = 0
231 |
232 | val response = consumer.fetch(new FetchRequest(
233 | correlationID,
234 | ConsumerConfig.DefaultClientId,
235 | ConsumerConfig.MaxFetchWaitMs,
236 | minBytes,
237 | Map(
238 | tp -> PartitionFetchInfo(0L, Int.MaxValue)
239 | )
240 | ))
241 |
242 | response.data(tp).messages.toSeq
243 | .map { mao => toByteSeq(mao.message.key) -> (partition.partitionId, toByteSeq(mao.message.payload)) }
244 | }
245 |
246 | topic.topic -> messages.groupBy { _._1 }
247 | .mapValues { p =>
248 | val (_, pairs) = p.unzip
249 | val (partitions, data) = pairs.unzip
250 | ({n: Int => partitions.head }, data)
251 | }
252 | }
253 | .toMap
254 |
255 | Messages(outputMessages)
256 |
257 | } finally {
258 |
259 | if (simple != null) {
260 | simple.values.foreach { _.close() }
261 | }
262 | }
263 | }
264 | }
265 |
266 | case class Messages(messages: Map[String, Map[Seq[Byte], (Int => Int, Seq[Seq[Byte]])]] = Map.empty) {
267 |
268 | def add[A : Serializer : Partitioner, B : Serializer](name: Topic[A,B], messages: Map[A, Seq[B]]): Messages = {
269 |
270 | val formatted = messages.map { case (k, vs) =>
271 | val pn: (Int => Int) = implicitly[Partitioner[A]].partition(k, _)
272 | Serializer.toArray(k).toSeq -> (pn, vs.map { v => Serializer.toArray(v).toSeq })
273 | }
274 |
275 | Messages(this.messages.updated(name.name, formatted))
276 | }
277 |
278 | def get[A : Serializer, B : Serializer](name: Topic[A, B]): Map[A, Seq[B]] = {
279 |
280 | val data = messages.getOrElse(name.name, Map.empty)
281 |
282 | data.map { case (k, (_, vs) ) =>
283 | Serializer.fromArray[A](k.toArray) -> vs.map { v => Serializer.fromArray[B](v.toArray) }
284 | }.withDefaultValue(Seq.empty[B])
285 | }
286 | }
287 |
288 | object Messages extends Messages(Map.empty)
--------------------------------------------------------------------------------
/samza/src/it/scala/com/monovore/integration/coast/SafeIntegrationSpec.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.integration.coast
2 |
3 | import com.monovore.coast.flow.{Flow, Topic}
4 | import com.monovore.coast.wire.Protocol
5 | import org.specs2.ScalaCheck
6 | import org.specs2.mutable._
7 |
8 | class SafeIntegrationSpec extends Specification with ScalaCheck {
9 |
10 | // unfortunately, a lot of issues show up only with particular timing
11 | // this helps make results more reproducible
12 | sequential
13 |
14 | val BigNumber = 15000 // pretty big?
15 |
16 | "a running samza-based job" should {
17 |
18 | import Protocol.common._
19 |
20 | val Foo = Topic[String, Int]("foo")
21 |
22 | val Bar = Topic[String, Int]("bar")
23 |
24 | // table of contents
25 | passthrough
26 | flatMap
27 | transforms
28 | composition
29 | merging
30 | grouping
31 | cycles
32 |
33 | def passthrough = "pass through data safely" in {
34 |
35 | val graph = Flow.sink(Bar) {
36 | Flow.source(Foo)
37 | }
38 |
39 | val inputData = Map(
40 | "first" -> (1 to BigNumber),
41 | "second" -> (1 to BigNumber),
42 | "third" -> (1 to BigNumber)
43 | )
44 |
45 | val input = Messages().add(Foo, inputData)
46 |
47 | val output = IntegrationTest.fuzz(graph, input).get(Bar)
48 |
49 | output("second") must_== inputData("second")
50 | }
51 |
52 | def flatMap = "flatMap nicely" in {
53 |
54 | val graph = Flow.sink(Bar) {
55 | Flow.source(Foo).flatMap { n => Seq.fill(3)(n) }
56 | }
57 |
58 | val inputData = Map(
59 | "foo" -> (1 to BigNumber),
60 | "bar" -> (1 to BigNumber)
61 | )
62 |
63 | val input = Messages().add(Foo, inputData)
64 |
65 | val output = IntegrationTest.fuzz(graph, input).get(Bar)
66 |
67 | output("bar") must_== inputData("bar").flatMap { n => Seq.fill(3)(n) }
68 | }
69 |
70 | def transforms = "accumulate state" in {
71 |
72 | val graph = Flow.sink(Bar) {
73 | Flow.source(Foo).fold(0) { (n, _) => n + 1 }.updates
74 | }
75 |
76 | val inputData = Map(
77 | "one" -> (1 to BigNumber),
78 | "two" -> (1 to BigNumber)
79 | )
80 |
81 | val input = Messages().add(Foo, inputData)
82 |
83 | val output = IntegrationTest.fuzz(graph, input).get(Bar)
84 |
85 | output("one") must_== inputData("one")
86 | output("two") must_== inputData("two")
87 | }
88 |
89 | def composition = "compose well across multiple Samza jobs" in {
90 |
91 | val graph = for {
92 |
93 | once <- Flow.stream("testing") {
94 | Flow.source(Foo).fold(1) { (n, _) => n + 1 }.updates
95 | }
96 |
97 | _ <- Flow.sink(Bar) { once.map { _ + 1 } }
98 |
99 | } yield ()
100 |
101 | val inputData = Map(
102 | "foo" -> (1 to BigNumber),
103 | "bar" -> (1 to BigNumber)
104 | )
105 |
106 | val input = Messages().add(Foo, inputData)
107 |
108 | val output = IntegrationTest.fuzz(graph, input).get(Bar)
109 |
110 | output("bar").filter { _ % 100 == 0 } must_== inputData("bar").map { _ + 2 }.filter { _ % 100 == 0 }
111 | }
112 |
113 | def merging = "do a merge" in {
114 |
115 | val Foo2 = Topic[String, Int]("foo-2")
116 |
117 | val graph = Flow.sink(Bar) {
118 | Flow.merge(
119 | "foo-1" -> Flow.source(Foo),
120 | "foo-2" -> Flow.source(Foo2)
121 | )
122 | }
123 |
124 | val input = Messages
125 | .add(Foo, Map("test" -> (1 to BigNumber by 2)))
126 | .add(Foo2, Map("test" -> (2 to BigNumber by 2)))
127 |
128 | val output = IntegrationTest.fuzz(graph, input).get(Bar)
129 |
130 | output("test").filter { _ % 2 == 1 } must_== (1 to BigNumber by 2)
131 | output("test").filter { _ % 2 == 0 } must_== (2 to BigNumber by 2)
132 | }
133 |
134 | def grouping = "regroup" in {
135 |
136 | val graph = for {
137 |
138 | grouped <- Flow.stream("grouped") {
139 | Flow.source(Foo).groupBy { n => (n % 10).toString }
140 | }
141 |
142 | _ <- Flow.sink(Bar) { grouped }
143 | } yield ()
144 |
145 | val input = Messages
146 | .add(Foo, Map(
147 | "first" -> (0 until BigNumber by 3),
148 | "second" -> (1 until BigNumber by 3),
149 | "third" -> (2 until BigNumber by 3)
150 | ))
151 |
152 | val output = IntegrationTest.fuzz(graph, input).get(Bar)
153 |
154 | forall(output) { case (key, values) =>
155 | val remainder = key.toInt
156 | values.sorted must_== (remainder until BigNumber by 10)
157 | }
158 |
159 | output must not be empty
160 | }
161 |
162 | def cycles = "handle cycles" in {
163 |
164 | val graph = for {
165 |
166 | looped <- Flow.cycle[String, Int]("looped") { looped =>
167 |
168 | val merged = Flow.merge(
169 | "foo" -> Flow.source(Foo),
170 | "loop" -> looped.filter { _ % 2 == 0 }
171 | )
172 |
173 | merged.map { _ + 1 }
174 | }
175 |
176 | _ <- Flow.sink(Bar) { looped }
177 |
178 | } yield ()
179 |
180 | val input = Messages
181 | .add(Foo, Map("test" -> (1 to BigNumber by 2)))
182 |
183 | val output = IntegrationTest.fuzz(graph, input).get(Bar)
184 |
185 | output("test").sorted must_== (2 to (BigNumber+1))
186 | }
187 | }
188 | }
189 |
190 |
--------------------------------------------------------------------------------
/samza/src/it/scala/com/monovore/integration/coast/SimpleIntegrationSpec.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.integration.coast
2 |
3 | import com.monovore.coast.wire.Protocol
4 | import org.scalacheck.Gen
5 | import org.specs2.ScalaCheck
6 | import org.specs2.mutable._
7 |
8 | class SimpleIntegrationSpec extends Specification with ScalaCheck {
9 |
10 | sequential
11 |
12 | "a 'simple' samza-backed job" should {
13 |
14 | "count words" in {
15 |
16 | import com.monovore.example.coast.WordCount._
17 | import Protocol.simple._
18 |
19 | val words = Gen.oneOf("testing", "scandal", "riviera", "salad", "Thursday")
20 | val sentences = Gen.listOf(words).map { _.mkString(" ") }
21 | val sentenceList = Seq.fill(100)(sentences.sample).flatten.toSeq
22 |
23 | val input = Messages.add(Sentences, Map(0L -> sentenceList))
24 |
25 | val output = IntegrationTest.fuzz(graph, input, simple = true).get(WordCounts)
26 |
27 | val testingCount =
28 | sentenceList
29 | .flatMap { _.split(" ") }
30 | .count { _ == "testing" }
31 |
32 | output("testing") must_== (1 to testingCount)
33 | }
34 | }
35 | }
36 |
--------------------------------------------------------------------------------
/samza/src/main/scala/com/monovore/coast/samza/CoastSerdeFactory.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.coast.samza
2 |
3 | import com.monovore.coast.wire.Serializer
4 | import org.apache.samza.config.Config
5 | import org.apache.samza.serializers.{Serde, SerdeFactory}
6 |
7 | class CoastSerde[A](format: Serializer[A]) extends Serde[A] {
8 |
9 | override def fromBytes(bytes: Array[Byte]): A = format.fromArray(bytes)
10 |
11 | override def toBytes(value: A): Array[Byte] = format.toArray(value)
12 | }
13 |
14 | class CoastSerdeFactory[A] extends SerdeFactory[A] {
15 |
16 | override def getSerde(name: String, config: Config): Serde[A] = {
17 |
18 | val format = SerializationUtil.fromBase64[Serializer[A]](
19 | config.get(s"serializers.registry.$name.serialized.base64")
20 | )
21 |
22 | new CoastSerde[A](format)
23 | }
24 | }
--------------------------------------------------------------------------------
/samza/src/main/scala/com/monovore/coast/samza/CoastTask.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.coast
2 | package samza
3 |
4 | import org.apache.samza.config.Config
5 | import org.apache.samza.system._
6 | import org.apache.samza.task._
7 | import org.apache.samza.util.Logging
8 |
9 | class CoastTask extends StreamTask with InitableTask with WindowableTask with Logging {
10 |
11 | var collector: MessageCollector = _
12 |
13 | var receiver: CoastTask.Receiver = _
14 |
15 | override def init(config: Config, context: TaskContext): Unit = {
16 |
17 | info("Initializing CoastTask...")
18 |
19 | val factory = SerializationUtil.fromBase64[CoastTask.Factory](config.get(SamzaConfig.TaskKey))
20 |
21 | val finalReceiver = new CoastTask.Receiver {
22 |
23 | override def send(stream: SystemStream, partition: Int, offset: Long, key: Array[Byte], value: Array[Byte]) {
24 |
25 | val out = new OutgoingMessageEnvelope(stream, partition, key, value)
26 |
27 | collector.send(out)
28 | }
29 | }
30 |
31 | receiver = factory.make(config, context, finalReceiver)
32 |
33 | info("Initialization complete.")
34 | }
35 |
36 | override def process(
37 | envelope: IncomingMessageEnvelope,
38 | collector: MessageCollector,
39 | coordinator: TaskCoordinator
40 | ): Unit = {
41 |
42 | val stream = envelope.getSystemStreamPartition.getSystemStream
43 | val partition = envelope.getSystemStreamPartition.getPartition.getPartitionId
44 | val offset = envelope.getOffset.toLong
45 | val key = envelope.getKey.asInstanceOf[Array[Byte]]
46 | val message = envelope.getMessage.asInstanceOf[Array[Byte]]
47 |
48 | this.collector = collector
49 |
50 | receiver.send(stream, partition, offset, key, message)
51 |
52 | this.collector = null
53 | }
54 |
55 | override def window(collector: MessageCollector, coordinator: TaskCoordinator): Unit = {
56 |
57 | this.collector = collector
58 |
59 | receiver.window()
60 |
61 | this.collector = null
62 | }
63 | }
64 |
65 | object CoastTask {
66 |
67 | trait Receiver {
68 | def send(systemStream: SystemStream, partition: Int, offset: Long, key: Array[Byte], value: Array[Byte])
69 |
70 | def window() {}
71 | }
72 |
73 | trait Factory extends Serializable {
74 | def make(config: Config, context: TaskContext, receiver: Receiver): Receiver
75 | }
76 | }
77 |
--------------------------------------------------------------------------------
/samza/src/main/scala/com/monovore/coast/samza/ConfigGenerator.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.coast
2 | package samza
3 |
4 | import com.monovore.coast.core._
5 | import org.apache.samza.config.Config
6 | import wire.Serializer
7 |
8 | import scala.language.existentials
9 |
10 | trait ConfigGenerator {
11 | def configure(graph: Graph): Map[String, Config]
12 | }
13 |
14 | object ConfigGenerator {
15 |
16 | def isRegrouped[A, B](node: Node[A, B]): Boolean = node match {
17 | case _: Source[_, _] => false
18 | case clock: Clock => false
19 | case trans: Transform[_, _, _, _] => isRegrouped(trans.upstream)
20 | case merge: Merge[_, _] => merge.upstreams.exists { case (_, up) => isRegrouped(up) }
21 | case group: GroupBy[_, _, _] => true
22 | }
23 |
24 | def storageFor[A, B](element: Node[A, B], path: Path): Map[Path, Storage] = element match {
25 | case Source(_) => Map.empty
26 | case clock: Clock => Map.empty
27 | case PureTransform(up, _) => storageFor(up, path)
28 | case Merge(ups) => {
29 | ups.flatMap { case (branch, up) => storageFor(up, path / branch)}.toMap
30 | }
31 | case agg @ StatefulTransform(up, _, _) => {
32 | val upstreamed = storageFor(up, path.next)
33 | val storage = Storage(path.toString, agg.keyFormat, agg.stateFormat)
34 | upstreamed.updated(path, storage)
35 | }
36 | case GroupBy(up, _) => storageFor(up, path)
37 | }
38 |
39 | def sourcesFor[A, B](element: Node[A, B]): Set[String] = element match {
40 | case Source(name) => Set(name)
41 | case clock: Clock => sys.error("Clocks not implemented yet!")
42 | case Transform(up, _, _) => sourcesFor(up)
43 | case Merge(ups) => ups.flatMap { case (_, up) => sourcesFor(up) }.toSet
44 | case GroupBy(up, _) => sourcesFor(up)
45 | }
46 |
47 | case class Storage(name: String, keyString: Serializer[_], valueString: Serializer[_])
48 | }
49 |
--------------------------------------------------------------------------------
/samza/src/main/scala/com/monovore/coast/samza/Path.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.coast.samza
2 |
3 | case class Path(root: String, branches: Seq[String] = Vector.empty, index: Int = 0) {
4 |
5 | def /(subpath: String): Path = copy(branches = branches :+ subpath, index = 0)
6 |
7 | def next: Path = copy(index = index + 1)
8 |
9 | override def toString: String = {
10 | val suffix = Seq(index).filter { _ != 0 }.map { _.toString }
11 | ((root +: branches) ++ suffix).mkString(".")
12 | }
13 | }
14 |
--------------------------------------------------------------------------------
/samza/src/main/scala/com/monovore/coast/samza/SafeBackend.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.coast
2 | package samza
3 |
4 | import com.google.common.base.Charsets
5 | import core._
6 | import org.apache.samza.config.{Config, JobConfig, MapConfig, TaskConfig}
7 | import org.apache.samza.storage.kv.KeyValueStore
8 | import org.apache.samza.system.{SystemStream, SystemStreamPartition}
9 | import org.apache.samza.task.TaskContext
10 | import org.apache.samza.util.Logging
11 | import org.apache.samza.{Partition, SamzaException}
12 | import safe._
13 |
14 | import collection.JavaConverters._
15 |
16 | object SafeBackend extends SamzaBackend {
17 |
18 | def apply(baseConfig: Config = new MapConfig()): ConfigGenerator = new SafeConfigGenerator(baseConfig)
19 |
20 | class SinkFactory[A, B](
21 | system: String,
22 | mergeStream: String,
23 | checkpointStream: String,
24 | sinkNode: Sink[A, B]
25 | ) extends CoastTask.Factory with Logging {
26 |
27 | override def make(config: Config, context: TaskContext, whatSink: CoastTask.Receiver): CoastTask.Receiver = {
28 |
29 | val streamName = config.get(SamzaConfig.TaskName)
30 |
31 | val partitionIndex = context.getTaskName.getTaskName.split("\\W+").last.toInt // ICK!
32 |
33 | info(s"Initializing safe coast backend for task [$streamName/$partitionIndex]")
34 |
35 | val regroupedStreams = config.get(SamzaConfig.RegroupedStreams).split(",")
36 | .filter { _.nonEmpty }
37 | .toSet
38 |
39 | val partitions = SamzaBackend.getPartitions(config, system, streamName)
40 |
41 | val offsetThreshold =
42 | if (regroupedStreams(streamName)) 0L
43 | else {
44 | val offset = partitions(partitionIndex)
45 | info(s"Downstream offset of $offset for [$streamName/$partitionIndex]")
46 | offset
47 | }
48 |
49 | val finalSink = (offset: Long, key: A, value: B) => {
50 |
51 | val keyBytes = sinkNode.keyFormat.toArray(key)
52 | val valueBytes = sinkNode.valueFormat.toArray(value)
53 |
54 | if (offset >= offsetThreshold) {
55 |
56 | if (regroupedStreams(streamName)) {
57 | val qualifier = partitionIndex.toString.getBytes(Charsets.UTF_8)
58 | val payload = Messages.InternalMessage.binaryFormat.toArray(qualifier, offset, valueBytes)
59 | val newPartition = sinkNode.keyPartitioner.partition(key, partitions.size)
60 | whatSink.send(new SystemStream(system, streamName), newPartition, -1, keyBytes, payload)
61 | }
62 | else {
63 | whatSink.send(new SystemStream(system, streamName), partitionIndex, offset, keyBytes, valueBytes)
64 | }
65 | }
66 |
67 | offset + 1
68 | }
69 |
70 | val compiler = new TaskCompiler(new TaskCompiler.Context {
71 | override def getStore[A, B](path: String, default: B): CoastState[A, B] =
72 | context.getStore(path).asInstanceOf[CoastStorageEngine[A, B]].withDefault(default)
73 | })
74 |
75 | val compiled = compiler.compile(sinkNode.element, finalSink, streamName)
76 |
77 | val checkpointStore = context.getStore(checkpointStream).asInstanceOf[KeyValueStore[Unit, Checkpoint]]
78 |
79 | var checkpoint: Checkpoint = Option(checkpointStore.get(unit)).getOrElse(Checkpoint(Map.empty, 0L, Map.empty))
80 |
81 | info(s"Restoring task [$streamName/$partitionIndex] to checkpoint: $checkpoint")
82 |
83 | checkpoint.inputStreams
84 | .foreach { case ((name, p), state) =>
85 | context.setStartingOffset(new SystemStreamPartition(system, name, new Partition(p)), state.offset.toString)
86 | }
87 |
88 | context.setStartingOffset(new SystemStreamPartition(system, mergeStream, new Partition(partitionIndex)), checkpoint.mergeOffset.toString)
89 |
90 | var mergeTip = checkpoint.mergeOffset
91 |
92 | new CoastTask.Receiver {
93 |
94 | import Checkpoint._
95 |
96 | override def send(systemStream: SystemStream, partition: Int, offset: Long, key: Array[Byte], value: Array[Byte]) {
97 |
98 | val stream = systemStream.getStream
99 |
100 | if (stream == mergeStream) {
101 | mergeTip = math.max(mergeTip, offset + 1)
102 | }
103 | else {
104 |
105 | val state = checkpoint.inputStreams.getOrElse(stream -> partition, InputState.default)
106 |
107 | if (offset >= state.offset) {
108 |
109 | if (mergeTip <= checkpoint.mergeOffset) {
110 |
111 | val mergeMessage = Messages.MergeInfo.binaryFormat.toArray(stream, partition, offset)
112 |
113 | whatSink.send(new SystemStream(system, mergeStream), partitionIndex, checkpoint.mergeOffset, Array.empty, mergeMessage)
114 | }
115 |
116 | val (qualifier, qualifierOffset, message) =
117 | if (regroupedStreams(stream)) Messages.InternalMessage.binaryFormat.fromArray(value)
118 | else (Array.empty[Byte], offset, value)
119 |
120 | val qualifierThreshold = state.qualifiers.getOrElse(qualifier, 0L)
121 |
122 | checkpoint =
123 | if (qualifierOffset >= qualifierThreshold) {
124 |
125 | val result = compiled
126 | .filter { _.inputStream == stream }
127 | .foldLeft(checkpoint.outputStreams) { (current, dispatch) =>
128 | val offset = current.getOrElse(dispatch.downstreamPath, 0L)
129 | val nextOffset = dispatch.handler(offset, key, message)
130 | current.updated(dispatch.downstreamPath, nextOffset)
131 | }
132 |
133 | checkpoint.copy(
134 | inputStreams = checkpoint.inputStreams.updated(stream -> partition,
135 | state.copy(
136 | offset = offset + 1,
137 | qualifiers = state.qualifiers.updated(qualifier, qualifierOffset + 1)
138 | )
139 | ),
140 | mergeOffset = checkpoint.mergeOffset + 1,
141 | outputStreams = result
142 | )
143 | }
144 | else {
145 | checkpoint.copy(
146 | inputStreams = checkpoint.inputStreams.updated(stream -> partition,
147 | state.copy(offset = offset + 1)
148 | ),
149 | mergeOffset = checkpoint.mergeOffset + 1
150 | )
151 | }
152 | }
153 | }
154 | }
155 |
156 | override def window(): Unit = {
157 | debug(s"Checkpointing task [$streamName/$partitionIndex] at: $checkpoint")
158 | checkpointStore.put(unit, checkpoint)
159 | }
160 | }
161 | }
162 | }
163 | }
164 |
165 | class SafeConfigGenerator(baseConfig: Config = new MapConfig()) extends ConfigGenerator {
166 |
167 | import ConfigGenerator._
168 | import SamzaConfig.className
169 |
170 | val base = SamzaConfig.Base(baseConfig)
171 |
172 | def configure(graph: Graph): Map[String, Config] = {
173 |
174 | val baseConfigMap = baseConfig.asScala.toMap
175 |
176 | val regrouped = graph.bindings
177 | .flatMap { case (name, sink) =>
178 | Some(name).filter { _ => isRegrouped(sink.element) }
179 | }
180 | .toSet
181 |
182 | val configs = graph.bindings.map { case (name -> sink) =>
183 |
184 | val mergeStream = base.merge(name)
185 | val checkpoint = base.checkpoint(name)
186 |
187 | val inputs = sourcesFor(sink.element)
188 |
189 | val statePaths = storageFor(sink.element, Path(name))
190 |
191 | val changelogDelays = statePaths
192 | .map { case (path, storage) => base.changelog(storage.name) -> (path.branches.size + 2) }
193 |
194 | val delays = changelogDelays ++ Map(
195 | mergeStream -> 0,
196 | name -> 1,
197 | checkpoint -> ((1 +: changelogDelays.values.toSeq).max + 1)
198 | )
199 |
200 | val factory: CoastTask.Factory = new SafeBackend.SinkFactory(base.system, mergeStream, checkpoint, sink)
201 |
202 | val configMap = Map(
203 |
204 | // Job
205 | JobConfig.JOB_NAME -> name,
206 |
207 | // Task
208 | TaskConfig.TASK_CLASS -> className[CoastTask],
209 | TaskConfig.INPUT_STREAMS -> (inputs + mergeStream).map { i => s"${base.system}.$i" }.mkString(","),
210 | TaskConfig.MESSAGE_CHOOSER_CLASS_NAME -> className[MergingChooserFactory],
211 | TaskConfig.WINDOW_MS -> base.windowMs,
212 |
213 | // No-op checkpoints!
214 | TaskConfig.CHECKPOINT_MANAGER_FACTORY -> className[StaticCheckpointManagerFactory],
215 | TaskConfig.COMMIT_MS -> "-1",
216 |
217 | // Kafka system
218 | s"systems.${base.system}.samza.offset.default" -> "oldest",
219 | s"systems.${base.system}.producer.producer.type" -> "sync",
220 | s"systems.${base.system}.producer.message.send.max.retries" -> "0",
221 | s"systems.${base.system}.producer.request.required.acks" -> "1",
222 | s"systems.${base.system}.samza.factory" -> className[CoastKafkaSystemFactory],
223 |
224 | // Merge info
225 | s"systems.${base.system}.streams.$mergeStream.merge" -> inputs.map { i => s"${base.system}.$i" }.mkString(","),
226 | s"systems.${base.system}.streams.$mergeStream.samza.bootstrap" -> "true",
227 | s"systems.${base.system}.streams.$mergeStream.samza.priority" -> "0",
228 |
229 | // Coast-specific
230 | SamzaConfig.TaskKey -> SerializationUtil.toBase64(factory),
231 | SamzaConfig.TaskName -> name,
232 | SamzaConfig.RegroupedStreams -> regrouped.mkString(",")
233 | )
234 |
235 |
236 | val storageMap = statePaths
237 | .flatMap { case (path, storage) =>
238 |
239 | val baseConfig = base.storageConfig(storage)
240 |
241 | val factoryKey = s"stores.${storage.name}.factory"
242 |
243 | baseConfig ++ Map(
244 | factoryKey -> className[CoastStoreFactory[_, _]],
245 | s"stores.${storage.name}.subfactory" -> {
246 | baseConfig.getOrElse(factoryKey, throw new SamzaException(s"Missing storage factory for ${storage.name}."))
247 | }
248 | )
249 | }
250 | .toMap
251 |
252 | val streamConfig = delays
253 | .map { case (stream, delay) =>
254 | s"systems.${base.system}.streams.$stream.delay" -> delay.toString
255 | }
256 |
257 | val checkpointConf = {
258 |
259 | base.storageConfig(Storage(checkpoint, Checkpoint.keyFormat, Checkpoint.format)) ++ Map(
260 | s"stores.$checkpoint.changelog" -> s"${base.system}.$checkpoint",
261 | s"stores.$checkpoint.type" -> "checkpoint"
262 | )
263 | }
264 |
265 | name -> new MapConfig(
266 | (baseConfigMap ++ configMap ++ storageMap ++ streamConfig ++ checkpointConf).asJava
267 | )
268 | }
269 |
270 | configs.toMap
271 | }
272 | }
--------------------------------------------------------------------------------
/samza/src/main/scala/com/monovore/coast/samza/SamzaApp.scala:
--------------------------------------------------------------------------------
1 | package com.monovore.coast.samza
2 |
3 | import java.io.{File, FileOutputStream}
4 | import java.util.Properties
5 |
6 | import com.google.common.base.Charsets
7 | import com.google.common.io.Files
8 | import com.monovore.coast.core.Graph
9 | import com.monovore.coast.viz.Dot
10 | import joptsimple.{OptionParser, OptionSet}
11 | import org.apache.samza.config.Config
12 | import org.apache.samza.job.JobRunner
13 | import org.apache.samza.util.{CommandLine, Logging}
14 |
15 | import scala.collection.JavaConverters._
16 |
17 | /**
18 | * A superclass that provides a basic CLI for basic Samza apps.
19 | *
20 | * @param backend The Samza backend this app should use. (eg. `SimpleBackend`
21 | * or `SafeBackend`)
22 | */
23 | abstract class SamzaApp(backend: SamzaBackend) extends Logging {
24 |
25 | import SamzaApp._
26 |
27 | /**
28 | * The dataflow graph for this job.
29 | */
30 | def graph: Graph
31 |
32 | def main(args: Array[String]): Unit = {
33 |
34 | args.toList match {
35 | case "dot" :: rest => {
36 |
37 | val dotParser = new OptionParser()
38 |
39 | dotParser
40 | .accepts("to-file", "Print the graph to the specified file, instead of stdout.")
41 | .withRequiredArg()
42 |
43 | val opts = dotParser.parse(rest: _*)
44 |
45 | (Option(opts.valueOf("to-file")): @unchecked) match {
46 | case Some(fileName: String) => Files.asCharSink(new File(fileName), Charsets.UTF_8).write(Dot.describe(graph))
47 | case None => println(Dot.describe(graph))
48 | }
49 | }
50 | case "gen-config" :: rest => {
51 |
52 | val opts = samzaCmd.parser.parse(rest: _*)
53 | val configs = withBaseConfig(opts)
54 |
55 | val targetDirectory = (Option(opts.valueOf("target-directory")): @unchecked) match {
56 | case Some(target: String) => {
57 | val targetDir = new File(target)
58 | if (!targetDir.isDirectory) {
59 | Console.err.println(s"Path $target is not a directory!")
60 | sys.exit(1)
61 | }
62 | targetDir
63 | }
64 | case None => {
65 | System.err.println("No target directory for config!")
66 | samzaCmd.parser.printHelpOn(System.err)
67 | sys.exit(1)
68 | }
69 | }
70 |
71 | generateConfigFiles(targetDirectory, configs)
72 | }
73 | case "run" :: rest => {
74 |
75 | val opts = samzaCmd.parser.parse(rest: _*)
76 | val configs = withBaseConfig(opts)
77 |
78 | for ((name, config) <- configs) {
79 | info(s"Launching samza job: $name")
80 | new JobRunner(config).run()
81 | }
82 | }
83 | case "info" :: rest => {
84 |
85 | val opts = samzaCmd.parser.parse(rest: _*)
86 | val configs = withBaseConfig(opts)
87 |
88 | for ((name, config) <- configs) {
89 | print(jobInfo(name, config))
90 | }
91 | }
92 | case unknown => Console.err.println("Available commands: dot, run, info, gen-config")
93 | }
94 | }
95 |
96 | private[this] val samzaCmd = {
97 |
98 | val cmd = new CommandLine
99 |
100 | cmd.parser
101 | .accepts("job", "Executes the command on the Samza job with the given name. Repeat to specify multiple jobs, or omit to run on all jobs.")
102 | .withRequiredArg()
103 | .describedAs("job-name")
104 |
105 | cmd.parser
106 | .accepts("target-directory", "When generating config, write it to the specified directory.")
107 | .withRequiredArg()
108 | .describedAs("./target/directory")
109 |
110 | cmd
111 | }
112 |
113 | private[this] def withBaseConfig(options: OptionSet): Map[String, Config] = {
114 |
115 | val baseConfig = samzaCmd.loadConfig(options)
116 |
117 | val jobFilter =
118 | Option(options.valuesOf("job"))
119 | .map { _.asScala.collect { case job: String => job }.toSet }
120 | .filter { _.nonEmpty }
121 | .getOrElse { _: String => true }
122 |
123 | backend(baseConfig).configure(graph)
124 | .filter { case (name, _) => jobFilter(name) }
125 | }
126 | }
127 |
128 | object SamzaApp {
129 |
130 | val helpText =
131 | """Invalid arguments! Try one of the following commands:
132 | |
133 | | dot
134 | | Print out a representation of the job graph in GraphViz's 'dot' format.
135 | |
136 | | gen-config