├── project ├── plugins.sbt └── build.properties ├── slides └── Akka Streams in Practice.pdf ├── src ├── main │ ├── resources │ │ ├── application.conf │ │ └── logback.xml │ └── scala │ │ └── org │ │ └── kunicki │ │ └── akka_streams │ │ ├── model │ │ └── ValidReading.scala │ │ ├── Importer.scala │ │ ├── repository │ │ └── CassandraRepository.scala │ │ ├── RandomDataGenerator.scala │ │ └── importer │ │ └── CsvImporter.scala └── test │ └── scala │ └── org │ └── kunicki │ └── akka_streams │ └── AverageCalculatorSpec.scala ├── .gitignore └── README.md /project/plugins.sbt: -------------------------------------------------------------------------------- 1 | logLevel := Level.Warn -------------------------------------------------------------------------------- /project/build.properties: -------------------------------------------------------------------------------- 1 | sbt.version = 0.13.8 -------------------------------------------------------------------------------- /slides/Akka Streams in Practice.pdf: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/rucek/akka-streams-in-practice/HEAD/slides/Akka Streams in Practice.pdf -------------------------------------------------------------------------------- /src/main/resources/application.conf: -------------------------------------------------------------------------------- 1 | importer { 2 | import-directory = "./data" 3 | lines-to-skip = 0 4 | concurrent-files = 10 5 | concurrent-writes = 5 6 | non-io-parallelism = 42 7 | } 8 | 9 | generator { 10 | number-of-files = 100 11 | number-of-pairs = 1000 12 | invalid-line-probability = 0.005 13 | } 14 | -------------------------------------------------------------------------------- /src/main/scala/org/kunicki/akka_streams/model/ValidReading.scala: -------------------------------------------------------------------------------- 1 | package org.kunicki.akka_streams.model 2 | 3 | import scala.util.Random 4 | 5 | sealed trait Reading { 6 | 7 | def id: Int 8 | } 9 | 10 | case class ValidReading(id: Int, value: Double = Random.nextDouble()) extends Reading 11 | 12 | case class InvalidReading(id: Int) extends Reading 13 | -------------------------------------------------------------------------------- /src/main/resources/logback.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | %date [%thread] %-5level %logger{36}:%line - %msg%n 5 | 6 | 7 | 8 | 9 | 10 | 11 | 12 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | # Created by .ignore support plugin (hsz.mobi) 2 | ### SBT template 3 | # Simple Build Tool 4 | # http://www.scala-sbt.org/release/docs/Getting-Started/Directories.html#configuring-version-control 5 | 6 | target/ 7 | lib_managed/ 8 | src_managed/ 9 | project/boot/ 10 | .history 11 | .cache 12 | ### Scala template 13 | *.class 14 | *.log 15 | 16 | # sbt specific 17 | .lib/ 18 | dist/* 19 | project/plugins/project/ 20 | 21 | # Scala-IDE specific 22 | .scala_dependencies 23 | .worksheet 24 | 25 | #IntelliJ-IDE specific 26 | .idea 27 | 28 | data 29 | -------------------------------------------------------------------------------- /src/main/scala/org/kunicki/akka_streams/Importer.scala: -------------------------------------------------------------------------------- 1 | package org.kunicki.akka_streams 2 | 3 | import akka.actor.ActorSystem 4 | import com.typesafe.config.ConfigFactory 5 | import org.kunicki.akka_streams.importer.CsvImporter 6 | import org.kunicki.akka_streams.repository.ReadingRepository 7 | 8 | 9 | object Importer extends App { 10 | 11 | implicit val system = ActorSystem("akka-streams-in-practice") 12 | 13 | private val config = ConfigFactory.load() 14 | private val readingRepository = new ReadingRepository 15 | 16 | import system.dispatcher 17 | 18 | new CsvImporter(config, readingRepository).importFromFiles 19 | .onComplete { _ => 20 | readingRepository.shuthdown 21 | system.terminate() 22 | } 23 | } 24 | -------------------------------------------------------------------------------- /src/main/scala/org/kunicki/akka_streams/repository/CassandraRepository.scala: -------------------------------------------------------------------------------- 1 | package org.kunicki.akka_streams.repository 2 | import com.websudos.phantom.Manager 3 | import com.websudos.phantom.dsl._ 4 | import org.kunicki.akka_streams.model.ValidReading 5 | 6 | import scala.concurrent.Future 7 | 8 | class ReadingRepository { 9 | 10 | def save(reading: ValidReading): Future[Unit] = MyDatabase.readings.store(reading).map(_ => ()) 11 | 12 | def shuthdown = { 13 | Manager.shutdown() 14 | MyDatabase.session.getCluster.close() 15 | MyDatabase.session.close() 16 | } 17 | } 18 | 19 | object Defaults { 20 | 21 | val connector = ContactPoint.local.keySpace("akka_streams") 22 | } 23 | 24 | class Readings extends CassandraTable[ConcreteReadings, ValidReading] { 25 | 26 | object id extends IntColumn(this) with PrimaryKey[Int] 27 | 28 | object value extends DoubleColumn(this) 29 | 30 | def fromRow(row: Row): ValidReading = ValidReading(id(row), value(row)) 31 | } 32 | 33 | abstract class ConcreteReadings extends Readings with RootConnector { 34 | 35 | def store(reading: ValidReading): Future[ResultSet] = { 36 | insert().value(_.id, reading.id).value(_.value, reading.value).future() 37 | } 38 | } 39 | 40 | class MyDatabase(val keyspace: KeySpaceDef) extends Database(keyspace) { 41 | object readings extends ConcreteReadings with keyspace.Connector 42 | } 43 | 44 | object MyDatabase extends MyDatabase(Defaults.connector) 45 | -------------------------------------------------------------------------------- /src/main/scala/org/kunicki/akka_streams/RandomDataGenerator.scala: -------------------------------------------------------------------------------- 1 | package org.kunicki.akka_streams 2 | 3 | import java.nio.file.Paths 4 | import java.util.UUID 5 | 6 | import akka.actor.ActorSystem 7 | import akka.stream.ActorMaterializer 8 | import akka.stream.scaladsl.{FileIO, Sink, Source} 9 | import akka.util.ByteString 10 | import com.typesafe.config.ConfigFactory 11 | import com.typesafe.scalalogging.LazyLogging 12 | import org.kunicki.akka_streams.model.ValidReading 13 | 14 | import scala.concurrent.Await 15 | import scala.concurrent.duration.Duration 16 | import scala.util.Random 17 | 18 | object RandomDataGenerator extends App with LazyLogging { 19 | 20 | implicit val system = ActorSystem("random-data-generator") 21 | implicit val materializer = ActorMaterializer() 22 | 23 | val config = ConfigFactory.load() 24 | val numberOfFiles = config.getInt("generator.number-of-files") 25 | val numberOfPairs = config.getInt("generator.number-of-pairs") 26 | val invalidLineProbability = config.getDouble("generator.invalid-line-probability") 27 | 28 | logger.info("Starting generation") 29 | 30 | val f = Source(1 to numberOfFiles) 31 | .mapAsyncUnordered(numberOfFiles) { _ => 32 | val fileName = UUID.randomUUID().toString 33 | Source(1 to numberOfPairs).map { case _ => 34 | val id = Random.nextInt(1000000) 35 | Seq(ValidReading(id), ValidReading(id)).map { reading => 36 | val value = if (Random.nextDouble() > invalidLineProbability) reading.value.toString else "invalid_value" 37 | ByteString(s"${reading.id};$value\n") 38 | }.foldLeft(ByteString())(_ concat _) 39 | }.runWith(FileIO.toPath(Paths.get(s"data/$fileName"))) 40 | } 41 | .runWith(Sink.ignore) 42 | 43 | Source(1 to 100) 44 | 45 | Await.ready(f, Duration.Inf) 46 | logger.info("Generated random data") 47 | } 48 | -------------------------------------------------------------------------------- /src/test/scala/org/kunicki/akka_streams/AverageCalculatorSpec.scala: -------------------------------------------------------------------------------- 1 | package org.kunicki.akka_streams 2 | 3 | import akka.actor.ActorSystem 4 | import akka.stream.ActorMaterializer 5 | import akka.stream.scaladsl.Source 6 | import akka.stream.testkit.scaladsl.TestSink 7 | import com.typesafe.config.ConfigFactory 8 | import org.kunicki.akka_streams.importer.CsvImporter 9 | import org.kunicki.akka_streams.model.{InvalidReading, ValidReading} 10 | import org.kunicki.akka_streams.repository.ReadingRepository 11 | import org.scalatest.{FlatSpec, Matchers} 12 | 13 | class AverageCalculatorSpec extends FlatSpec with Matchers { 14 | 15 | implicit val system = ActorSystem("test") 16 | implicit val materializer = ActorMaterializer() 17 | 18 | val tested = new CsvImporter(ConfigFactory.load(), new ReadingRepository).computeAverage 19 | 20 | it should "count an average of valid readings" in { 21 | // given 22 | val readings = List( 23 | ValidReading(1, 2), 24 | ValidReading(1, 3), 25 | ValidReading(2, 4), 26 | ValidReading(2, 5) 27 | ) 28 | 29 | // when 30 | val flow = Source(readings).via(tested) 31 | 32 | // then 33 | flow.runWith(TestSink.probe[ValidReading]) 34 | .request(4) 35 | .expectNextUnordered(ValidReading(1, 2.5), ValidReading(2, 4.5)) 36 | .expectComplete() 37 | } 38 | 39 | it should "count an average of mixed valid and invalid readings" in { 40 | // given 41 | val readings = List( 42 | ValidReading(1, 2), 43 | InvalidReading(1), 44 | InvalidReading(2), 45 | ValidReading(2, 5) 46 | ) 47 | 48 | // when 49 | val flow = Source(readings).via(tested) 50 | 51 | // then 52 | flow.runWith(TestSink.probe[ValidReading]) 53 | .request(4) 54 | .expectNextUnordered(ValidReading(1, 2), ValidReading(2, 5)) 55 | .expectComplete() 56 | } 57 | 58 | it should "count a fake average of invalid readings" in { 59 | // given 60 | val readings = List( 61 | InvalidReading(1), 62 | InvalidReading(1) 63 | ) 64 | 65 | // when 66 | val flow = Source(readings).via(tested) 67 | 68 | // then 69 | flow.runWith(TestSink.probe[ValidReading]) 70 | .request(2) 71 | .expectNext(ValidReading(1, -1)) 72 | .expectComplete() 73 | } 74 | } 75 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # Akka Streams in Practice 2 | 3 | This is a sample Akka Streams project which uses the library to import data from a number of Gzipped CSV files into a Cassandra table. 4 | 5 | The CSV files contain some kind of readings, i.e. `(id, value)` pairs, where every `id` has two associated `value`s and the records for a given `id` appear in subsequent lines in the file. Any of the `value`s may ocassionally be an invalid number. Example: 6 | 7 | ``` 8 | 93500;0.5287942176336127 9 | 93500;0.3404326895942348 10 | 961989;invalid_value 11 | 961989;0.27452559752437566 12 | 136308;0.07525660747531115 13 | 136308;0.6509485024097678 14 | ``` 15 | 16 | The importer streams the Gzipped files and extracts them on the fly, then converts every line to a domain object representing either a valid or an invalid reading. The next step is to compute an average value for the readings under a given `id` when any of the readings is valid. When both readings for a given `id` are invalid, the average is assumed to be `-1`. Finally, the computed average values are written to Cassandra. 17 | 18 | # Prerequisites 19 | 20 | ## CSV files 21 | You can generate the CSV data yourself using the provided `RandomDataGenerator`. There are a few configurable properties of the generator in `application.conf`: 22 | 23 | ``` 24 | generator { 25 | number-of-files = 100 26 | number-of-pairs = 1000 27 | invalid-line-probability = 0.005 28 | } 29 | ``` 30 | 31 | They are pretty self-explanatory: `number-of-files` is the number of files to be generated, `number-of-pairs` is the number of `(id, value)` pairs in each file (since two `value`s are generated for each `id`), `invalid-line-probability` is the probability of the generator inserting a line with a value that is not a valid number. 32 | 33 | Note that the importer expectt the files to be compressed with Gzip. You can easily compress the generated files with the following command run in the `./data` directory: 34 | 35 | ```bash 36 | find . -type f -exec gzip "{}" \; 37 | ``` 38 | 39 | Now you're ready to generate the CSV files: 40 | 41 | ```bash 42 | sbt "runMain org.kunicki.akka_streams.RandomDataGenerator" 43 | ``` 44 | 45 | ## Cassandra 46 | 47 | The probably easiest way to have Cassandra up and running is to use a [Docker](http://docker.io/) image - then all you need to do is run the following command: 48 | 49 | ```bash 50 | docker run -d --name cassandra cassandra 51 | ``` 52 | 53 | and in a while you should have Cassandra ready at port 9042. When the container has started, it's time to create a keyspace and a table for our data. 54 | 55 | Depending on the setup that you have you have you might want to bind the container directly to port 9042: 56 | ```bash 57 | docker run --name cassandra -p 127.0.0.1:9042:9042 -d cassandra 58 | ``` 59 | 60 | First you need to run the CQL shell: 61 | 62 | ```bash 63 | docker exec -it cassandra cqlsh 64 | ``` 65 | 66 | Then, in `cqlsh` you create an `akka_streams` keyspace: 67 | 68 | ```cql 69 | CREATE KEYSPACE akka_streams WITH REPLICATION = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 }; 70 | ``` 71 | 72 | Finally, let's create the `readings` table: 73 | 74 | ```cql 75 | CREATE TABLE akka_streams.readings (id int PRIMARY KEY, value float); 76 | ``` 77 | 78 | # Running 79 | 80 | Before running the import you may wish to tweak some configuration settings in `application.conf`: 81 | 82 | ``` 83 | importer { 84 | import-directory = "./data" 85 | lines-to-skip = 0 86 | concurrent-files = 10 87 | concurrent-writes = 5 88 | non-io-parallelism = 42 89 | } 90 | ``` 91 | 92 | The `import-directory` is the directory with the CSV files, `lines-to-skip` allows you to optionally skip a number of lines from the top of each file (e.g. CSV headers if you had any), `concurrent-files` tells the importer how many files to read in parallel, `concurrent-writes` determines the number of parallel inserts to Cassandra, `non-io-parallelism` defines the number of threads for in-memory calculations. 93 | 94 | Having the configuration tweaked, the test data generated and a Cassandra instance running, you can now run the actual import: 95 | 96 | ```bash 97 | sbt "runMain org.kunicki.akka_streams.Importer" 98 | ``` 99 | -------------------------------------------------------------------------------- /src/main/scala/org/kunicki/akka_streams/importer/CsvImporter.scala: -------------------------------------------------------------------------------- 1 | package org.kunicki.akka_streams.importer 2 | 3 | import java.io.{File, FileInputStream} 4 | import java.nio.file.Paths 5 | import java.util.zip.GZIPInputStream 6 | 7 | import akka.actor.ActorSystem 8 | import akka.stream.scaladsl.{Balance, Flow, Framing, GraphDSL, Keep, Merge, Sink, Source, StreamConverters} 9 | import akka.stream.{ActorAttributes, ActorMaterializer, FlowShape, Supervision} 10 | import akka.util.ByteString 11 | import akka.{Done, NotUsed} 12 | import com.typesafe.config.Config 13 | import com.typesafe.scalalogging.LazyLogging 14 | import org.kunicki.akka_streams.model.{InvalidReading, Reading, ValidReading} 15 | import org.kunicki.akka_streams.repository.ReadingRepository 16 | 17 | import scala.concurrent.Future 18 | import scala.util.{Failure, Success} 19 | 20 | class CsvImporter(config: Config, readingRepository: ReadingRepository) 21 | (implicit system: ActorSystem) extends LazyLogging { 22 | 23 | import system.dispatcher 24 | 25 | private val importDirectory = Paths.get(config.getString("importer.import-directory")).toFile 26 | private val linesToSkip = config.getInt("importer.lines-to-skip") 27 | private val concurrentFiles = config.getInt("importer.concurrent-files") 28 | private val concurrentWrites = config.getInt("importer.concurrent-writes") 29 | private val nonIOParallelism = config.getInt("importer.non-io-parallelism") 30 | 31 | def parseLine(filePath: String)(line: String): Future[Reading] = Future { 32 | val fields = line.split(";") 33 | val id = fields(0).toInt 34 | try { 35 | val value = fields(1).toDouble 36 | ValidReading(id, value) 37 | } catch { 38 | case t: Throwable => 39 | logger.error(s"Unable to parse line in $filePath:\n$line: ${t.getMessage}") 40 | InvalidReading(id) 41 | } 42 | } 43 | 44 | val lineDelimiter: Flow[ByteString, ByteString, NotUsed] = 45 | Framing.delimiter(ByteString("\n"), 128, allowTruncation = true) 46 | 47 | val parseFile: Flow[File, Reading, NotUsed] = 48 | Flow[File].flatMapConcat { file => 49 | val gzipInputStream = new GZIPInputStream(new FileInputStream(file)) 50 | 51 | StreamConverters.fromInputStream(() => gzipInputStream) 52 | .via(lineDelimiter) 53 | .drop(linesToSkip) 54 | .map(_.utf8String) 55 | .mapAsync(parallelism = nonIOParallelism)(parseLine(file.getPath)) 56 | } 57 | 58 | val computeAverage: Flow[Reading, ValidReading, NotUsed] = 59 | Flow[Reading].grouped(2).mapAsyncUnordered(parallelism = nonIOParallelism) { readings => 60 | Future { 61 | val validReadings = readings.collect { case r: ValidReading => r } 62 | val average = if (validReadings.nonEmpty) validReadings.map(_.value).sum / validReadings.size else -1 63 | ValidReading(readings.head.id, average) 64 | } 65 | } 66 | 67 | val storeReadings: Sink[ValidReading, Future[Done]] = 68 | Flow[ValidReading] 69 | .mapAsyncUnordered(concurrentWrites)(readingRepository.save) 70 | .toMat(Sink.ignore)(Keep.right) 71 | 72 | val processSingleFile: Flow[File, ValidReading, NotUsed] = 73 | Flow[File] 74 | .via(parseFile) 75 | .via(computeAverage) 76 | 77 | def importFromFiles = { 78 | implicit val materializer = ActorMaterializer() 79 | 80 | val files = importDirectory.listFiles.toList 81 | logger.info(s"Starting import of ${files.size} files from ${importDirectory.getPath}") 82 | 83 | val startTime = System.currentTimeMillis() 84 | 85 | val balancer = GraphDSL.create() { implicit builder => 86 | import GraphDSL.Implicits._ 87 | 88 | val balance = builder.add(Balance[File](concurrentFiles)) 89 | val merge = builder.add(Merge[ValidReading](concurrentFiles)) 90 | 91 | (1 to concurrentFiles).foreach { _ => 92 | balance ~> processSingleFile ~> merge 93 | } 94 | 95 | FlowShape(balance.in, merge.out) 96 | } 97 | 98 | Source(files) 99 | .via(balancer) 100 | .withAttributes(ActorAttributes.supervisionStrategy { e => 101 | logger.error("Exception thrown during stream processing", e) 102 | Supervision.Resume 103 | }) 104 | .runWith(storeReadings) 105 | .andThen { 106 | case Success(_) => 107 | val elapsedTime = (System.currentTimeMillis() - startTime) / 1000.0 108 | logger.info(s"Import finished in ${elapsedTime}s") 109 | case Failure(e) => logger.error("Import failed", e) 110 | } 111 | } 112 | } 113 | --------------------------------------------------------------------------------