├── .gitignore ├── .travis.yml ├── LICENSE.txt ├── README.md ├── build.sbt ├── modules ├── core │ └── src │ │ └── main │ │ └── scala │ │ └── domsj │ │ └── streamy │ │ └── db │ │ ├── HeartBeatGenerator.scala │ │ ├── TopicsCreator.scala │ │ ├── TransactionGenerator.scala │ │ └── package.scala └── runners │ ├── beam │ └── src │ │ └── main │ │ └── scala │ │ └── domsj │ │ └── streamy │ │ └── db │ │ └── beam │ │ ├── StreamyDb.scala │ │ └── package.scala │ └── flink │ └── src │ └── main │ └── scala │ └── domsj │ └── streamy │ └── db │ └── flink │ ├── StreamyDb.scala │ └── package.scala └── project └── build.properties /.gitignore: -------------------------------------------------------------------------------- 1 | .idea/ 2 | target/ 3 | -------------------------------------------------------------------------------- /.travis.yml: -------------------------------------------------------------------------------- 1 | dist: xenial 2 | language: scala 3 | scala: 4 | - 2.12.7 5 | -------------------------------------------------------------------------------- /LICENSE.txt: -------------------------------------------------------------------------------- 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 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | Streamy-db ![](https://travis-ci.com/domsj/streamy-db.svg?branch=master) 2 | -- 3 | 4 | Streamy-db is a deterministic streaming database implementation. 5 | It should (in theory) be scalable and low latency. 6 | 7 | It adapts [Calvin](http://cs-www.cs.yale.edu/homes/dna/papers/calvin-sigmod12.pdf) onto a stream processing framework. 8 | 9 | Currently there is scala code for both [flink](https://flink.apache.org/) and [beam](https://beam.apache.org/) (using [scio](https://github.com/spotify/scio)). 10 | 11 | This is only a prototype / proof of concept. 12 | 13 | A blog post explaining the contents of the repo is [available](https://domsj.info/2018/12/30/introducing-streamy-db.html). 14 | -------------------------------------------------------------------------------- /build.sbt: -------------------------------------------------------------------------------- 1 | import sbt.Keys.libraryDependencies 2 | 3 | name := "streamy-db" 4 | 5 | val scioVersion = "0.7.0-beta3" 6 | val beamVersion = "2.9.0" 7 | val flinkVersion = "1.7.0" 8 | val akkaVersion = "2.5.19" 9 | val kafkaVersion = "1.1.1" 10 | 11 | ThisBuild / organization := "domsj.streamy.db" 12 | ThisBuild / version := "0.1.0-SNAPSHOT" 13 | ThisBuild / scalaVersion := "2.12.8" 14 | 15 | lazy val core = (project in file("modules/core")) 16 | .settings( 17 | name := "streamy-db-core", 18 | libraryDependencies ++= Seq( 19 | "com.typesafe.akka" %% "akka-actor-typed" % akkaVersion, 20 | "org.apache.kafka" %% "kafka" % kafkaVersion, 21 | "com.typesafe.scala-logging" %% "scala-logging" % "3.9.0", 22 | "org.slf4j" % "slf4j-simple" % "1.7.25", 23 | "com.lihaoyi" %% "upickle" % "0.7.1", 24 | ) 25 | ) 26 | 27 | lazy val beamRunner = (project in file("modules/runners/beam")) 28 | .dependsOn(core) 29 | .settings( 30 | name := "streamy-db-beam-runner", 31 | libraryDependencies ++= Seq( 32 | "org.apache.beam" % "beam-runners-direct-java" % beamVersion, 33 | "org.apache.beam" % "beam-sdks-java-io-kafka" % beamVersion, 34 | "com.spotify" %% "scio-core" % scioVersion, 35 | ) 36 | ) 37 | 38 | lazy val flinkRunner = (project in file("modules/runners/flink")) 39 | .dependsOn(core) 40 | .settings( 41 | name := "streamy-db-flink-runner", 42 | libraryDependencies ++= Seq( 43 | "org.apache.flink" % "flink-core" % flinkVersion, 44 | "org.apache.flink" %% "flink-streaming-scala" % flinkVersion, 45 | "org.apache.flink" %% "flink-connector-kafka-0.11" % flinkVersion, 46 | ) 47 | ) 48 | 49 | -------------------------------------------------------------------------------- /modules/core/src/main/scala/domsj/streamy/db/HeartBeatGenerator.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 @domsj 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, 11 | * software distributed under the License is distributed on an 12 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 13 | * KIND, either express or implied. See the License for the 14 | * specific language governing permissions and limitations 15 | * under the License. 16 | */ 17 | 18 | package domsj.streamy.db 19 | 20 | import java.util.concurrent.TimeUnit 21 | 22 | import akka.actor.typed.{ActorSystem, Behavior, PostStop, Signal} 23 | import akka.actor.typed.scaladsl.{AbstractBehavior, ActorContext, Behaviors} 24 | import com.typesafe.scalalogging.Logger 25 | import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord, RecordMetadata} 26 | import org.apache.kafka.common.PartitionInfo 27 | 28 | import scala.concurrent.duration.FiniteDuration 29 | 30 | 31 | object HeartBeatGenerator extends App { 32 | val logger = Logger(HeartBeatGenerator.getClass) 33 | 34 | val kafkaProperties = makeKafkaProperties() 35 | kafkaProperties.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer") 36 | kafkaProperties.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer") 37 | 38 | val producer = new KafkaProducer[String, String](kafkaProperties) 39 | 40 | val duration = FiniteDuration(5000, TimeUnit.MILLISECONDS) 41 | 42 | object PartitionHeartBeatActor { 43 | sealed trait Msg 44 | case object Timeout extends Msg 45 | case object KafkaPushed extends Msg 46 | 47 | def apply(partitionInfo: PartitionInfo): Behavior[Msg] = 48 | Behaviors.withTimers(timerScheduler => { 49 | timerScheduler.startSingleTimer(Timeout, Timeout, duration) 50 | 51 | Behaviors.receive[Msg]((ctx, msg) => { 52 | msg match { 53 | case Timeout => 54 | logger.info("pushing heartbeat message for partition {}", partitionInfo) 55 | val record = new ProducerRecord[String, String]( 56 | transactionInputsTopic, 57 | partitionInfo.partition(), 58 | "heartbeat", 59 | upickle.default.write[List[Transaction]](Nil) 60 | ) 61 | val self = ctx.self 62 | producer.send(record, (_: RecordMetadata, _: Exception) => { 63 | logger.debug("kafka onCompletion") 64 | self.tell(KafkaPushed) 65 | }) 66 | Behaviors.same 67 | case KafkaPushed => 68 | logger.debug("pushed message to kafka, starting another timer") 69 | timerScheduler.startSingleTimer(Timeout, Timeout, duration) 70 | Behaviors.same 71 | } 72 | }) 73 | }) 74 | } 75 | 76 | val partitionInfos = producer.partitionsFor(transactionInputsTopic) 77 | 78 | object HeartBeatActor { 79 | def apply(): Behavior[Nothing] = 80 | Behaviors.setup[Nothing](context ⇒ new HeartBeatActor(context)) 81 | } 82 | 83 | class HeartBeatActor(context: ActorContext[Nothing]) extends AbstractBehavior[Nothing] { 84 | context.log.info("HeartBeatActor started") 85 | 86 | partitionInfos.forEach(partitionInfo => 87 | context.spawn( 88 | PartitionHeartBeatActor(partitionInfo), 89 | "%s_%s".format(partitionInfo.topic(), partitionInfo.partition()) 90 | ) 91 | ) 92 | 93 | override def onMessage(msg: Nothing): Behavior[Nothing] = { 94 | // No need to handle any messages 95 | Behaviors.unhandled 96 | } 97 | 98 | override def onSignal: PartialFunction[Signal, Behavior[Nothing]] = { 99 | case PostStop ⇒ 100 | context.log.info("HeartBeat Application stopped") 101 | this 102 | } 103 | } 104 | 105 | ActorSystem[Nothing](HeartBeatActor(), "heart-beats") 106 | } -------------------------------------------------------------------------------- /modules/core/src/main/scala/domsj/streamy/db/TopicsCreator.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 @domsj 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, 11 | * software distributed under the License is distributed on an 12 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 13 | * KIND, either express or implied. See the License for the 14 | * specific language governing permissions and limitations 15 | * under the License. 16 | */ 17 | 18 | package domsj.streamy.db 19 | 20 | import java.util.Collections 21 | 22 | import com.typesafe.scalalogging.Logger 23 | import org.apache.kafka.clients.admin.{AdminClient, NewTopic} 24 | import org.apache.kafka.common.config.TopicConfig 25 | 26 | import scala.collection.JavaConverters._ 27 | 28 | object TopicsCreator extends App { 29 | val logger = Logger(TopicsCreator.getClass) 30 | 31 | // TODO change replication factor! 32 | val replicationFactor : Short = 1 33 | val inputsTopic = new NewTopic(transactionInputsTopic, 4, replicationFactor) 34 | 35 | // https://cwiki.apache.org/confluence/display/KAFKA/KIP-32+-+Add+timestamps+to+Kafka+message 36 | // The good things about LogAppendTime are: 1) ... 2) Monotonically increasing. 37 | inputsTopic.configs(Map(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG -> "LogAppendTime").asJava) 38 | 39 | val adminClient = AdminClient.create(makeKafkaProperties()) 40 | val result = adminClient.createTopics(Collections.singleton(inputsTopic)) 41 | 42 | result.all().get() 43 | } -------------------------------------------------------------------------------- /modules/core/src/main/scala/domsj/streamy/db/TransactionGenerator.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 @domsj 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, 11 | * software distributed under the License is distributed on an 12 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 13 | * KIND, either express or implied. See the License for the 14 | * specific language governing permissions and limitations 15 | * under the License. 16 | */ 17 | 18 | package domsj.streamy.db 19 | 20 | import java.util.UUID 21 | 22 | import com.typesafe.scalalogging.Logger 23 | import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} 24 | 25 | import scala.util.Random 26 | 27 | object TransactionGenerator { 28 | val logger = Logger(TransactionGenerator.getClass) 29 | 30 | def main(cmdLineArgs: Array[String]): Unit = { 31 | 32 | val kafkaProperties = makeKafkaProperties() 33 | kafkaProperties.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer") 34 | kafkaProperties.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer") 35 | 36 | val producer = new KafkaProducer[String, String](kafkaProperties) 37 | 38 | val push_transaction = (transaction : Transaction) => { 39 | logger.debug("pushing transaction {}", transaction) 40 | producer.send(new ProducerRecord[String, String](transactionInputsTopic, upickle.default.write(List(transaction)))).get() 41 | } 42 | 43 | val numKeys = 1000000 44 | val generateKey = () => "key_%010d".format(Random.nextInt(numKeys)) 45 | 46 | val keysPerTransaction = 4 47 | val numTransactions = 10 48 | 49 | for (_ <- Range(0, numTransactions)) { 50 | val transactionId = UUID.randomUUID().toString 51 | val keys = Seq.fill(keysPerTransaction)(generateKey()) 52 | 53 | val asserts = keys.map(k => KeyValueOption(k, None)) 54 | val updates = keys.map(k => KeyValueOption(k, Some(k + "_value"))) 55 | 56 | val transaction = Transaction(transactionId, asserts.toList, updates.toList) 57 | 58 | push_transaction(transaction) 59 | } 60 | } 61 | } -------------------------------------------------------------------------------- /modules/core/src/main/scala/domsj/streamy/db/package.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 @domsj 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, 11 | * software distributed under the License is distributed on an 12 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 13 | * KIND, either express or implied. See the License for the 14 | * specific language governing permissions and limitations 15 | * under the License. 16 | */ 17 | 18 | package domsj.streamy 19 | 20 | import java.util.Properties 21 | 22 | import upickle.default._ 23 | 24 | package object db { 25 | 26 | type Key = String 27 | type Value = String 28 | type ValueOption = Option[Value] 29 | 30 | case class KeyValueOption(key: Key, valueOption: ValueOption) 31 | object KeyValueOption{ 32 | implicit val rw: ReadWriter[KeyValueOption] = macroRW 33 | } 34 | 35 | type TransactionId = String 36 | 37 | sealed trait TransactionProcessorMessage { 38 | def transactionId: TransactionId 39 | } 40 | 41 | case class Transaction(transactionId: TransactionId, asserts: List[KeyValueOption], updates: List[KeyValueOption]) extends TransactionProcessorMessage 42 | object Transaction{ 43 | implicit val rw = macroRW[Transaction] 44 | } 45 | 46 | case class ReadResult(transactionId: TransactionId, key: Key, valueOption: ValueOption) extends TransactionProcessorMessage 47 | 48 | case class TransactionResult(transaction: Transaction, succeeded: Boolean) 49 | object TransactionResult { 50 | implicit val rw = macroRW[TransactionResult] 51 | } 52 | 53 | sealed trait KeyProcessorMessage extends Product with Serializable { 54 | def key: Key 55 | } 56 | case class ReadRequest(transactionId: TransactionId, key: Key) extends KeyProcessorMessage 57 | case class LockRequest(transactionId: TransactionId, key: Key) extends KeyProcessorMessage 58 | case class KeyTransactionResult(transactionId: TransactionId, key: Key, valueOptionOption: Option[ValueOption]) extends KeyProcessorMessage 59 | 60 | 61 | val transactionInputsTopic = "transaction-inputs" 62 | val transactionResultsTopic = "transaction-results" 63 | 64 | def makeKafkaProperties() : Properties = { 65 | val kafkaProperties= new Properties() 66 | kafkaProperties.setProperty("bootstrap.servers", "localhost:9092") 67 | kafkaProperties 68 | } 69 | 70 | } -------------------------------------------------------------------------------- /modules/runners/beam/src/main/scala/domsj/streamy/db/beam/StreamyDb.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 @domsj 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, 11 | * software distributed under the License is distributed on an 12 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 13 | * KIND, either express or implied. See the License for the 14 | * specific language governing permissions and limitations 15 | * under the License. 16 | */ 17 | 18 | package domsj.streamy.db.beam 19 | 20 | import com.spotify.scio.ContextAndArgs 21 | import com.spotify.scio.coders._ 22 | import com.spotify.scio.values.WindowOptions 23 | import domsj.streamy.db._ 24 | import org.apache.beam.sdk.io.kafka.KafkaIO 25 | import org.apache.beam.sdk.state._ 26 | import org.apache.beam.sdk.transforms.DoFn 27 | import org.apache.beam.sdk.transforms.DoFn.{OnTimer, ProcessElement, StateId, TimerId} 28 | import org.apache.beam.sdk.transforms.windowing.TimestampCombiner 29 | import org.apache.beam.sdk.values.KV 30 | import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode 31 | import org.apache.kafka.common.serialization.StringSerializer 32 | import org.joda.time.Instant 33 | 34 | object StreamyDb { 35 | 36 | class KeyedEventTimeSorter[K : Coder, V : Coder] extends DoFn[KV[K, V], KV[K, List[V]]] { 37 | @StateId("elements") private val elementsSpec = StateSpecs.map[Instant, KV[K, List[V]]]( 38 | CoderMaterializer.beamWithDefault(Coder[Instant]), 39 | CoderMaterializer.beamWithDefault(Coder[KV[K, List[V]]]) 40 | ) 41 | @TimerId("timer") private val timerSpec = TimerSpecs.timer(TimeDomain.EVENT_TIME) 42 | 43 | @ProcessElement 44 | def processElement(context: ProcessContext, 45 | @StateId("elements") elements : MapState[Instant, KV[K, List[V]]], 46 | @TimerId("timer") timer: Timer 47 | ): Unit = { 48 | val timestamp = context.timestamp() 49 | val inputElement = context.element() 50 | val outputElement = Option(elements.get(timestamp).read()) match { 51 | case Some(v) => KV.of(inputElement.getKey, v.getValue.+:(inputElement.getValue)) 52 | case None => KV.of(inputElement.getKey, List(inputElement.getValue)) 53 | } 54 | elements.put(timestamp, outputElement) 55 | timer.set(timestamp) 56 | } 57 | 58 | @OnTimer("timer") 59 | def onTimer(context: OnTimerContext, 60 | @StateId("elements") elements : MapState[Instant, KV[K, List[V]]] 61 | ): Unit = { 62 | // NOTE should be able to get key : K from OnTimerContext, but beam isn't awesome enough 63 | val timestamp = context.timestamp() 64 | val element = elements.get(timestamp).read() 65 | context.outputWithTimestamp(element, timestamp) 66 | elements.remove(timestamp) 67 | } 68 | } 69 | 70 | class KeyTransactionProcessor extends DoFn[KV[Key, KeyProcessorMessage], KV[Key, ReadResult]] { 71 | @StateId("value") private val valueSpec = StateSpecs.value[Value]() 72 | @StateId("write-lock-waiters") private val writeLockWaitersSpec = StateSpecs.map[TransactionId, List[TransactionId]]() 73 | @StateId("latest-write-lock") private val latestWriteLockSpec = StateSpecs.value[TransactionId]() 74 | 75 | 76 | @ProcessElement 77 | def processElement(context: ProcessContext, 78 | @StateId("value") valueState: ValueState[Value], 79 | @StateId("write-lock-waiters") writeLockWaiters: MapState[TransactionId, List[TransactionId]], 80 | @StateId("latest-write-lock") latestWriteLock: ValueState[TransactionId] 81 | ): Unit = { 82 | 83 | context.element().getValue match { 84 | 85 | case LockRequest(transactionId, key) => 86 | latestWriteLock.write(transactionId) 87 | writeLockWaiters.put(transactionId, Nil) 88 | 89 | case ReadRequest(transactionId, key) => 90 | val latestWriteLockTransactionId = latestWriteLock.read() 91 | if (latestWriteLockTransactionId == null) { 92 | // no write locks, so can read value immediately 93 | val readResult = ReadResult(transactionId, context.element().getKey, Option(valueState.read())) 94 | context.outputWithTimestamp(KV.of(key, readResult), context.timestamp()) 95 | } else { 96 | // one or more write locks in the queue, register with the last one 97 | val waiters = writeLockWaiters.get(latestWriteLockTransactionId).read() 98 | writeLockWaiters.put(latestWriteLockTransactionId, waiters.::(transactionId)) 99 | } 100 | 101 | case KeyTransactionResult(transactionId, key, valueOptionOption) => 102 | 103 | val waiters = writeLockWaiters.get(transactionId).read() 104 | writeLockWaiters.remove(transactionId) 105 | 106 | if (waiters != null) { 107 | 108 | // maybe update state 109 | valueOptionOption match { 110 | case Some(valueOption) => 111 | valueOption match { 112 | case Some(value) => valueState.write(value) 113 | case None => valueState.clear() 114 | } 115 | case None => () 116 | } 117 | 118 | val maybeValue = Option(valueState.read()) 119 | waiters.foreach(transactionId => context.output(KV.of(key, ReadResult(transactionId, key, maybeValue)))) 120 | } 121 | 122 | if (latestWriteLock.read() == transactionId) { 123 | latestWriteLock.clear() 124 | } 125 | } 126 | } 127 | } 128 | 129 | class TransactionProcessor extends DoFn[KV[TransactionId, TransactionProcessorMessage], KV[TransactionId, TransactionResult]] { 130 | @StateId("transaction") private val transactionSpec = StateSpecs.value[Transaction](CoderMaterializer.beamWithDefault(Coder[Transaction])) 131 | @StateId("read-results") private val readResultsSpec = StateSpecs.map[Key, ValueOption]() 132 | @StateId("read-results-count") private val readResultsCountSpec = StateSpecs.value[Int](CoderMaterializer.beamWithDefault(Coder[Int])) 133 | 134 | @ProcessElement 135 | def processElement(context: ProcessContext, 136 | @StateId("transaction") transactionState: ValueState[Transaction], 137 | @StateId("read-results") readResultsState: MapState[Key, ValueOption], 138 | @StateId("read-results-count") readResultsCountState: ValueState[Int] 139 | ): Unit = { 140 | val getReadResultsCount = () => Option(readResultsCountState.read()).getOrElse(0) 141 | 142 | // gather everything into the state 143 | val message = context.element().getValue 144 | message match { 145 | case Transaction(_, _, _) => 146 | transactionState.write(message.asInstanceOf[Transaction]) 147 | case ReadResult(_, key, valueOption) => 148 | readResultsState.put(key, valueOption) 149 | readResultsCountState.write(getReadResultsCount() + 1) 150 | } 151 | 152 | // process the transaction once the state is complete 153 | // NOTE there is an optimisation opportunity for for failing transactions by processing ReadResults more incremental 154 | val transaction = transactionState.read() 155 | if (transaction != null && transaction.asserts.size == getReadResultsCount()) { 156 | val assertsAllSucceed = 157 | transaction 158 | .asserts 159 | .forall((keyValueOption: KeyValueOption) => 160 | readResultsState.get(keyValueOption.key).read() == keyValueOption.valueOption) 161 | 162 | context.output(KV.of(transaction.transactionId, TransactionResult(transaction, assertsAllSucceed))) 163 | } 164 | } 165 | } 166 | 167 | def main(cmdlineArgs: Array[String]): Unit = { 168 | val (sc, args) = ContextAndArgs(cmdlineArgs) 169 | 170 | 171 | val transactionsInput = 172 | sc.customInput("transactions-input", makeKafkaIOWithLogAppendTime(transactionInputsTopic)) 173 | .map(v => upickle.default.read[List[Transaction]](v.getValue)) 174 | .withGlobalWindow( 175 | WindowOptions( 176 | timestampCombiner = TimestampCombiner.LATEST, 177 | accumulationMode = AccumulationMode.DISCARDING_FIRED_PANES, 178 | // trigger = Trigger.AfterAny.newBuilder().build() 179 | ) 180 | ) 181 | .flatMap(identity) 182 | 183 | val keyTransactionResults = 184 | sc.customInput("transaction-results-input", makeKafkaIOWithLogAppendTime(transactionResultsTopic)) 185 | .map(v => upickle.default.read[TransactionResult](v.getValue)) 186 | .flatMap(tr => 187 | tr.transaction.updates.map(kvo => 188 | KeyTransactionResult( 189 | tr.transaction.transactionId, 190 | kvo.key, 191 | if (tr.succeeded) { Some(kvo.valueOption) } else { None }) 192 | .asInstanceOf[KeyProcessorMessage] 193 | ) 194 | ) 195 | .keyBy(tr => tr.key) 196 | 197 | val sortedLockAndReadRequests = transactionsInput 198 | .flatMap(t => { 199 | val readRequests = t.asserts.map(a => ReadRequest(t.transactionId, a.key)) 200 | val lockRequests = t.updates.map(u => LockRequest(t.transactionId, u.key)) 201 | readRequests.++(lockRequests) 202 | }) 203 | .keyBy(kpm => kpm.key) 204 | .applyPerKeyDoFn(new KeyedEventTimeSorter[Key, KeyProcessorMessage]) 205 | .flatMap(messages => 206 | messages 207 | ._2 208 | // for messages with the same event time, first order by 209 | // transactionId, and then give prio to ReadRequest over LockRequest 210 | .sortBy((message: KeyProcessorMessage) => message match { 211 | case ReadRequest(transactionId, key) => (transactionId, 0, key) 212 | case LockRequest(transactionId, key) => (transactionId, 1, key) 213 | case KeyTransactionResult(transactionId, key, _) => (transactionId, 2, key) 214 | }) 215 | .map(m => (messages._1, m)) 216 | ) 217 | 218 | val keyProcessorMessages = sortedLockAndReadRequests.union(keyTransactionResults) 219 | 220 | val readResults = keyProcessorMessages.applyPerKeyDoFn(new KeyTransactionProcessor()) 221 | 222 | val transactionResults = readResults.map(_._2.asInstanceOf[TransactionProcessorMessage]) 223 | .union(transactionsInput.map(_.asInstanceOf[TransactionProcessorMessage])) 224 | .keyBy(_.transactionId) 225 | .applyPerKeyDoFn(new TransactionProcessor()) 226 | 227 | val kafkaOut = 228 | KafkaIO 229 | .write[TransactionId, String]() 230 | .withBootstrapServers("localhost:9092") 231 | .withTopic(transactionResultsTopic) 232 | .withKeySerializer(classOf[StringSerializer]) 233 | .withValueSerializer(classOf[StringSerializer]) 234 | 235 | transactionResults 236 | .map(tr => KV.of(tr._1, upickle.default.write(tr._2))) 237 | .saveAsCustomOutput("transaction-results-output", kafkaOut) 238 | 239 | sc.pipeline.run().waitUntilFinish() 240 | } 241 | 242 | } -------------------------------------------------------------------------------- /modules/runners/beam/src/main/scala/domsj/streamy/db/beam/package.scala: -------------------------------------------------------------------------------- 1 | package domsj.streamy.db 2 | 3 | import java.util.Optional 4 | 5 | import org.apache.beam.sdk.io.kafka._ 6 | import org.apache.kafka.common.TopicPartition 7 | import org.apache.kafka.common.serialization.StringDeserializer 8 | import org.joda.time.Instant 9 | 10 | package object beam { 11 | 12 | def makeKafkaIOWithLogAppendTime(topic : String) = { 13 | KafkaIO 14 | .read() 15 | .withKeyDeserializer(classOf[StringDeserializer]) 16 | .withValueDeserializer(classOf[StringDeserializer]) 17 | .withBootstrapServers("localhost:9092") 18 | .withTopic(topic) 19 | .withTimestampPolicyFactory(new TimestampPolicyFactory[String, String] { 20 | override def createTimestampPolicy(tp: TopicPartition, 21 | previousWatermark: Optional[Instant] 22 | ): TimestampPolicy[String, String] = { 23 | 24 | new TimestampPolicy[String, String] { 25 | 26 | private var currentWaterMark = previousWatermark.orElse(new Instant(0)) 27 | 28 | override def getTimestampForRecord(ctx: TimestampPolicy.PartitionContext, 29 | record: KafkaRecord[String, String]): Instant = { 30 | 31 | if (record.getTimestampType != KafkaTimestampType.LOG_APPEND_TIME) { 32 | throw new IllegalStateException("Wrong kafka record timestamp type. Expected LOG_APPEND_TIME but got %s instead".format(record.getTimestampType)) 33 | } 34 | 35 | currentWaterMark = new Instant(record.getTimestamp) 36 | currentWaterMark 37 | } 38 | 39 | override def getWatermark(ctx: TimestampPolicy.PartitionContext): Instant = { 40 | currentWaterMark 41 | } 42 | } 43 | } 44 | }) 45 | .withReadCommitted() 46 | .withoutMetadata() 47 | 48 | } 49 | } -------------------------------------------------------------------------------- /modules/runners/flink/src/main/scala/domsj/streamy/db/flink/StreamyDb.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 @domsj 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, 11 | * software distributed under the License is distributed on an 12 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 13 | * KIND, either express or implied. See the License for the 14 | * specific language governing permissions and limitations 15 | * under the License. 16 | */ 17 | 18 | package domsj.streamy.db.flink 19 | 20 | import com.typesafe.scalalogging.Logger 21 | import domsj.streamy.db._ 22 | import org.apache.flink.api.common.serialization.SimpleStringSchema 23 | import org.apache.flink.api.common.state.{MapState, MapStateDescriptor, ValueState, ValueStateDescriptor} 24 | import org.apache.flink.configuration.Configuration 25 | import org.apache.flink.runtime.state.StateBackend 26 | import org.apache.flink.runtime.state.memory.MemoryStateBackend 27 | import org.apache.flink.streaming.api.TimeCharacteristic 28 | import org.apache.flink.streaming.api.functions.KeyedProcessFunction 29 | import org.apache.flink.streaming.api.scala.{StreamExecutionEnvironment, createTypeInformation} 30 | import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer011 31 | import org.apache.flink.util.Collector 32 | 33 | object StreamyDb { 34 | 35 | class KeyedEventTimeSorter[K, V] extends KeyedProcessFunction[K, V, List[V]] { 36 | private val logger = Logger[KeyedEventTimeSorter[K, V]] 37 | 38 | private var elements : MapState[Long, List[V]] = _ 39 | 40 | override def open(parameters: Configuration): Unit = { 41 | elements = getRuntimeContext.getMapState(new MapStateDescriptor("elements", classOf[Long], classOf[List[V]])) 42 | } 43 | 44 | override def processElement(value: V, 45 | ctx: KeyedProcessFunction[K, V, List[V]]#Context, 46 | out: Collector[List[V]] 47 | ): Unit = { 48 | logger.info("processElements: {}", value) 49 | 50 | val timestamp = ctx.timestamp() 51 | val timestampElements = Option(elements.get(timestamp)).getOrElse(Nil) 52 | elements.put(timestamp, timestampElements.+:(value)) 53 | ctx.timerService().registerEventTimeTimer(timestamp) 54 | } 55 | 56 | override def onTimer(timestamp: Long, 57 | ctx: KeyedProcessFunction[K, V, List[V]]#OnTimerContext, 58 | out: Collector[List[V]] 59 | ): Unit = { 60 | logger.info("onTimer: {}", timestamp) 61 | 62 | out.collect(elements.get(timestamp)) 63 | } 64 | } 65 | 66 | class KeyTransactionProcessor extends KeyedProcessFunction[Key, KeyProcessorMessage, ReadResult]{ 67 | private val logger = Logger[KeyTransactionProcessor] 68 | 69 | private var value : ValueState[Value] = _ 70 | private var writeLockWaiters : MapState[TransactionId, List[TransactionId]] = _ 71 | private var latestWriteLock : ValueState[TransactionId] = _ 72 | 73 | 74 | override def open(parameters: Configuration): Unit = { 75 | value = getRuntimeContext.getState(new ValueStateDescriptor("value", classOf[Value])) 76 | writeLockWaiters = getRuntimeContext.getMapState(new MapStateDescriptor("writeLockWaiters", classOf[TransactionId], classOf[List[TransactionId]])) 77 | latestWriteLock = getRuntimeContext.getState(new ValueStateDescriptor("latestWriteLock", classOf[TransactionId])) 78 | } 79 | 80 | override def processElement(kpm: KeyProcessorMessage, 81 | ctx: KeyedProcessFunction[Key, KeyProcessorMessage, ReadResult]#Context, 82 | out: Collector[ReadResult]): Unit = { 83 | logger.info("processElement: {}", kpm) 84 | 85 | kpm match { 86 | 87 | case LockRequest(transactionId, key) => 88 | latestWriteLock.update(transactionId) 89 | writeLockWaiters.put(transactionId, Nil) 90 | 91 | case ReadRequest(transactionId, key) => 92 | val latestWriteLockTransactionId = latestWriteLock.value() 93 | if (latestWriteLockTransactionId == null) { 94 | // no write locks, so can read value immediately 95 | val readResult = ReadResult(transactionId, ctx.getCurrentKey, Option(value.value())) 96 | out.collect(readResult) 97 | } else { 98 | // one or more write locks in the queue, register with the last one 99 | val waiters = writeLockWaiters.get(latestWriteLockTransactionId) 100 | writeLockWaiters.put(latestWriteLockTransactionId, waiters.::(transactionId)) 101 | } 102 | 103 | case KeyTransactionResult(transactionId, key, valueOptionOption) => 104 | 105 | val waiters = writeLockWaiters.get(transactionId) 106 | writeLockWaiters.remove(transactionId) 107 | 108 | if (waiters != null) { 109 | 110 | // maybe update state 111 | valueOptionOption match { 112 | case Some(valueOption) => 113 | valueOption match { 114 | case Some(v) => value.update(v) 115 | case None => value.clear() 116 | } 117 | case None => () 118 | } 119 | 120 | val maybeValue = Option(value.value()) 121 | waiters.foreach(transactionId => out.collect(ReadResult(transactionId, key, maybeValue))) 122 | } 123 | 124 | if (latestWriteLock.value() == transactionId) { 125 | latestWriteLock.clear() 126 | } 127 | } 128 | } 129 | } 130 | 131 | class TransactionProcessor extends KeyedProcessFunction[TransactionId, TransactionProcessorMessage, TransactionResult] { 132 | private val logger = Logger[TransactionProcessor] 133 | 134 | private var transactionState : ValueState[Transaction] = _ 135 | private var readResultsState : MapState[Key, ValueOption] = _ 136 | private var readResultsCountState : ValueState[Int] = _ 137 | 138 | 139 | override def open(parameters: Configuration): Unit = { 140 | transactionState = getRuntimeContext.getState(new ValueStateDescriptor("transaction", classOf[Transaction])) 141 | readResultsState = getRuntimeContext.getMapState(new MapStateDescriptor("read-results", classOf[Key], classOf[ValueOption])) 142 | readResultsCountState = getRuntimeContext.getState(new ValueStateDescriptor("read-results-count", classOf[Int])) 143 | } 144 | 145 | override def processElement(value: TransactionProcessorMessage, 146 | ctx: KeyedProcessFunction[TransactionId, TransactionProcessorMessage, TransactionResult]#Context, 147 | out: Collector[TransactionResult] 148 | ): Unit = { 149 | logger.info("processElement: {}", value) 150 | 151 | val getReadResultsCount = () => Option(readResultsCountState.value()).getOrElse(0) 152 | 153 | // gather everything into the state 154 | value match { 155 | case Transaction(_, _, _) => 156 | transactionState.update(value.asInstanceOf[Transaction]) 157 | case ReadResult(_, key, valueOption) => 158 | readResultsState.put(key, valueOption) 159 | readResultsCountState.update(getReadResultsCount() + 1) 160 | } 161 | 162 | // process the transaction once the state is complete 163 | // NOTE there is an optimisation opportunity for for failing transactions by processing ReadResults more incremental 164 | val transaction = transactionState.value() 165 | if (transaction != null && transaction.asserts.size == getReadResultsCount()) { 166 | val assertsAllSucceed = 167 | transaction 168 | .asserts 169 | .forall((keyValueOption: KeyValueOption) => 170 | readResultsState.get(keyValueOption.key) == keyValueOption.valueOption) 171 | 172 | out.collect(TransactionResult(transaction, assertsAllSucceed)) 173 | } 174 | } 175 | } 176 | 177 | def main(cmdLineArgs: Array[String]): Unit = { 178 | val environment = StreamExecutionEnvironment.getExecutionEnvironment 179 | environment.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) 180 | environment.enableCheckpointing(5000) 181 | environment.setStateBackend(new MemoryStateBackend().asInstanceOf[StateBackend]) 182 | 183 | val kafkaProperties = makeKafkaProperties() 184 | 185 | val transactionInputs = 186 | streamFromKafka(environment, transactionInputsTopic) 187 | .flatMap(s => upickle.default.read[List[Transaction]](s)) 188 | 189 | val keyTransactionResults = 190 | streamFromKafka(environment, transactionResultsTopic) 191 | .map(s => upickle.default.read[TransactionResult](s)) 192 | .flatMap(tr => 193 | tr.transaction.updates.map(kvo => 194 | KeyTransactionResult( 195 | tr.transaction.transactionId, 196 | kvo.key, 197 | if (tr.succeeded) { Some(kvo.valueOption) } else { None }) 198 | .asInstanceOf[KeyProcessorMessage] 199 | ) 200 | ) 201 | 202 | 203 | val sortedLockAndReadRequests = transactionInputs 204 | .flatMap(t => { 205 | val readRequests = t.asserts.map(a => ReadRequest(t.transactionId, a.key)) 206 | val lockRequests = t.updates.map(u => LockRequest(t.transactionId, u.key)) 207 | readRequests.++(lockRequests) 208 | }) 209 | .keyBy(kpm => kpm.key) 210 | .process(new KeyedEventTimeSorter[TransactionId, KeyProcessorMessage]()) 211 | .flatMap(l => l.sortBy((message: KeyProcessorMessage) => message match { 212 | case ReadRequest(transactionId, key) => (transactionId, 0, key) 213 | case LockRequest(transactionId, key) => (transactionId, 1, key) 214 | case KeyTransactionResult(transactionId, key, _) => (transactionId, 2, key) 215 | })) 216 | 217 | sortedLockAndReadRequests.print("sortedLockAndReadRequests") 218 | 219 | val keyProcessorMessages = sortedLockAndReadRequests.union(keyTransactionResults) 220 | 221 | val readResults = keyProcessorMessages 222 | .keyBy(_.key) 223 | .process(new KeyTransactionProcessor()) 224 | 225 | readResults.print("readResults") 226 | 227 | val transactionResults = 228 | readResults.map(_.asInstanceOf[TransactionProcessorMessage]) 229 | .union(transactionInputs.map(_.asInstanceOf[TransactionProcessorMessage])) 230 | .keyBy(_.transactionId) 231 | .process(new TransactionProcessor()) 232 | 233 | transactionResults 234 | .map(tr => upickle.default.write(tr)) 235 | .addSink(new FlinkKafkaProducer011[String](transactionResultsTopic, new SimpleStringSchema(), kafkaProperties)) 236 | 237 | transactionResults.print("transactionResults") 238 | 239 | val jobExecutionResult = environment.execute() 240 | 241 | print(jobExecutionResult) 242 | } 243 | } -------------------------------------------------------------------------------- /modules/runners/flink/src/main/scala/domsj/streamy/db/flink/package.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 @domsj 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, 11 | * software distributed under the License is distributed on an 12 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 13 | * KIND, either express or implied. See the License for the 14 | * specific language governing permissions and limitations 15 | * under the License. 16 | */ 17 | 18 | package domsj.streamy.db 19 | 20 | import com.typesafe.scalalogging.Logger 21 | import org.apache.flink.api.common.serialization.SimpleStringSchema 22 | import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks 23 | import org.apache.flink.streaming.api.scala.{DataStream, StreamExecutionEnvironment, createTypeInformation} 24 | import org.apache.flink.streaming.api.watermark.Watermark 25 | import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer011 26 | 27 | package object flink { 28 | def streamFromKafka(environment: StreamExecutionEnvironment, 29 | topic : String) 30 | : DataStream[String] = { 31 | 32 | val logger = Logger("streamFromKafka") 33 | 34 | val kafkaProperties = makeKafkaProperties() 35 | 36 | val kafkaConsumer = new FlinkKafkaConsumer011[String]( 37 | topic, 38 | new SimpleStringSchema(), 39 | kafkaProperties) 40 | 41 | val assigner: AssignerWithPunctuatedWatermarks[String] = new AssignerWithPunctuatedWatermarks[String] { 42 | private var previous = Long.MinValue 43 | 44 | override def checkAndGetNextWatermark(lastElement: String, extractedTimestamp: Long): Watermark = { 45 | new Watermark(extractedTimestamp) 46 | } 47 | 48 | override def extractTimestamp(element: String, elementTimestamp: Long): Long = { 49 | // ensure timestamps are strictly monotonic (so the watermark emission above is justified) 50 | val res = Math.max(elementTimestamp, previous + 1) 51 | previous = res 52 | res 53 | } 54 | 55 | } 56 | kafkaConsumer.assignTimestampsAndWatermarks(assigner) 57 | 58 | environment.addSource(kafkaConsumer) 59 | } 60 | 61 | } -------------------------------------------------------------------------------- /project/build.properties: -------------------------------------------------------------------------------- 1 | sbt.version = 1.2.7 --------------------------------------------------------------------------------