├── .github ├── dependabot.yml └── workflows │ └── ci.yml ├── .gitignore ├── .scalafmt.conf ├── LICENSE ├── NOTICE ├── README.md ├── build.sbt ├── catalog-info.yaml ├── make-site.sh ├── project ├── build.properties └── plugins.sbt └── src ├── main └── scala │ └── com │ └── spotify │ └── bdrc │ ├── pipeline │ ├── AverageScorePerItem.scala │ ├── BloomFilterSetDifference.scala │ ├── Count.scala │ ├── CountDistinctItems.scala │ ├── CountUsers.scala │ ├── DistinctItems.scala │ ├── FieldStatistics.scala │ ├── FindMedian.scala │ ├── InvertedIndex.scala │ ├── JoinLogAndMetadata.scala │ ├── JoinLogs.scala │ ├── MaxItemPerUser.scala │ ├── MinItemPerUser.scala │ ├── PageRank.scala │ ├── Sessions.scala │ ├── Statistics.scala │ ├── SumPerItem.scala │ ├── TfIdf.scala │ ├── TopItems.scala │ ├── TopItemsPerUser.scala │ ├── TotalAndDistinctCount.scala │ └── WordCount.scala │ ├── scala │ ├── Collections.scala │ ├── FilterMessyData.scala │ ├── HandlingOptions.scala │ └── JavaPrimitives.scala │ └── util │ └── Records.scala └── test └── scala └── com └── spotify └── bdrc ├── bench └── ForYieldBenchmark.scala └── testing ├── T01EndToEndTest.scala ├── T02TransformTest.scala ├── T03FunctionTest.scala ├── T04PropertyBasedTest.scala └── T05MixedTest.scala /.github/dependabot.yml: -------------------------------------------------------------------------------- 1 | version: 2 2 | updates: 3 | - package-ecosystem: github-actions 4 | directory: "/" 5 | schedule: 6 | interval: daily 7 | time: "04:00" 8 | open-pull-requests-limit: 10 9 | -------------------------------------------------------------------------------- /.github/workflows/ci.yml: -------------------------------------------------------------------------------- 1 | name: CI 2 | on: [push, pull_request] 3 | 4 | jobs: 5 | test: 6 | runs-on: ubuntu-latest 7 | steps: 8 | - uses: actions/checkout@v4.1.1 9 | - name: Cache sbt 10 | uses: coursier/cache-action@v6 11 | - name: Java 8 setup 12 | uses: olafurpg/setup-scala@v14 13 | with: 14 | java-version: 1.8.0 15 | - run: sbt test 16 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | .bsp 2 | .idea 3 | target 4 | -------------------------------------------------------------------------------- /.scalafmt.conf: -------------------------------------------------------------------------------- 1 | version = "3.5.9" 2 | maxColumn = 100 3 | 4 | binPack.literalArgumentLists = true 5 | 6 | continuationIndent { 7 | callSite = 2 8 | defnSite = 2 9 | } 10 | 11 | newlines { 12 | alwaysBeforeMultilineDef = false 13 | sometimesBeforeColonInMethodReturnType = true 14 | } 15 | 16 | verticalMultiline { 17 | newlineAfterImplicitKW = true 18 | newlineBeforeImplicitKW = true 19 | } 20 | 21 | docstrings = JavaDoc 22 | 23 | project.git = false 24 | 25 | rewrite { 26 | rules = [PreferCurlyFors, RedundantBraces, RedundantParens, SortImports] 27 | redundantBraces.generalExpressions = false 28 | redundantBraces.maxLines = 1 29 | } 30 | -------------------------------------------------------------------------------- /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 | Big Data Rosetta Code 2 | Copyright 2016 Spotify AB 3 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | big-data-rosetta-code 2 | ===================== 3 | 4 | [![Build Status](https://img.shields.io/github/actions/workflow/status/spotify/big-data-rosetta-code/.github/workflows/ci.yml)](https://github.com/spotify/big-data-rosetta-code/actions?query=workflow%3ACI) 5 | [![GitHub license](https://img.shields.io/github/license/spotify/big-data-rosetta-code.svg)](./LICENSE) 6 | 7 | Code snippets for solving common big data problems on various platforms. Inspired by [Rosetta Code](http://rosettacode.org/). 8 | 9 | For examples rended side by side with comments see: 10 | 11 | http://spotify.github.io/big-data-rosetta-code/ 12 | 13 | Currently the following are covered: 14 | 15 | - [Scalding](https://github.com/twitter/scalding) 16 | - [Scio](https://github.com/spotify/scio) 17 | - [Spark](https://github.com/apache/spark) 18 | 19 | # Topics 20 | 21 | - [src/main/scala/com/spotify/bdrc/scala](./src/main/scala/com/spotify/bdrc/scala) Scala tricks for data processing 22 | - [src/main/scala/com/spotify/bdrc/pipeline](./src/main/scala/com/spotify/bdrc/pipeline) Data pipeline snippets 23 | - [src/test/scala/com/spotify/bdrc/testing](./src/test/scala/com/spotify/bdrc/testing) Examples for pipeline testing 24 | 25 | # License 26 | 27 | Copyright 2016 Spotify AB. 28 | 29 | Licensed under the Apache License, Version 2.0: http://www.apache.org/licenses/LICENSE-2.0 30 | -------------------------------------------------------------------------------- /build.sbt: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Spotify AB. 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 | import com.github.sbt.git.SbtGit.GitKeys.gitRemoteRepo 19 | import _root_.io.regadas.sbt.SbtSoccoKeys._ 20 | 21 | organization := "com.spotify" 22 | name := "big-data-rosetta-code" 23 | version := "0.1.0-SNAPSHOT" 24 | 25 | val scioVersion = "0.13.5" 26 | val scaldingVersion = "0.17.4" 27 | val sparkVersion = "3.5.0" 28 | val algebirdVersion = "0.13.10" 29 | val scalacheckVersion = "1.17.0" 30 | val scalameterVersion = "0.19" 31 | val scalatestVersion = "3.2.17" 32 | val scalatestPlusVersion = "3.2.17.0" 33 | 34 | scalaVersion := "2.12.18" 35 | scalacOptions ++= Seq( 36 | "-target:jvm-1.8", 37 | "-deprecation", 38 | "-feature", 39 | "-unchecked", 40 | "-language:higherKinds" 41 | ) 42 | javacOptions ++= Seq("-source", "1.8", "-target", "1.8") 43 | 44 | resolvers += "Cascading libraries" at "https://conjars.wensel.net/repo/" 45 | libraryDependencies ++= Seq( 46 | "com.spotify" %% "scio-core" % scioVersion, 47 | "com.spotify" %% "scio-extra" % scioVersion, 48 | "com.spotify" %% "scio-test" % scioVersion % "test", 49 | "com.twitter" %% "scalding-core" % scaldingVersion, 50 | "com.twitter" %% "algebird-spark" % algebirdVersion, 51 | "org.apache.spark" %% "spark-core" % sparkVersion, 52 | "org.apache.spark" %% "spark-mllib" % sparkVersion, 53 | "org.scalatest" %% "scalatest" % scalatestVersion % "test", 54 | "org.scalatestplus" %% "scalacheck-1-17" % scalatestPlusVersion % "test", 55 | "org.scalacheck" %% "scalacheck" % scalacheckVersion % "test", 56 | "com.storm-enroute" %% "scalameter" % scalameterVersion % "test" 57 | ) 58 | 59 | val scalaMeterFramework = new TestFramework( 60 | "org.scalameter.ScalaMeterFramework" 61 | ) 62 | testFrameworks += scalaMeterFramework 63 | testOptions += Tests.Argument(scalaMeterFramework, "-silent") 64 | Test / parallelExecution := false 65 | logBuffered := false 66 | 67 | soccoOnCompile := true 68 | soccoPackage := List( 69 | "com.spotify.scio:http://spotify.github.io/scio/api", 70 | "com.twitter.algebird:http://twitter.github.io/algebird/api", 71 | "com.twitter.scalding:http://twitter.github.io/scalding/api", 72 | "org.apache.spark:http://spark.apache.org/docs/latest/api/scala" 73 | ) 74 | addCompilerPlugin(("io.regadas" %% "socco-ng" % "0.1.4").cross(CrossVersion.full)) 75 | makeSite := makeSite.dependsOn(Compile / compile).value 76 | gitRemoteRepo := "git@github.com:spotify/big-data-rosetta-code.git" 77 | 78 | enablePlugins(SbtSoccoPlugin) 79 | enablePlugins(GhpagesPlugin) 80 | -------------------------------------------------------------------------------- /catalog-info.yaml: -------------------------------------------------------------------------------- 1 | apiVersion: backstage.io/v1alpha1 2 | kind: Resource 3 | metadata: 4 | name: big-data-rosetta-code 5 | spec: 6 | type: resource 7 | owner: flatmap 8 | -------------------------------------------------------------------------------- /make-site.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | sbt makeSite ghpagesPushSite 4 | -------------------------------------------------------------------------------- /project/build.properties: -------------------------------------------------------------------------------- 1 | sbt.version=1.9.7 2 | -------------------------------------------------------------------------------- /project/plugins.sbt: -------------------------------------------------------------------------------- 1 | addSbtPlugin("com.github.sbt" % "sbt-ghpages" % "0.8.0") 2 | addSbtPlugin("org.scalameta" % "sbt-scalafmt" % "2.5.2") 3 | addSbtPlugin("io.regadas" % "sbt-socco" % "0.1.5") 4 | -------------------------------------------------------------------------------- /src/main/scala/com/spotify/bdrc/pipeline/AverageScorePerItem.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Spotify AB. 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 | // Example: Compute Average Score per Item 19 | // Input is a collection of (user, item, score) 20 | package com.spotify.bdrc.pipeline 21 | 22 | import com.spotify.bdrc.util.Records.Rating 23 | import com.spotify.scio.values.SCollection 24 | import com.twitter.algebird.Semigroup 25 | import com.twitter.scalding.TypedPipe 26 | import org.apache.spark.rdd.RDD 27 | 28 | object AverageScorePerItem { 29 | 30 | // ## Scalding 31 | def scalding(input: TypedPipe[Rating]): TypedPipe[(String, Double)] = { 32 | input 33 | .groupBy(_.user) 34 | // Map into (sum, count) 35 | .mapValues(x => (x.score, 1L)) 36 | // Sum both per key with an implicit `Semigroup[(Double, Long)]` 37 | .sum 38 | // Map (sum, count) into average 39 | .mapValues(p => p._1 / p._2) 40 | .toTypedPipe 41 | } 42 | 43 | // ## Scalding with Algebird `Aggregator` 44 | def scaldingWithAlgebird(input: TypedPipe[Rating]): TypedPipe[(String, Double)] = { 45 | import com.twitter.algebird.AveragedValue 46 | input 47 | .groupBy(_.user) 48 | // Map values into `Double` 49 | .mapValues(_.score) 50 | // Aggregate average per key 51 | .aggregate(AveragedValue.aggregator) 52 | .toTypedPipe 53 | } 54 | 55 | // ## Scio 56 | def scio(input: SCollection[Rating]): SCollection[(String, Double)] = { 57 | input 58 | .keyBy(_.user) 59 | // Map into (sum, count) 60 | .mapValues(x => (x.score, 1L)) 61 | // Sum both per key with an implicit `Semigroup[(Double, Long)]` 62 | .sumByKey 63 | // Map (sum, count) into average 64 | .mapValues(p => p._1 / p._2) 65 | } 66 | 67 | // ## Spark 68 | // Summon an Algebird `Semigroup[(Double, Long)]` with implicit argument 69 | def spark(input: RDD[Rating])(implicit sg: Semigroup[(Double, Long)]): RDD[(String, Double)] = { 70 | input 71 | .keyBy(_.user) 72 | // Map into (sum, count) 73 | .mapValues(x => (x.score, 1L)) 74 | // Reduce both per key with `plus = (T, T) => T` where `T` is `(Double, Long)` 75 | .reduceByKey(sg.plus) // plus: (T, T) => T where T is (Double, Long) 76 | // Map (sum, count) into average 77 | .mapValues(p => p._1 / p._2) 78 | } 79 | 80 | // ## Spark with Algebird `Aggregator` 81 | def sparkWithAlgebird(input: RDD[Rating]): RDD[(String, Double)] = { 82 | import com.twitter.algebird.AveragedValue 83 | import com.twitter.algebird.spark._ 84 | input 85 | .keyBy(_.user) 86 | .mapValues(_.score) 87 | // Map values into `Double` 88 | .algebird 89 | // Aggregate average per key 90 | .aggregateByKey(AveragedValue.aggregator) 91 | } 92 | 93 | } 94 | -------------------------------------------------------------------------------- /src/main/scala/com/spotify/bdrc/pipeline/BloomFilterSetDifference.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 Spotify AB. 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 com.spotify.bdrc.pipeline 19 | 20 | import com.spotify.scio.values.SCollection 21 | import com.twitter.algebird._ 22 | import com.twitter.scalding.TypedPipe 23 | import org.apache.spark.rdd.RDD 24 | 25 | /** 26 | * Filter LHS by removing items that exist in the RHS using a Bloom Filter. 27 | * 28 | * Inputs are collections of strings. 29 | */ 30 | object BloomFilterSetDifference { 31 | 32 | def scalding(lhs: TypedPipe[String], rhs: TypedPipe[String]): TypedPipe[String] = { 33 | val width = BloomFilter.optimalWidth(1000, 0.01).get 34 | val numHashes = BloomFilter.optimalNumHashes(1000, width) 35 | lhs 36 | .cross(rhs.aggregate(BloomFilterAggregator(numHashes, width))) 37 | .filter { case (s, bf) => bf.contains(s).isTrue } 38 | .keys 39 | } 40 | 41 | def scio(lhs: SCollection[String], rhs: SCollection[String]): SCollection[String] = { 42 | val width = BloomFilter.optimalWidth(1000, 0.01).get 43 | val numHashes = BloomFilter.optimalNumHashes(1000, width) 44 | lhs 45 | .cross(rhs.aggregate(BloomFilterAggregator[String](numHashes, width))) 46 | .filter { case (s, bf) => bf.contains(s).isTrue } 47 | .keys 48 | } 49 | 50 | def spark(lhs: RDD[String], rhs: RDD[String]): RDD[String] = { 51 | import com.twitter.algebird.spark._ 52 | val width = BloomFilter.optimalWidth(1000, 0.01).get 53 | val numHashes = BloomFilter.optimalNumHashes(1000, width) 54 | val bf = rhs.algebird.aggregate(BloomFilterAggregator(numHashes, width)) 55 | lhs.filter(s => bf.contains(s).isTrue) 56 | } 57 | 58 | } 59 | -------------------------------------------------------------------------------- /src/main/scala/com/spotify/bdrc/pipeline/Count.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Spotify AB. 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 | // Example: Count Number of Items 19 | // Input is a collection of (user, item, score) 20 | package com.spotify.bdrc.pipeline 21 | 22 | import com.spotify.bdrc.util.Records.Rating 23 | import com.spotify.scio.values.SCollection 24 | import com.twitter.scalding.TypedPipe 25 | import org.apache.spark.rdd.RDD 26 | 27 | object Count { 28 | 29 | // ## Scalding 30 | def scalding(input: TypedPipe[Rating]): TypedPipe[Long] = { 31 | input 32 | .map(_ => 1L) 33 | // Sum with an implicit `Semigroup[Long]` 34 | .sum 35 | .toTypedPipe 36 | } 37 | 38 | // ## Scalding with Algebird `Aggregator` 39 | def scaldingWithAlgebird(input: TypedPipe[Rating]): TypedPipe[Long] = { 40 | import com.twitter.algebird.Aggregator.size 41 | input 42 | .aggregate(size) 43 | .toTypedPipe 44 | } 45 | 46 | // ## Scio 47 | def scio(input: SCollection[Rating]): SCollection[Long] = 48 | input.count 49 | 50 | // ## Scio with Algebird `Aggregator` 51 | def scioWithAlgebird(input: SCollection[Rating]): SCollection[Long] = { 52 | import com.twitter.algebird.Aggregator.size 53 | input 54 | .aggregate(size) 55 | } 56 | 57 | // # Spark 58 | def spark(input: RDD[Rating]): Long = { 59 | input 60 | // `count` is an action and collects data back to the driver node 61 | .count 62 | } 63 | 64 | // ## Spark with Algebird `Aggregator` 65 | def sparkWithAlgebird(input: RDD[Rating]): Long = { 66 | import com.twitter.algebird.Aggregator.size 67 | import com.twitter.algebird.spark._ 68 | input.algebird 69 | // `aggregate` is an action and collects data back to the driver node 70 | .aggregate(size) 71 | } 72 | 73 | } 74 | -------------------------------------------------------------------------------- /src/main/scala/com/spotify/bdrc/pipeline/CountDistinctItems.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Spotify AB. 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 | // Example: Count Number of Distinct Items 19 | // Input is a collection of (user, item, score) 20 | package com.spotify.bdrc.pipeline 21 | 22 | import com.google.common.base.Charsets 23 | import com.spotify.bdrc.util.Records.Rating 24 | import com.spotify.scio.values.SCollection 25 | import com.twitter.scalding.TypedPipe 26 | import org.apache.spark.rdd.RDD 27 | 28 | object CountDistinctItems { 29 | 30 | // ## Scalding Exact Approach 31 | def scalding(input: TypedPipe[Rating]): TypedPipe[Long] = { 32 | input 33 | .map(_.item) 34 | // Remove duplicates, requires a shuffle 35 | .distinct 36 | .map(_ => 1L) 37 | // Sum with an implicit `Semigroup[Long]` 38 | .sum 39 | .toTypedPipe 40 | } 41 | 42 | // ## Scalding Approximate Approach 43 | def scaldingApproxWithAlgebird(input: TypedPipe[Rating]): TypedPipe[Double] = { 44 | import com.twitter.algebird.HyperLogLogAggregator 45 | val aggregator = HyperLogLogAggregator.sizeAggregator(bits = 12) 46 | input 47 | // `HyperLogLog` expects bytes input 48 | .map(_.item.getBytes(Charsets.UTF_8)) 49 | // Aggregate globally into a `Double` 50 | .aggregate(aggregator) 51 | .toTypedPipe 52 | } 53 | 54 | // ## Scio Exact Approach 55 | def scio(input: SCollection[Rating]): SCollection[Long] = { 56 | input 57 | .map(_.item) 58 | .distinct 59 | .count 60 | } 61 | 62 | // ## Scio Approximate Approach 63 | def scioApprox(input: SCollection[Rating]): SCollection[Long] = { 64 | input 65 | .map(_.item) 66 | .countApproxDistinct() 67 | } 68 | 69 | // ## Spark Exact Approach 70 | def spark(input: RDD[Rating]): Long = { 71 | input 72 | .map(_.item) 73 | .distinct() 74 | .count() 75 | } 76 | 77 | // ## Spark Approximate Approach 78 | def sparkApprox(input: RDD[Rating]): Long = { 79 | input 80 | .map(_.item) 81 | .countApproxDistinct() 82 | } 83 | 84 | } 85 | -------------------------------------------------------------------------------- /src/main/scala/com/spotify/bdrc/pipeline/CountUsers.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Spotify AB. 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 | // Example: Count the Number of Items of a Given User 19 | // Input is a collection of (user, item, score) 20 | package com.spotify.bdrc.pipeline 21 | 22 | import com.spotify.bdrc.util.Records.Rating 23 | import com.spotify.scio.values.SCollection 24 | import com.twitter.scalding.TypedPipe 25 | import org.apache.spark.rdd.RDD 26 | 27 | object CountUsers { 28 | 29 | // ## Scalding 30 | def scalding(input: TypedPipe[Rating]): TypedPipe[Long] = { 31 | input 32 | .filter(_.user == "Smith") 33 | .map(_ => 1L) 34 | // Sum with an implicit `Semigroup[Long]` 35 | .sum 36 | .toTypedPipe 37 | } 38 | 39 | // ## Sclading with Algebird `Aggregator` 40 | def scaldingWithAlgebird(input: TypedPipe[Rating]): TypedPipe[Long] = { 41 | import com.twitter.algebird.Aggregator.count 42 | input 43 | // Aggregate globally into a single `Long` 44 | .aggregate(count(_.user == "Smith")) 45 | .toTypedPipe 46 | } 47 | 48 | def scio(input: SCollection[Rating]): SCollection[Long] = { 49 | input 50 | .filter(_.user == "Smith") 51 | .count 52 | } 53 | 54 | // ## Scio with Algebird `Aggregator` 55 | def scioWithAlgebird(input: SCollection[Rating]): SCollection[Long] = { 56 | import com.twitter.algebird.Aggregator.count 57 | input 58 | // Aggregate globally into a single `Long` 59 | .aggregate(count((_: Rating).user == "Smith")) 60 | } 61 | 62 | // ## Spark 63 | def spark(input: RDD[Rating]): Long = { 64 | input 65 | .filter(_.user == "Smith") 66 | // `count` is an action and collects data back to the driver node 67 | .count() 68 | } 69 | 70 | // ## Spark with Algebird `Aggregator` 71 | def sparkWithAlgebird(input: RDD[Rating]): Long = { 72 | import com.twitter.algebird.Aggregator.count 73 | import com.twitter.algebird.spark._ 74 | input.algebird 75 | // `aggregate` is an action and collects data back to the driver node 76 | .aggregate(count(_.user == "Smith")) 77 | } 78 | 79 | } 80 | -------------------------------------------------------------------------------- /src/main/scala/com/spotify/bdrc/pipeline/DistinctItems.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Spotify AB. 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 | // Example: Compute Collection of Distinct Items 19 | // Input is a collection of (user, item, score) 20 | package com.spotify.bdrc.pipeline 21 | 22 | import com.spotify.bdrc.util.Records.Rating 23 | import com.spotify.scio.values.SCollection 24 | import com.twitter.scalding.TypedPipe 25 | import org.apache.spark.rdd.RDD 26 | 27 | object DistinctItems { 28 | 29 | // ## Scalding 30 | def scalding(input: TypedPipe[Rating]): TypedPipe[String] = { 31 | input 32 | .map(_.item) 33 | .distinct 34 | } 35 | 36 | // ## Scio 37 | def scio(input: SCollection[Rating]): SCollection[String] = { 38 | input 39 | .map(_.item) 40 | .distinct 41 | } 42 | 43 | // ## Spark 44 | def spark(input: RDD[Rating]): RDD[String] = { 45 | input 46 | .map(_.item) 47 | .distinct() 48 | } 49 | 50 | } 51 | -------------------------------------------------------------------------------- /src/main/scala/com/spotify/bdrc/pipeline/FieldStatistics.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Spotify AB. 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 | // Example: Compute Basic Descriptive Statistics for Each Field 19 | // Input is a collection of case classes 20 | package com.spotify.bdrc.pipeline 21 | 22 | import com.spotify.scio.coders.Coder 23 | import com.spotify.scio.values.SCollection 24 | import com.twitter.scalding.TypedPipe 25 | import org.apache.spark.rdd.RDD 26 | 27 | object FieldStatistics { 28 | 29 | case class User(age: Int, income: Double, score: Double) 30 | case class Stats(max: Double, min: Double, mean: Double, stddev: Double) 31 | case class UserStats(age: Stats, income: Stats, score: Stats) 32 | 33 | import com.twitter.algebird._ 34 | implicit val momentsCoder: Coder[Moments] = Coder.kryo[Moments] 35 | 36 | // ## Algebird `Aggregator` 37 | def aggregator = { 38 | // Create 3 `Aggregator`s on `age` field with different logic 39 | 40 | // The first 2 are of type `Aggregator[User, _, Int]` which means it takes `User` as input and 41 | // generates `Int` as output. The last one is of type `Aggregator[User, _, Moments]`, 42 | // where `Moments` include count, mean, standard deviation, etc. The input `User` is prepared 43 | // with a `User => Int` function `_.age`. 44 | val maxAgeOp = Aggregator.max[Int].composePrepare[User](_.age) 45 | val minAgeOp = Aggregator.min[Int].composePrepare[User](_.age) 46 | val momentsAgeOp = Moments.aggregator.composePrepare[User](_.age) 47 | 48 | // Create 3 `Aggregator`s on `income` field with different logic 49 | val maxIncomeOp = Aggregator.max[Double].composePrepare[User](_.income) 50 | val minIncomeOp = Aggregator.min[Double].composePrepare[User](_.income) 51 | val momentsIncomeOp = Moments.aggregator.composePrepare[User](_.income) 52 | 53 | // Create 3 `Aggregator`s on `score` field with different logic 54 | val maxScoreOp = Aggregator.max[Double].composePrepare[User](_.score) 55 | val minScoreOp = Aggregator.min[Double].composePrepare[User](_.score) 56 | val momentsScoreOp = Moments.aggregator.composePrepare[User](_.score) 57 | 58 | // Apply 12 `Aggregator`s on the same input, present result tuple 12 as `UserStats`. 59 | MultiAggregator( 60 | maxAgeOp, 61 | minAgeOp, 62 | momentsAgeOp, 63 | maxIncomeOp, 64 | minIncomeOp, 65 | momentsIncomeOp, 66 | maxScoreOp, 67 | minScoreOp, 68 | momentsScoreOp 69 | ).andThenPresent { t => 70 | val (maxAge, minAge, mAge, maxIncome, minIncome, mIncome, maxScore, minScore, mScore) = t 71 | UserStats( 72 | age = Stats(maxAge, minAge, mAge.mean, mAge.stddev), 73 | income = Stats(maxIncome, minIncome, mIncome.mean, mIncome.stddev), 74 | score = Stats(maxScore, minScore, mScore.mean, mScore.stddev) 75 | ) 76 | } 77 | } 78 | 79 | // ## Scalding 80 | def scalding(input: TypedPipe[User]): TypedPipe[UserStats] = 81 | input.aggregate(aggregator) 82 | 83 | // ## Scio 84 | def scio(input: SCollection[User]): SCollection[UserStats] = 85 | input.aggregate(aggregator) 86 | 87 | // ## Spark 88 | def spark(input: RDD[User]): UserStats = { 89 | // Compute each field separately, potentially in-efficient if input is not cached 90 | val s1 = input.map(_.age).stats() 91 | val s2 = input.map(_.income).stats() 92 | val s3 = input.map(_.score).stats() 93 | UserStats( 94 | age = Stats(s1.max, s1.min, s1.mean, s1.stdev), 95 | income = Stats(s2.max, s2.min, s2.mean, s2.stdev), 96 | score = Stats(s3.max, s3.min, s3.mean, s3.stdev) 97 | ) 98 | } 99 | 100 | // ## Spark with Algebird `Aggregator` 101 | def sparkAlgebird(input: RDD[User]): UserStats = { 102 | import com.twitter.algebird.spark._ 103 | input.algebird.aggregate(aggregator) 104 | } 105 | 106 | } 107 | -------------------------------------------------------------------------------- /src/main/scala/com/spotify/bdrc/pipeline/FindMedian.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Spotify AB. 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 com.spotify.bdrc.pipeline 19 | 20 | import com.spotify.scio.values.SCollection 21 | import com.twitter.scalding.TypedPipe 22 | import org.apache.spark.rdd.RDD 23 | 24 | /** 25 | * Compute the median of a collection of numbers. 26 | */ 27 | object FindMedian { 28 | 29 | // Computing the exact median is very expensive as it requires sorting and counting elements. 30 | // QTree is a compact data structure for approximate quantile and range queries. 31 | 32 | def scalding(input: TypedPipe[Long]): TypedPipe[(Double, Double)] = { 33 | import com.twitter.algebird._ 34 | input 35 | .aggregate(QTreeAggregator[Long](0.5)) 36 | .map(i => (i.lower.lower, i.upper.upper)) 37 | } 38 | 39 | def scio(input: SCollection[Long]): SCollection[(Double, Double)] = { 40 | import com.twitter.algebird._ 41 | input 42 | .aggregate(QTreeAggregator[Long](0.5)) 43 | .map(i => (i.lower.lower, i.upper.upper)) 44 | } 45 | 46 | def spark(input: RDD[Long]): (Double, Double) = { 47 | import com.twitter.algebird._ 48 | import com.twitter.algebird.spark._ 49 | val i = input.algebird.aggregate(QTreeAggregator[Long](0.5)) 50 | (i.lower.lower, i.upper.upper) 51 | } 52 | 53 | // TODO: exact version 54 | 55 | } 56 | -------------------------------------------------------------------------------- /src/main/scala/com/spotify/bdrc/pipeline/InvertedIndex.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 Spotify AB. 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 | // Example: Build Inverted Index 19 | // Build inverted index from a corpus of text documents 20 | 21 | // Input is a collection of (id, text) 22 | package com.spotify.bdrc.pipeline 23 | 24 | import com.spotify.scio.values.SCollection 25 | import com.twitter.scalding.TypedPipe 26 | import org.apache.spark.rdd.RDD 27 | 28 | object InvertedIndex { 29 | 30 | case class Document(id: Int, text: String) 31 | case class Posting(word: String, ids: Seq[Int]) 32 | 33 | // ## Scalding 34 | def scalding(input: TypedPipe[Document]): TypedPipe[Posting] = { 35 | input 36 | // Split text and output (word, document ID) 37 | .flatMap(d => d.text.split("[^a-zA-Z']+").map(w => (w, d.id))) 38 | // Group and convert document IDs per key to `List[Int]` 39 | .group 40 | .toList 41 | .map(Posting.tupled) 42 | } 43 | 44 | // ## Scio 45 | def scio(input: SCollection[Document]): SCollection[Posting] = { 46 | input 47 | // Split text and output (word, document ID) 48 | .flatMap(d => d.text.split("[^a-zA-Z']+").map(w => (w, d.id))) 49 | // Group document IDs per key into `Iterable[Int]` 50 | .groupByKey 51 | .map(kv => Posting(kv._1, kv._2.toSeq)) 52 | } 53 | 54 | // ## Spark 55 | def spark(input: RDD[Document]): RDD[Posting] = { 56 | input 57 | // Split text and output (word, document ID) 58 | .flatMap(d => d.text.split("[^a-zA-Z']+").map(w => (w, d.id))) 59 | // Group document IDs per key into `Iterable[Int]` 60 | .groupByKey() 61 | .map(kv => Posting(kv._1, kv._2.toSeq)) 62 | } 63 | 64 | } 65 | -------------------------------------------------------------------------------- /src/main/scala/com/spotify/bdrc/pipeline/JoinLogAndMetadata.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Spotify AB. 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 | // Example: Join Log and Metadata Datasets 19 | // Compute average age of users who listened to a track by joining log event and user metadata. 20 | // 21 | // - LHS input is a large collection of (user, page, timestamp). 22 | // - RHS input is a small collection of (user, age). 23 | package com.spotify.bdrc.pipeline 24 | 25 | import com.spotify.bdrc.util.Records.{LogEvent, UserMeta} 26 | import com.spotify.scio.values.SCollection 27 | import com.twitter.scalding.TypedPipe 28 | import org.apache.spark.rdd.RDD 29 | 30 | object JoinLogAndMetadata { 31 | 32 | // ## Scalding Naive Approach 33 | def scaldingNaive( 34 | left: TypedPipe[LogEvent], 35 | right: TypedPipe[UserMeta] 36 | ): TypedPipe[(String, Double)] = { 37 | import com.twitter.algebird.AveragedValue 38 | left 39 | .groupBy(_.user) 40 | // Join as (user, (LogEvent, UserMeta)) 41 | .join(right.groupBy(_.user)) 42 | // Drop user key 43 | .values 44 | // Map into (track, age) 45 | .map { case (logEvent, userMeta) => 46 | (logEvent.track, userMeta.age.toDouble) 47 | } 48 | .group 49 | // Aggregate average age per track 50 | .aggregate(AveragedValue.aggregator) 51 | .toTypedPipe 52 | } 53 | 54 | // ## Scalding with Hash Join 55 | // `hashJoin` replicates the smaller RHS to all mappers on the LHS 56 | def scaldingHashJoin( 57 | left: TypedPipe[LogEvent], 58 | right: TypedPipe[UserMeta] 59 | ): TypedPipe[(String, Double)] = { 60 | import com.twitter.algebird.AveragedValue 61 | 62 | // Map out fields to avoid shuffing large objects 63 | val lhs = left.map(e => (e.user, e.track)) 64 | // Force to disk to avoid repeating the same computation on each mapper on the LHS 65 | val rhs = right.map(u => (u.user, u.age.toDouble)).forceToDisk 66 | 67 | lhs 68 | .hashJoin(rhs) 69 | .values 70 | .group 71 | .aggregate(AveragedValue.aggregator) 72 | .toTypedPipe 73 | } 74 | 75 | // ## Scio Naive Approach 76 | def scioNaive( 77 | left: SCollection[LogEvent], 78 | right: SCollection[UserMeta] 79 | ): SCollection[(String, Double)] = { 80 | import com.twitter.algebird.AveragedValue 81 | val lhs = left.map(e => (e.user, e.track)) 82 | val rhs = right.map(u => (u.user, u.age.toDouble)) 83 | // Join as (user, (track, age)) 84 | lhs 85 | .join(rhs) 86 | // Drop user key to make track as new key in (track, age) 87 | .values 88 | // Aggregate average age per track 89 | .aggregateByKey(AveragedValue.aggregator) 90 | } 91 | 92 | // ## Scio with Side Input 93 | // Side input makes RHS available on all workers 94 | def scioSideInput( 95 | left: SCollection[LogEvent], 96 | right: SCollection[UserMeta] 97 | ): SCollection[(String, Double)] = { 98 | import com.twitter.algebird.AveragedValue 99 | 100 | // Convert RHS to a side input of `Map[String, Double]` 101 | val rhs = right.map(u => (u.user, u.age.toDouble)).asMapSideInput 102 | 103 | // Replicate RHS to each worker 104 | left 105 | .withSideInputs(rhs) 106 | // Access side input via the context 107 | .map { case (e, sideContext) => (e.track, sideContext(rhs).getOrElse(e.user, 0.0)) } 108 | // Convert back to regular SCollection 109 | .toSCollection 110 | .aggregateByKey(AveragedValue.aggregator) 111 | } 112 | 113 | // ## Scio with Hash Join 114 | // `hashJoin` is a short cut to the side input approach 115 | def scioHashJoin( 116 | left: SCollection[LogEvent], 117 | right: SCollection[UserMeta] 118 | ): SCollection[(String, Double)] = { 119 | import com.twitter.algebird.AveragedValue 120 | val lhs = left.map(e => (e.user, e.track)) 121 | val rhs = right.map(u => (u.user, u.age.toDouble)) 122 | lhs 123 | .hashJoin(rhs) 124 | .values 125 | .aggregateByKey(AveragedValue.aggregator) 126 | } 127 | 128 | // ## Spark Naive Approach 129 | def sparkNaive(left: RDD[LogEvent], right: RDD[UserMeta]): RDD[(String, Double)] = { 130 | import com.twitter.algebird.spark._ 131 | import com.twitter.algebird.AveragedValue 132 | val lhs = left.map(e => (e.user, e.track)) 133 | val rhs = right.map(u => (u.user, u.age.toDouble)) 134 | // Join as (user, (track, age)) 135 | lhs 136 | .join(rhs) 137 | // Drop user key to make track as new key in (track, age) 138 | .values 139 | .algebird 140 | // Aggregate average age per track 141 | .aggregateByKey(AveragedValue.aggregator) 142 | } 143 | 144 | // ## Spark with Broadcast Variable 145 | def sparkBroadcast(left: RDD[LogEvent], right: RDD[UserMeta]): RDD[(String, Double)] = { 146 | import com.twitter.algebird.spark._ 147 | import com.twitter.algebird.AveragedValue 148 | 149 | // Retrieve `SparkContext` for creating broadcast variable 150 | val sc = left.context 151 | 152 | // Collect RHS to driver memory and broadcast back to workers 153 | val map = right.map(u => (u.user, u.age.toDouble)).collectAsMap() 154 | val b = sc.broadcast(map) 155 | 156 | left 157 | // In-memory lookup on each worker 158 | .map(e => (e.track, b.value.getOrElse(e.user, 0.0))) 159 | .algebird 160 | .aggregateByKey(AveragedValue.aggregator) 161 | } 162 | 163 | } 164 | -------------------------------------------------------------------------------- /src/main/scala/com/spotify/bdrc/pipeline/JoinLogs.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Spotify AB. 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 | // Example: Join Two Log Datasets and Compute Action Sequence 19 | // Given two log datasets of play track and save track events, compute tracks that a user saved 20 | // after playing in a session. 21 | 22 | // Inputs are collections of (user, item, timestamp). 23 | package com.spotify.bdrc.pipeline 24 | 25 | import com.spotify.bdrc.util.Records.LogEvent 26 | import com.spotify.scio.values.SCollection 27 | import com.twitter.scalding.TypedPipe 28 | import org.apache.spark.rdd.RDD 29 | 30 | object JoinLogs { 31 | 32 | val gapDuration = 3600000 33 | 34 | // Detect if a pair of (event type, LogEvent) tuples match a play and save sequence 35 | def detectPlaySaveSequence(pair: Seq[(String, LogEvent)]): Option[String] = { 36 | val Seq(first, second) = pair 37 | if ( 38 | first._1 == "play" && second._1 == "save" && first._2.track == second._2.track && 39 | second._2.timestamp - first._2.timestamp <= gapDuration 40 | ) { 41 | Some(first._2.track) 42 | } else { 43 | None 44 | } 45 | } 46 | 47 | // ## Scalding 48 | def scalding( 49 | playEvents: TypedPipe[LogEvent], 50 | saveEvents: TypedPipe[LogEvent] 51 | ): TypedPipe[(String, String)] = { 52 | // Map inputs to key-values and add event type information 53 | val plays = playEvents.map(e => (e.user, ("play", e))).group 54 | val saves = saveEvents.map(e => (e.user, ("save", e))).group 55 | 56 | plays 57 | .cogroup(saves) { (user, p, s) => 58 | // `Iterable`s of play and save events for the user 59 | (p ++ s).toList 60 | .sortBy(_._2.timestamp) 61 | // Neighboring pairs 62 | .sliding(2) 63 | .flatMap(detectPlaySaveSequence) 64 | } 65 | .toTypedPipe 66 | } 67 | 68 | // ## Scio 69 | def scio( 70 | playEvents: SCollection[LogEvent], 71 | saveEvents: SCollection[LogEvent] 72 | ): SCollection[(String, String)] = { 73 | // Map inputs to key-values and add event type information 74 | val plays = playEvents.map(e => (e.user, ("play", e))) 75 | val saves = saveEvents.map(e => (e.user, ("save", e))) 76 | 77 | plays 78 | .cogroup(saves) 79 | // `Iterable`s of play and save events for the user 80 | .flatMapValues { case (p, s) => 81 | (p ++ s).toList 82 | .sortBy(_._2.timestamp) 83 | // Neighboring pairs 84 | .sliding(2) 85 | .flatMap(detectPlaySaveSequence) 86 | } 87 | } 88 | 89 | // ## Spark 90 | def spark(playEvents: RDD[LogEvent], saveEvents: RDD[LogEvent]): RDD[(String, String)] = { 91 | // Map inputs to key-values and add event type information 92 | val plays = playEvents.map(e => (e.user, ("play", e))) 93 | val saves = saveEvents.map(e => (e.user, ("save", e))) 94 | 95 | plays 96 | .cogroup(saves) 97 | .flatMapValues { case (p, s) => 98 | // `Iterable`s of play and save events for the user 99 | (p ++ s).toList 100 | .sortBy(_._2.timestamp) 101 | // Neighboring pairs 102 | .sliding(2) 103 | .flatMap(detectPlaySaveSequence) 104 | } 105 | } 106 | 107 | } 108 | -------------------------------------------------------------------------------- /src/main/scala/com/spotify/bdrc/pipeline/MaxItemPerUser.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Spotify AB. 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 | // Example: Compute One Item with Max Score per User 19 | // Input is a collection of (user, item, score) 20 | package com.spotify.bdrc.pipeline 21 | 22 | import com.spotify.bdrc.util.Records.Rating 23 | import com.spotify.scio.values.SCollection 24 | import com.twitter.scalding.TypedPipe 25 | import org.apache.spark.rdd.RDD 26 | 27 | object MaxItemPerUser { 28 | 29 | // ## Scalding 30 | def scalding(input: TypedPipe[Rating]): TypedPipe[Rating] = { 31 | input 32 | .groupBy(_.user) 33 | // Reduce items per key by picking the side with higher score for each pair of input 34 | .reduce((x, y) => if (x.score > y.score) x else y) 35 | .values 36 | } 37 | 38 | // ## Scalding with Algebird `Aggregator` 39 | def scaldingWithAlgebird(input: TypedPipe[Rating]): TypedPipe[Rating] = { 40 | import com.twitter.algebird.Aggregator.maxBy 41 | input 42 | .groupBy(_.user) 43 | // Aggregate per key into a single `Rating` based on `Double` value via `_.score` 44 | .aggregate(maxBy(_.score)) 45 | .values 46 | } 47 | 48 | // ## Scio 49 | def scio(input: SCollection[Rating]): SCollection[Rating] = { 50 | input 51 | .keyBy(_.user) 52 | // Compute top one item per key as an `Iterable[Rating]` 53 | .topByKey(1)(Ordering.by(_.score)) 54 | // Drop user key 55 | .values 56 | // Flatten result `Iterable[Rating]` 57 | .flatten 58 | } 59 | 60 | // ## Scio with Algebird `Aggregator` 61 | def scioWithAlgebird(input: SCollection[Rating]): SCollection[Rating] = { 62 | import com.twitter.algebird.Aggregator.maxBy 63 | input 64 | .keyBy(_.user) 65 | // Aggregate per key into a single `Rating` based on `Double` value via `_.score`. Explicit 66 | // type due to type inference limitation. 67 | .aggregateByKey(maxBy { x: Rating => x.score }) 68 | .values 69 | } 70 | 71 | // ## Spark 72 | def spark(input: RDD[Rating]): RDD[Rating] = { 73 | input 74 | .keyBy(_.user) 75 | // Reduce items per key by picking the side with higher score for each pair of input 76 | .reduceByKey((x, y) => if (x.score > y.score) x else y) 77 | .values 78 | } 79 | 80 | // ## Spark with Algebird `Aggregator` 81 | def sparkWithAlgebird(input: RDD[Rating]): RDD[Rating] = { 82 | import com.twitter.algebird.Aggregator.maxBy 83 | import com.twitter.algebird.spark._ 84 | input 85 | .keyBy(_.user) 86 | .algebird 87 | // Aggregate per key into a single `Rating` based on `Double` value via `_.score`. Explicit 88 | // type due to type inference limitation. 89 | .aggregateByKey(maxBy { x: Rating => x.score }) 90 | .values 91 | } 92 | 93 | // ## Spark with MLLib 94 | def sparkWithMllib(input: RDD[Rating]): RDD[Rating] = { 95 | import org.apache.spark.mllib.rdd.MLPairRDDFunctions._ 96 | input 97 | .keyBy(_.user) 98 | // From `spark-mllib`, compute top K per key with a priority queue 99 | .topByKey(1)(Ordering.by(_.score)) 100 | .flatMap(_._2) 101 | } 102 | 103 | } 104 | -------------------------------------------------------------------------------- /src/main/scala/com/spotify/bdrc/pipeline/MinItemPerUser.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Spotify AB. 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 | // Example: Compute One Item with Min Score per User 19 | // Input is a collection of (user, item, score) 20 | package com.spotify.bdrc.pipeline 21 | 22 | import com.spotify.bdrc.util.Records.Rating 23 | import com.spotify.scio.values.SCollection 24 | import com.twitter.scalding.TypedPipe 25 | import org.apache.spark.rdd.RDD 26 | 27 | object MinItemPerUser { 28 | 29 | // ## Scalding 30 | def scalding(input: TypedPipe[Rating]): TypedPipe[Rating] = { 31 | input 32 | .groupBy(_.user) 33 | // Reduce items per key by picking the side with lower score for each pair of input 34 | .reduce((x, y) => if (x.score < y.score) x else y) 35 | .values 36 | } 37 | 38 | // ## Scalding with Algebird `Aggregator` 39 | def scaldingWithAlgebird(input: TypedPipe[Rating]): TypedPipe[Rating] = { 40 | import com.twitter.algebird.Aggregator.minBy 41 | input 42 | .groupBy(_.user) 43 | // Aggregate per key into a single `Rating` based on `Double` value via `_.score` 44 | .aggregate(minBy(_.score)) 45 | .values 46 | } 47 | 48 | // ## Scio 49 | def scio(input: SCollection[Rating]): SCollection[Rating] = { 50 | input 51 | .keyBy(_.user) 52 | // Compute top one item per key as an `Iterable[Rating]` with a reverse comparator 53 | .topByKey(1)(Ordering.by(-_.score)) 54 | // Drop user key 55 | .values 56 | // Flatten result `Iterable[Rating]` 57 | .flatten 58 | } 59 | 60 | // ## Scio with Algebird `Aggregator` 61 | def scioWithAlgebird(input: SCollection[Rating]): SCollection[Rating] = { 62 | import com.twitter.algebird.Aggregator.minBy 63 | input 64 | .keyBy(_.user) 65 | // Aggregate per key into a single `Rating` based on `Double` value via `_.score`. Explicit 66 | // type due to type inference limitation. 67 | .aggregateByKey(minBy { x: Rating => x.score }) 68 | .values 69 | } 70 | 71 | // ## Spark 72 | def spark(input: RDD[Rating]): RDD[Rating] = { 73 | input 74 | .keyBy(_.user) 75 | // Reduce items per key by picking the side with lower score for each pair of input 76 | .reduceByKey((x, y) => if (x.score < y.score) x else y) 77 | .values 78 | } 79 | 80 | // ## Spark with Algebird `Aggregator` 81 | def sparkWithAlgebird(input: RDD[Rating]): RDD[Rating] = { 82 | import com.twitter.algebird.Aggregator.minBy 83 | import com.twitter.algebird.spark._ 84 | input 85 | .keyBy(_.user) 86 | .algebird 87 | // Aggregate per key into a single `Rating` based on `Double` value via `_.score`. Explicit 88 | // type due to type inference limitation. 89 | .aggregateByKey(minBy { x: Rating => x.score }) 90 | .values 91 | } 92 | 93 | // ## Spark with MLLib 94 | def sparkWithMllib(input: RDD[Rating]): RDD[Rating] = { 95 | import org.apache.spark.mllib.rdd.MLPairRDDFunctions._ 96 | input 97 | .keyBy(_.user) 98 | // From `spark-mllib`, compute top K per key with a priority queue and a reverse comparator 99 | .topByKey(1)(Ordering.by(-_.score)) 100 | .flatMap(_._2) 101 | } 102 | 103 | } 104 | -------------------------------------------------------------------------------- /src/main/scala/com/spotify/bdrc/pipeline/PageRank.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Spotify AB. 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 com.spotify.bdrc.pipeline 19 | 20 | import com.spotify.scio.values.SCollection 21 | import com.twitter.scalding.TypedPipe 22 | import org.apache.spark.rdd.RDD 23 | 24 | /** 25 | * Classic PageRank. 26 | * 27 | * Input is a collection of (source URL, destination URL). 28 | */ 29 | object PageRank { 30 | 31 | val iterations = 10 32 | val dampingFactor = 0.85 33 | 34 | def scalding(input: TypedPipe[(String, String)]): TypedPipe[(String, Double)] = { 35 | val links = input.group.toList // (src URL, list of dst URL) 36 | var ranks = input.keys.distinct.map((_, 1.0)) // (src URL, 1.0) 37 | 38 | for (i <- 1 to 10) { 39 | val contribs = links 40 | .join(ranks) 41 | .toTypedPipe 42 | .values 43 | // re-distribute rank of src URL among collection of dst URLs 44 | .flatMap { case (urls, rank) => 45 | val size = urls.size 46 | urls.map((_, rank / size)) 47 | } 48 | ranks = contribs.group.sum 49 | .mapValues((1 - dampingFactor) + dampingFactor * _) 50 | .toTypedPipe 51 | } 52 | 53 | ranks 54 | } 55 | 56 | def scio(input: SCollection[(String, String)]): SCollection[(String, Double)] = { 57 | val links = input.groupByKey 58 | var ranks = links.mapValues(_ => 1.0) 59 | 60 | for (i <- 1 to 10) { 61 | val contribs = links 62 | .join(ranks) 63 | .values 64 | .flatMap { case (urls, rank) => 65 | val size = urls.size 66 | urls.map((_, rank / size)) 67 | } 68 | ranks = contribs.sumByKey 69 | .mapValues((1 - dampingFactor) + dampingFactor * _) 70 | } 71 | 72 | ranks 73 | } 74 | 75 | def spark(input: RDD[(String, String)]): RDD[(String, Double)] = { 76 | val links = input 77 | .groupByKey() // (src URL, iterable of dst URL) 78 | .cache() // links is reused in every iteration 79 | var ranks = links.mapValues(_ => 1.0) // (src URL, 1.0) 80 | 81 | for (i <- 1 to 10) { 82 | val contribs = links 83 | .join(ranks) 84 | .values 85 | // re-distribute rank of src URL among collection of dst URLs 86 | .flatMap { case (urls, rank) => 87 | val size = urls.size 88 | urls.map((_, rank / size)) 89 | } 90 | ranks = contribs 91 | .reduceByKey(_ + _) 92 | .mapValues((1 - dampingFactor) + dampingFactor * _) 93 | } 94 | 95 | ranks 96 | } 97 | } 98 | -------------------------------------------------------------------------------- /src/main/scala/com/spotify/bdrc/pipeline/Sessions.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Spotify AB. 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 | // Example: Compute Session Duration and Number of Items from Log Data 19 | // Input is a collection of log events 20 | package com.spotify.bdrc.pipeline 21 | 22 | import com.spotify.bdrc.util.Records.LogEvent 23 | import com.spotify.scio.extra.Iterators._ 24 | import com.spotify.scio.values.SCollection 25 | import com.twitter.scalding.TypedPipe 26 | import org.apache.spark.rdd.RDD 27 | import org.joda.time.Instant 28 | 29 | import scala.collection.mutable 30 | 31 | object Sessions { 32 | 33 | val gapDuration = 3600000 34 | 35 | case class Session(user: String, duration: Long, numItems: Int) 36 | 37 | // Wrapper for `Iterator[LogEvent]` that group items into sessions 38 | class SessionIterator(self: Iterator[LogEvent]) extends Iterator[Seq[LogEvent]] { 39 | // `BufferedIterator` allows peak ahead 40 | private val bi = self.buffered 41 | override def hasNext: Boolean = bi.hasNext 42 | override def next(): Seq[LogEvent] = { 43 | val buf = mutable.Buffer(bi.next()) 44 | var last = buf.head.timestamp 45 | 46 | // Consume subsequent events until a gap is detected 47 | while (bi.hasNext && bi.head.timestamp - last < gapDuration) { 48 | val n = bi.next() 49 | buf.append(n) 50 | last = n.timestamp 51 | } 52 | buf 53 | } 54 | } 55 | 56 | // ## Scalding 57 | def scalding(input: TypedPipe[LogEvent]): TypedPipe[Session] = { 58 | input 59 | .groupBy(_.user) 60 | // `sortBy` uses Hadoop secondary sort to sort keys during shuffle 61 | .sortBy(_.timestamp) 62 | // Iterate over values lazily and group items into sessions 63 | .mapValueStream(new SessionIterator(_)) 64 | .toTypedPipe 65 | // Map over each (user, session items) 66 | .map { case (user, items) => 67 | Session(user, items.last.timestamp - items.head.timestamp, items.size) 68 | } 69 | } 70 | 71 | // ## Scio 72 | def scio(input: SCollection[LogEvent]): SCollection[Session] = { 73 | input 74 | // Values in `groupBy` are sorted by timestamp 75 | .timestampBy(e => new Instant(e.timestamp)) 76 | // No secondary sort in Scio, shuffle all items 77 | .groupBy(_.user) 78 | .flatMapValues { 79 | _.iterator 80 | // Generic version of `SessionIterator` from `scio-extra` 81 | .timeSeries(_.timestamp) 82 | .session(gapDuration) 83 | } 84 | // Map over each (user, session items) 85 | .map { case (user, items) => 86 | Session(user, items.last.timestamp - items.head.timestamp, items.size) 87 | } 88 | } 89 | 90 | // ## Spark 91 | def spark(input: RDD[LogEvent]): RDD[Session] = { 92 | input 93 | // No secondary sort in Spark, shuffle all items 94 | .groupBy(_.user) 95 | .flatMapValues { 96 | _ 97 | // Order of values after shuffle is not guaranteed 98 | .toList 99 | .sortBy(_.timestamp) 100 | .iterator 101 | // Generic version of `SessionIterator` from `scio-extra` 102 | .timeSeries(_.timestamp) 103 | .session(gapDuration) 104 | } 105 | // Map over each (user, session items) 106 | .map { case (user, items) => 107 | Session(user, items.last.timestamp - items.head.timestamp, items.size) 108 | } 109 | } 110 | 111 | } 112 | -------------------------------------------------------------------------------- /src/main/scala/com/spotify/bdrc/pipeline/Statistics.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Spotify AB. 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 | // Example: Compute Basic Descriptive Statistics 19 | // Input is a collection of (user, item, score) 20 | package com.spotify.bdrc.pipeline 21 | 22 | import com.spotify.bdrc.util.Records.Rating 23 | import com.spotify.scio.coders.Coder 24 | import com.spotify.scio.values.SCollection 25 | import com.twitter.scalding.TypedPipe 26 | import org.apache.spark.rdd.RDD 27 | 28 | object Statistics { 29 | 30 | case class Stats(max: Double, min: Double, sum: Double, count: Long, mean: Double, stddev: Double) 31 | 32 | import com.twitter.algebird._ 33 | implicit val momentsCoder: Coder[Moments] = Coder.kryo[Moments] 34 | 35 | // ## Algebird `Aggregator` 36 | def aggregator = { 37 | // Create 4 `Aggregator`s with different logic 38 | 39 | // The first 3 are of type `Aggregator[Rating, _, Double]` which means it takes `Rating` as 40 | // input and generates `Double` as output. The last one is of type 41 | // `Aggregator[Rating, _, Moments]`, where `Moments` include count, mean, standard deviation, 42 | // etc. The input `Rating` is prepared with a `Rating => Double` function `_.score`. 43 | val maxOp = Aggregator.max[Double].composePrepare[Rating](_.score) 44 | val minOp = Aggregator.min[Double].composePrepare[Rating](_.score) 45 | val sumOp = Aggregator.prepareMonoid[Rating, Double](_.score) 46 | val momentsOp = Moments.aggregator.composePrepare[Rating](_.score) 47 | 48 | // Apply 4 `Aggregator`s on the same input, present result tuple 4 of 49 | // `(Double, Double, Double, Moments)` as `Stats` 50 | MultiAggregator(maxOp, minOp, sumOp, momentsOp) 51 | .andThenPresent { case (max, min, sum, moments) => 52 | Stats(max, min, sum, moments.count, moments.mean, moments.stddev) 53 | } 54 | } 55 | 56 | // ## Scalding 57 | def scalding(input: TypedPipe[Rating]): TypedPipe[Stats] = 58 | input.aggregate(aggregator) 59 | 60 | // ## Scio 61 | def scio(input: SCollection[Rating]): SCollection[Stats] = { 62 | input 63 | .map(_.score) 64 | .stats 65 | .map(s => Stats(s.max, s.min, s.sum, s.count, s.mean, s.stdev)) 66 | } 67 | 68 | // ## Scio with Algebird `Aggregator` 69 | def scioAlgebird(input: SCollection[Rating]): SCollection[Stats] = 70 | input.aggregate(aggregator) 71 | 72 | // ## Spark 73 | def spark(input: RDD[Rating]): Stats = { 74 | val s = input.map(_.score).stats() 75 | Stats(s.max, s.min, s.sum, s.count, s.mean, s.stdev) 76 | } 77 | 78 | // ## Spark with Algebird `Aggregator` 79 | def sparkAlgebird(input: RDD[Rating]): Stats = { 80 | import com.twitter.algebird.spark._ 81 | input.algebird.aggregate(aggregator) 82 | } 83 | 84 | } 85 | -------------------------------------------------------------------------------- /src/main/scala/com/spotify/bdrc/pipeline/SumPerItem.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Spotify AB. 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 | // Example: Compute the Sum of Scores per Item 19 | // Input is a collection of (user, item, score) 20 | package com.spotify.bdrc.pipeline 21 | 22 | import com.spotify.bdrc.util.Records.Rating 23 | import com.spotify.scio.values.SCollection 24 | import com.twitter.scalding.TypedPipe 25 | import org.apache.spark.rdd.RDD 26 | 27 | object SumPerItem { 28 | 29 | // ## Scalding 30 | def scalding(input: TypedPipe[Rating]): TypedPipe[(String, Double)] = { 31 | input 32 | .groupBy(_.item) 33 | .mapValues(_.score) 34 | // Sum per key with an implicit `Semigroup[Double]` 35 | .sum 36 | .toTypedPipe 37 | } 38 | 39 | // ## Scalding with Algebird `Aggregator` 40 | def scaldingWithAlgebird(input: TypedPipe[Rating]): TypedPipe[(String, Double)] = { 41 | import com.twitter.algebird.Aggregator.prepareMonoid 42 | input 43 | .groupBy(_.item) 44 | // Aggregate per key with an aggregator that converts `UserItemData` to `Double` via 45 | // `_.score` before reduce 46 | .aggregate(prepareMonoid(_.score)) 47 | .toTypedPipe 48 | } 49 | 50 | // ## Scio 51 | def scio(input: SCollection[Rating]): SCollection[(String, Double)] = { 52 | input 53 | .map(x => (x.item, x.score)) 54 | .sumByKey 55 | } 56 | 57 | // ## Spark 58 | def spark(input: RDD[Rating]): RDD[(String, Double)] = { 59 | input 60 | .map(x => (x.item, x.score)) 61 | .reduceByKey(_ + _) 62 | } 63 | 64 | // ## Spark with Algebird `Semigroup` 65 | def sparkWithAlgebird1(input: RDD[Rating]): RDD[(String, Double)] = { 66 | import com.twitter.algebird.spark._ 67 | input 68 | .map(x => (x.item, x.score)) 69 | .algebird 70 | // Sum per key with an implicit `Semigroup[Double]` 71 | .sumByKey 72 | } 73 | 74 | // ## Spark with Algebird `Aggregator` 75 | def sparkWithAlgebird2(input: RDD[Rating]): RDD[(String, Double)] = { 76 | import com.twitter.algebird.Aggregator.prepareMonoid 77 | import com.twitter.algebird.spark._ 78 | input 79 | .keyBy(_.item) 80 | .algebird 81 | // Aggregate per key with an aggregator that converts `UserItemData` to `Double` via 82 | // `_.score` before reduce. Explicit type due to type inference limitation. 83 | .aggregateByKey(prepareMonoid { x: Rating => x.score }) 84 | } 85 | 86 | } 87 | -------------------------------------------------------------------------------- /src/main/scala/com/spotify/bdrc/pipeline/TfIdf.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Spotify AB. 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 com.spotify.bdrc.pipeline 19 | 20 | import com.spotify.scio.values.SCollection 21 | import com.twitter.scalding.TypedPipe 22 | import org.apache.spark.rdd.RDD 23 | 24 | /** 25 | * Compute TF-IDF for a set of documents. 26 | * 27 | * Input is a Seq of (doc, text). 28 | */ 29 | object TfIdf { 30 | 31 | case class Score(term: String, doc: String, score: Double) 32 | 33 | def scalding(input: Seq[(String, TypedPipe[String])]): TypedPipe[Score] = { 34 | val numDocs = input.size 35 | 36 | val docToTerms = input 37 | .map { case (doc, pipe) => 38 | pipe 39 | .flatMap(_.split("\\W+").filter(_.nonEmpty)) 40 | .map(t => (doc, t.toLowerCase)) 41 | } 42 | // union input collections 43 | .reduce(_ ++ _) // (d, t) 44 | 45 | val docToTermAndFreq = docToTerms 46 | .groupBy(identity) 47 | .size 48 | .toTypedPipe 49 | .map { case ((d, t), tf) => (d, (t, tf)) } 50 | 51 | val termToDfN = docToTerms.distinct.values 52 | .groupBy(identity) 53 | .size // (t, df) 54 | .mapValues(_.toDouble / numDocs) // (t, df/N) 55 | 56 | docToTerms.keys 57 | .groupBy(identity) 58 | .size // (d, |d|) 59 | .join(docToTermAndFreq) 60 | .toTypedPipe 61 | .map { case (d, (dLen, (t, tf))) => (t, (d, tf.toDouble / dLen)) } // (t, (d, tf/|d|)) 62 | .join(termToDfN) 63 | .toTypedPipe 64 | .map { case (t, ((d, tfd), dfN)) => Score(t, d, tfd * math.log(1 / dfN)) } 65 | } 66 | 67 | def scio(input: Seq[(String, SCollection[String])]): SCollection[Score] = { 68 | val numDocs = input.size 69 | 70 | val docToTerms = input 71 | .map { case (doc, pipe) => 72 | pipe 73 | .flatMap(_.split("\\W+").filter(_.nonEmpty)) 74 | .map(t => (doc, t.toLowerCase)) 75 | } 76 | // union input collections 77 | .reduce(_ ++ _) // (d, t) 78 | 79 | val docToTermAndCFreq = docToTerms 80 | // equivalent to .countByValue but returns RDD instead of Map 81 | .map((_, 1L)) 82 | .reduceByKey(_ + _) 83 | .map { case ((d, t), tf) => (d, (t, tf)) } 84 | 85 | val termToDfN = docToTerms.distinct.values 86 | // equivalent to .countByValue but returns RDD instead of Map 87 | .map((_, 1L)) 88 | .reduceByKey(_ + _) // (t, df) 89 | .mapValues(_.toDouble / numDocs) // (t, df/N) 90 | 91 | docToTerms.keys 92 | // equivalent to .countByValue but returns RDD instead of Map 93 | .map((_, 1L)) 94 | .reduceByKey(_ + _) // (d, |d|) 95 | .join(docToTermAndCFreq) 96 | .map { case (d, (dLen, (t, tf))) => (t, (d, tf.toDouble / dLen)) } // (t, (d, tf/|d|)) 97 | .join(termToDfN) 98 | .map { case (t, ((d, tfd), dfN)) => Score(t, d, tfd * math.log(1 / dfN)) } 99 | } 100 | 101 | /** Spark implementation using transformations to keep computation distributed. */ 102 | def sparkTransformations(input: Seq[(String, RDD[String])]): RDD[Score] = { 103 | val numDocs = input.size 104 | 105 | val docToTerms = input 106 | .map { case (doc, pipe) => 107 | pipe 108 | .flatMap(_.split("\\W+").filter(_.nonEmpty)) 109 | .map(t => (doc, t.toLowerCase)) 110 | } 111 | // union input collections 112 | .reduce(_ ++ _) // (d, t) 113 | .cache() // docToTerms is reused 3 times 114 | 115 | val docToTermAndCFreq = docToTerms 116 | // equivalent to .countByValue but returns RDD instead of Map 117 | .map((_, 1L)) 118 | .reduceByKey(_ + _) 119 | .map { case ((d, t), tf) => (d, (t, tf)) } 120 | 121 | val termToDfN = docToTerms 122 | .distinct() 123 | .values 124 | // equivalent to .countByValue but returns RDD instead of Map 125 | .map((_, 1L)) 126 | .reduceByKey(_ + _) // (t, df) 127 | .mapValues(_.toDouble / numDocs) // (t, df/N) 128 | 129 | docToTerms.keys 130 | // equivalent to .countByValue but returns RDD instead of Map 131 | .map((_, 1L)) 132 | .reduceByKey(_ + _) // (d, |d|) 133 | .join(docToTermAndCFreq) 134 | .map { case (d, (dLen, (t, tf))) => (t, (d, tf.toDouble / dLen)) } // (t, (d, tf/|d|)) 135 | .join(termToDfN) 136 | .map { case (t, ((d, tfd), dfN)) => Score(t, d, tfd * math.log(1 / dfN)) } 137 | } 138 | 139 | /** Spark implementation using actions to compute some steps on the driver node. */ 140 | def sparkActions(input: Seq[(String, RDD[String])]): Seq[Score] = { 141 | val numDocs = input.size 142 | 143 | val docToTerms = input 144 | .map { case (doc, pipe) => 145 | pipe 146 | .flatMap(_.split("\\W+").filter(_.nonEmpty)) 147 | .map(t => (doc, t.toLowerCase)) 148 | } 149 | .reduce(_ ++ _) // (d, t) 150 | .cache() // docToTerms is reused 3 times 151 | 152 | val docToTermAndCFreq = docToTerms 153 | .countByValue() 154 | // performed on driver node 155 | .map { case ((d, t), tf) => (d, (t, tf)) } 156 | 157 | val termToDfN = docToTerms 158 | .distinct() 159 | .values 160 | .countByValue() // (t, df) 161 | // performed on driver node 162 | .mapValues(_.toDouble / numDocs) // (t, df/N) 163 | 164 | docToTerms.keys 165 | .countByValue() // (d, |d|) 166 | // performed on driver node 167 | .toSeq 168 | .map { case (d, dLen) => 169 | val (t, tf) = docToTermAndCFreq(d) 170 | //(t, (d, tf.toDouble / dLen)) // (t, (d, tf/|d|)) 171 | val tfd = tf.toDouble / dLen 172 | val dfN = termToDfN(t) 173 | Score(t, d, tfd * math.log(1 / dfN)) 174 | } 175 | } 176 | 177 | } 178 | -------------------------------------------------------------------------------- /src/main/scala/com/spotify/bdrc/pipeline/TopItems.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Spotify AB. 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 | // Example: Compute Top K Items Globally 19 | // Input is a collection of (user, item, score) 20 | package com.spotify.bdrc.pipeline 21 | 22 | import com.spotify.bdrc.util.Records.Rating 23 | import com.spotify.scio.values.SCollection 24 | import com.twitter.scalding.TypedPipe 25 | import org.apache.spark.rdd.RDD 26 | 27 | object TopItems { 28 | 29 | val topK = 100 30 | 31 | // ## Scalding 32 | def scalding(input: TypedPipe[Rating]): TypedPipe[(String, Double)] = { 33 | input 34 | .map(x => (x.item, x.score)) 35 | .group 36 | // Sum values with an implicit `Semigroup[Double]` 37 | .sum 38 | // Group all elements with a single key `Unit` 39 | .groupAll 40 | // Take top K with a priority queue 41 | .sortedReverseTake(topK)(Ordering.by(_._2)) 42 | // Drop `Unit` key 43 | .values 44 | // Flatten result `Seq[(String, Double)]` 45 | .flatten 46 | } 47 | 48 | // ## Scalding with Algebird `Aggregator` 49 | def scaldingWithAlgebird(input: TypedPipe[Rating]): TypedPipe[(String, Double)] = { 50 | import com.twitter.algebird.Aggregator.sortedReverseTake 51 | val aggregator = sortedReverseTake[(String, Double)](topK)(Ordering.by(_._2)) 52 | input 53 | .map(x => (x.item, x.score)) 54 | .group 55 | // Sum values with an implicit `Semigroup[Double]` 56 | .sum 57 | .toTypedPipe 58 | // Aggregate globally into a single `Seq[(String, Double)]` 59 | .aggregate(aggregator) 60 | // Flatten result `Seq[(String, Double)]` 61 | .flatten 62 | } 63 | 64 | // ## Scio 65 | def scio(input: SCollection[Rating]): SCollection[(String, Double)] = { 66 | input 67 | .map(x => (x.item, x.score)) 68 | // Sum values with an implicit `Semigroup[Double]` 69 | .sumByKey 70 | // Compute top K as an `Iterable[(String, Double)]` 71 | .top(topK)(Ordering.by(_._2)) 72 | // Flatten result `Iterable[(String, Double)]` 73 | .flatten 74 | } 75 | 76 | // ## Scio with Algebird `Aggregator` 77 | def scioWithAlgebird(input: SCollection[Rating]): SCollection[(String, Double)] = { 78 | import com.twitter.algebird.Aggregator.sortedReverseTake 79 | val aggregator = sortedReverseTake[(String, Double)](topK)(Ordering.by(_._2)) 80 | input 81 | .map(x => (x.item, x.score)) 82 | // Sum values with an implicit `Semigroup[Double]` 83 | .sumByKey 84 | // Aggregate globally into a single `Seq[(String, Double)]` 85 | .aggregate(aggregator) 86 | // Flatten result `Seq[(String, Double)]` 87 | .flatten 88 | } 89 | 90 | // ## Spark 91 | def spark(input: RDD[Rating]): Seq[(String, Double)] = { 92 | input 93 | .map(x => (x.item, x.score)) 94 | // Sum values with addition 95 | .reduceByKey(_ + _) 96 | // `top` is an action and collects data back to the driver node 97 | .top(topK)(Ordering.by(_._2)) 98 | } 99 | 100 | // ## Spark with Algebird `Aggregator` 101 | def sparkWithAlgebird(input: RDD[Rating]): Seq[(String, Double)] = { 102 | import com.twitter.algebird.Aggregator.sortedReverseTake 103 | import com.twitter.algebird.spark._ 104 | val aggregator = sortedReverseTake[(String, Double)](topK)(Ordering.by(_._2)) 105 | input 106 | .map(x => (x.item, x.score)) 107 | // Sum values with addition 108 | .reduceByKey(_ + _) 109 | .algebird 110 | // `aggregate` is an action and collects data back to the driver node 111 | .aggregate(aggregator) 112 | } 113 | 114 | } 115 | -------------------------------------------------------------------------------- /src/main/scala/com/spotify/bdrc/pipeline/TopItemsPerUser.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Spotify AB. 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 | // Example: Compute Top K Items Globally 19 | // Input is a collection of (user, item, score) 20 | package com.spotify.bdrc.pipeline 21 | 22 | import com.spotify.bdrc.util.Records.Rating 23 | import com.spotify.scio.values.SCollection 24 | import com.twitter.scalding.TypedPipe 25 | import org.apache.spark.rdd.RDD 26 | 27 | object TopItemsPerUser { 28 | 29 | val topK = 100 30 | 31 | // ## Scalding 32 | def scalding(input: TypedPipe[Rating]): TypedPipe[Rating] = { 33 | input 34 | .groupBy(_.user) 35 | // Take top K per group with a priority queue 36 | .sortedReverseTake(topK)(Ordering.by(_.score)) 37 | // Drop user key 38 | .values 39 | // Flatten result `Seq[Rating]` 40 | .flatten 41 | } 42 | 43 | // ## Scio 44 | def scio(input: SCollection[Rating]): SCollection[Rating] = { 45 | input 46 | .keyBy(_.user) 47 | // Compute top K per key 48 | .topByKey(topK)(Ordering.by(_.score)) 49 | // Drop user key 50 | .values 51 | // Flatten result `Iterable[Rating]` 52 | .flatten 53 | } 54 | 55 | // ## Spark Naive Approach 56 | def spark(input: RDD[Rating]): RDD[Rating] = { 57 | input 58 | // `groupBy` shuffles all data, inefficient 59 | .groupBy(_.user) 60 | // Drop user key 61 | .values 62 | // Convert grouped values to a `List[Rating]` and sort on a single node, inefficient 63 | .flatMap(_.toList.sortBy(-_.score).take(topK)) 64 | } 65 | 66 | // ## Spark with Algebird `Aggregator` 67 | def sparkWithAlgebird(input: RDD[Rating]): RDD[Rating] = { 68 | import com.twitter.algebird.Aggregator.sortedReverseTake 69 | import com.twitter.algebird.spark._ 70 | val aggregator = sortedReverseTake[Rating](topK)(Ordering.by(_.score)) 71 | input 72 | .keyBy(_.user) 73 | .algebird 74 | // Aggregate per key into a `Seq[Rating]` 75 | .aggregateByKey(aggregator) 76 | // Flatten result `Seq[Rating]` 77 | .flatMap(_._2) 78 | } 79 | 80 | // ## Spark with MLLib 81 | def sparkWithMllib(input: RDD[Rating]): RDD[Rating] = { 82 | import org.apache.spark.mllib.rdd.MLPairRDDFunctions._ 83 | input 84 | .keyBy(_.user) 85 | // From `spark-mllib`, compute top K per key with a priority queue 86 | .topByKey(topK)(Ordering.by(_.score)) 87 | // Flatten result `Seq[Rating]` 88 | .flatMap(_._2) 89 | } 90 | 91 | } 92 | -------------------------------------------------------------------------------- /src/main/scala/com/spotify/bdrc/pipeline/TotalAndDistinctCount.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Spotify AB. 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 com.spotify.bdrc.pipeline 19 | 20 | import com.spotify.scio.values.SCollection 21 | import com.twitter.scalding.TypedPipe 22 | import org.apache.spark.rdd.RDD 23 | 24 | /** 25 | * Compute number of total and distinct items. 26 | * 27 | * Input is a collection of (user, item, score). 28 | */ 29 | object TotalAndDistinctCount { 30 | 31 | def aggregator = { 32 | import com.twitter.algebird._ 33 | // Exact total count, approximate unique count 34 | val totalCount = Aggregator.size 35 | val uniqueCount = Aggregator.approximateUniqueCount[String] 36 | MultiAggregator(totalCount, uniqueCount) 37 | } 38 | 39 | def scaldingExact(input: TypedPipe[String]): TypedPipe[(Long, Long)] = { 40 | input 41 | .map((_, 1L)) 42 | .group 43 | .sum // (key, total count per key) 44 | .toTypedPipe 45 | .map(kv => (kv._1, (kv._2, 1L))) 46 | .group 47 | .sum // (key, (total count, distinct count)) 48 | .values 49 | } 50 | 51 | def scaldingApproximate(input: TypedPipe[String]): TypedPipe[(Long, Long)] = 52 | input.aggregate(aggregator) 53 | 54 | def scioExact(input: SCollection[String]): SCollection[(Long, Long)] = { 55 | input 56 | .map((_, 1L)) 57 | .sumByKey // (key, total count per key) 58 | .map(kv => (kv._1, (kv._2, 1L))) 59 | .sumByKey // (key, (total count, distinct count)) 60 | .values 61 | } 62 | 63 | def scioApproximate(input: SCollection[String]): SCollection[(Long, Long)] = 64 | input.aggregate(aggregator) 65 | 66 | def sparkAlgebird(input: RDD[String]): RDD[(Long, Long)] = { 67 | import com.twitter.algebird.spark._ 68 | input 69 | .map((_, 1L)) 70 | .algebird 71 | .sumByKey[String, Long] // (key, total count per key) 72 | .map(kv => (kv._1, (kv._2, 1L))) 73 | .algebird 74 | .sumByKey[String, (Long, Long)] // (key, (total count, distinct count)) 75 | .values 76 | } 77 | 78 | def sparkInMemory(input: RDD[String]): (Long, Long) = { 79 | input.cache() 80 | (input.count(), input.distinct().count()) 81 | } 82 | 83 | def sparkApproximate(input: RDD[String]): (Long, Long) = { 84 | input.cache() 85 | (input.count(), input.countApproxDistinct()) 86 | } 87 | 88 | } 89 | -------------------------------------------------------------------------------- /src/main/scala/com/spotify/bdrc/pipeline/WordCount.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Spotify AB. 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 | // Example: Classic Word Count 19 | package com.spotify.bdrc.pipeline 20 | 21 | import com.spotify.scio.values.SCollection 22 | import com.twitter.scalding.TypedPipe 23 | import org.apache.spark.rdd.RDD 24 | 25 | object WordCount { 26 | 27 | // ## Scalding 28 | def scalding(input: TypedPipe[String]): TypedPipe[(String, Long)] = { 29 | input 30 | .flatMap(_.split("[^a-zA-Z']+").filter(_.nonEmpty)) 31 | // `groupBy` is lazy 32 | .groupBy(identity) 33 | // Operations like `size` after `groupBy` can be lifted into the map phase 34 | .size 35 | .toTypedPipe 36 | } 37 | 38 | // ## Scio 39 | def scio(input: SCollection[String]): SCollection[(String, Long)] = { 40 | input 41 | .flatMap(_.split("[^a-zA-Z']+").filter(_.nonEmpty)) 42 | .countByValue 43 | } 44 | 45 | // ## Spark Transformation 46 | def sparkTransformation(input: RDD[String]): RDD[(String, Long)] = { 47 | input 48 | .flatMap(_.split("[^a-zA-Z']+").filter(_.nonEmpty)) 49 | // There is no `countByValue` transformation in Spark although it is equivalent to mapping 50 | // into initial count of `1` and reduce with addition 51 | .map((_, 1L)) 52 | // `reduceByKey` can lift function into the map phase 53 | .reduceByKey(_ + _) 54 | } 55 | 56 | // ## Spark Action 57 | def sparkAction(input: RDD[String]): Seq[(String, Long)] = { 58 | input 59 | .flatMap(_.split("[^a-zA-Z']+").filter(_.nonEmpty)) 60 | // `countByValue` is an action and collects data back to the driver node 61 | .countByValue() 62 | .toSeq 63 | } 64 | 65 | } 66 | -------------------------------------------------------------------------------- /src/main/scala/com/spotify/bdrc/scala/Collections.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 Spotify AB. 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 com.spotify.bdrc.scala 19 | 20 | /** 21 | * Examples for working with Scala collections. 22 | */ 23 | object Collections { 24 | 25 | def mapValues: Unit = { 26 | val m = Map("a" -> 1, "b" -> 2, "c" -> 3) 27 | 28 | // Native approach, inefficient since .toList, .map, .toMap each creates a copy 29 | m.toList.map(t => (t._1, t._2 + 1)).toMap 30 | 31 | // Better, one copy 32 | m.map(kv => (kv._1, kv._2 + 1)) 33 | 34 | // Lazy version, no copy 35 | m.mapValues(_ + 1) 36 | } 37 | 38 | def mergeMaps: Unit = { 39 | val m1 = Map("a" -> 1.0, "b" -> 2.0, "c" -> 3.0) 40 | val m2 = Map("a" -> 1.5, "b" -> 2.5, "d" -> 3.5) 41 | 42 | // Native approach, inefficient since it creates many copies 43 | val i = m1.keySet intersect m2.keySet 44 | val m = i.map(k => k -> (m1(k) + m2(k))) // sum values of common keys 45 | (m1 -- i) ++ (m2 -- i) ++ m // inefficient, creates 2 more temporary maps 46 | m1 ++ m2 ++ m // slightly better, values from RHS overwrites those from LHS 47 | 48 | // Slightly better but still creates a temporary set 49 | (m1.keySet ++ m2.keySet).map(k => k -> (m1.getOrElse(k, 0.0) + m2.getOrElse(k, 0.0))) 50 | 51 | // Better but slightly cryptic 52 | m1 ++ m2.map { case (k, v) => k -> (v + m1.getOrElse(k, 0.0)) } 53 | } 54 | 55 | def listToMap: Unit = { 56 | val l = List(1, 2, 3, 4, 5) 57 | 58 | // Native approach, creates a temporary copy 59 | l.map(x => "key" + x -> x).toMap 60 | 61 | // Slightly better, using a mutable builder 62 | val b = Map.newBuilder[String, Int] 63 | l.foreach(x => b += "key" + x -> x) 64 | b.result() 65 | 66 | // Use implicits to automatically build for the target collection type Map[String, Int] 67 | val m: Map[String, Int] = l.map(x => "key" + x -> x)(scala.collection.breakOut) 68 | } 69 | 70 | } 71 | -------------------------------------------------------------------------------- /src/main/scala/com/spotify/bdrc/scala/FilterMessyData.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Spotify AB. 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 com.spotify.bdrc.scala 19 | 20 | import scala.util.Try 21 | 22 | /** 23 | * Filter out messy data that may cause computation to fail. 24 | * 25 | * Input is a collection of case classes with messy values. 26 | */ 27 | object FilterMessyData { 28 | 29 | case class MessyData(user: String, gender: String, scores: Array[Double], favorites: Set[String]) 30 | 31 | /** Dummy method that may fail for invalid records. */ 32 | def compute(x: MessyData): String = "dummy_result" 33 | 34 | /** Naive approach that checks every field accessed. */ 35 | def naive(input: Seq[MessyData]): Seq[String] = { 36 | input 37 | .filter { x => 38 | x.user != null && x.gender != null && 39 | x.scores != null && x.scores.nonEmpty && 40 | x.favorites != null && x.favorites.nonEmpty 41 | } 42 | .map(compute) // may still fail for unexpected cases 43 | } 44 | 45 | /** 46 | * Smart approach that throws any failed records away. 47 | * 48 | * Try.toOption returns Some if the computation succeeds or None if it fails. 49 | * Option[U] is implicitly converted to TraversableOnce[U] that flatMap expects. 50 | * 51 | * WARNING: THIS APPROACH IGNORES ANY EXCEPTION AND IS POTENTIALLY UNSAFE. 52 | */ 53 | def withUnsafeFlatMap(input: Seq[MessyData]): Seq[String] = 54 | input 55 | .flatMap(x => Try(compute(x)).toOption) 56 | 57 | /** 58 | * Smart approach that throws any failed records away. 59 | * 60 | * Try/catch block returns a Seq of one item if compute succeeds and Nil if it fails. 61 | * This approach is safer since you have control over what exceptions to expect. 62 | */ 63 | def withSafeFlatMap(input: Seq[MessyData]): Seq[String] = { 64 | input 65 | .flatMap { x => 66 | try { 67 | Seq(compute(x)) 68 | } catch { 69 | case _: NullPointerException => Nil 70 | } 71 | } 72 | } 73 | 74 | } 75 | -------------------------------------------------------------------------------- /src/main/scala/com/spotify/bdrc/scala/HandlingOptions.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Spotify AB. 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 com.spotify.bdrc.scala 19 | 20 | /** 21 | * Handling data with multiple Option[T]s more gracefully. 22 | * 23 | * Input is a collection of case classes with nested Option[T]. 24 | */ 25 | object HandlingOptions { 26 | 27 | case class Metadata(track: Option[Track], audio: Option[Audio]) 28 | case class Track(id: String, name: String, artist: Option[Artist]) 29 | case class Artist(id: String, name: String) 30 | case class Audio(tempo: Int, key: String) 31 | 32 | /** Naive approach that checks every field accessed is defined. */ 33 | def naive(input: Seq[Metadata]): Seq[(String, Int)] = { 34 | input 35 | .filter(m => m.track.isDefined && m.track.get.artist.isDefined && m.audio.isDefined) 36 | .map { m => 37 | // Option[T].get is safe since we already checked with Option[T].isDefined 38 | (m.track.get.artist.get.id, m.audio.get.tempo) 39 | } 40 | } 41 | 42 | /** 43 | * Smart approach that uses for comprehension. 44 | * 45 | * For-comprehension extracts values from Options and yields Some if all Options are defined. 46 | * It yields None if any of the Options is None. 47 | */ 48 | def withFlatMap(input: Seq[Metadata]): Seq[(String, Int)] = { 49 | input.flatMap { md => 50 | for { 51 | tr <- md.track // extract Track from Option[Track] 52 | ar <- tr.artist // extract Artist from Option[Artist] 53 | au <- md.audio // extract Audio from Option[Audio] 54 | } yield (ar.id, au.tempo) 55 | } 56 | } 57 | 58 | /** The for-comprehension above translates to nested flatMaps. */ 59 | def withNestedFlatMap(input: Seq[Metadata]): Seq[(String, Int)] = { 60 | input.flatMap { md => 61 | md.track.flatMap { tr => 62 | tr.artist.flatMap(ar => md.audio.map(au => (ar.id, au.tempo))) 63 | } 64 | } 65 | } 66 | 67 | } 68 | -------------------------------------------------------------------------------- /src/main/scala/com/spotify/bdrc/scala/JavaPrimitives.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Spotify AB. 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 com.spotify.bdrc.scala 19 | 20 | /** 21 | * Examples for working with Java primitives. 22 | */ 23 | object JavaPrimitives { 24 | 25 | // java.lang.Double is a boxed type (object) while double in Java is a primitive type. 26 | // scala.Double can be either boxed or primitive depending on the context, e.g. it's boxed when 27 | // used as a type parameter in a generic class but primitive when used in an array or on the 28 | // stack. 29 | // Due to type system limitations, M[java.lang.Double] and M[scala.Double] are incompatible types 30 | // but they can be casted safely back and forth since both are implemented as Java boxed types. 31 | import java.lang.{Double => JDouble} 32 | import java.util.{List => JList} 33 | 34 | import scala.collection.JavaConverters._ 35 | 36 | /** 37 | * `xs.asScala` returns `mutable.Buffer[JDouble]` where `Buffer` is a sub-type of `Seq` but 38 | * `JDouble` is not the same type as `Double` (`scala.Double`). Casting is safe because `JDouble` 39 | * and `Double` are equivalent when used as type parameters (boxed objects). It's also cheaper 40 | * than `.map(_.toDouble)` which creates a copy of the `Buffer`. 41 | */ 42 | def jDoubleListToSeq(xs: JList[JDouble]): Seq[Double] = xs.asScala.asInstanceOf[Seq[Double]] 43 | 44 | /** 45 | * Array[Double] is more efficient since it's implemented as a Java primitive array. Arrays are 46 | * also mutable so it'scheaper to pre-allocate and mutate elements. Java iterator and while loop 47 | * are faster than `xs.asScala.asInstanceOf[Seq[Double]].toArray`. 48 | */ 49 | def jDoubleListToArray(xs: JList[JDouble]): Array[Double] = { 50 | val a = new Array[Double](xs.size()) 51 | var i = 0 52 | val iterator = xs.iterator() 53 | while (iterator.hasNext) { 54 | a(i) = iterator.next() 55 | i += 1 56 | } 57 | a 58 | } 59 | 60 | } 61 | -------------------------------------------------------------------------------- /src/main/scala/com/spotify/bdrc/util/Records.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Spotify AB. 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 com.spotify.bdrc.util 19 | 20 | object Records { 21 | 22 | case class LogEvent(user: String, track: String, timestamp: Long) 23 | case class Rating(user: String, item: String, score: Double) 24 | case class UserMeta(user: String, age: Int) 25 | 26 | } 27 | -------------------------------------------------------------------------------- /src/test/scala/com/spotify/bdrc/bench/ForYieldBenchmark.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Spotify AB. 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 com.spotify.bdrc.bench 19 | 20 | import java.lang.{Iterable => JIterable} 21 | 22 | import com.google.common.collect.Lists 23 | import org.scalameter.api._ 24 | import org.scalameter.picklers.noPickler._ 25 | 26 | import scala.collection.JavaConverters._ 27 | 28 | /** Micro-benchmark for for/yield pattern. */ 29 | object ForYieldBenchmark extends Bench.LocalTime { 30 | 31 | val lSizes = Gen.enumeration("lSize")(1, 10, 100, 1000) 32 | val rSizes = Gen.enumeration("rSize")(1, 10, 100, 1000) 33 | 34 | def jIterable(i: Int): JIterable[String] = 35 | Lists.newArrayList((0 until i).map("v%05d".format(_)): _*).asInstanceOf[JIterable[String]] 36 | 37 | val inputs = for { 38 | l <- lSizes 39 | r <- rSizes 40 | } yield (jIterable(l), jIterable(r)) 41 | 42 | performance of "Join" in { 43 | measure method "forIterable" in { 44 | using(inputs) in { p => 45 | for { 46 | a <- p._1.asScala 47 | b <- p._2.asScala 48 | } yield ("key", (a, b)) 49 | } 50 | } 51 | 52 | // Iterator version is lazy and more efficient 53 | measure method "forIterator" in { 54 | using(inputs) in { p => 55 | val r = for { 56 | a <- p._1.asScala.iterator 57 | b <- p._2.asScala.iterator 58 | } yield ("key", (a, b)) 59 | r.toIterable 60 | } 61 | } 62 | } 63 | 64 | } 65 | -------------------------------------------------------------------------------- /src/test/scala/com/spotify/bdrc/testing/T01EndToEndTest.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Spotify AB. 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 com.spotify.bdrc.testing 19 | 20 | import com.spotify.scio._ 21 | import com.spotify.scio.io.TextIO 22 | import com.spotify.scio.testing.PipelineSpec 23 | 24 | object WordCount1 { 25 | def main(cmdlineArgs: Array[String]): Unit = { 26 | val (sc, args) = ContextAndArgs(cmdlineArgs) 27 | sc.textFile(args("input")) 28 | .flatMap(_.split("[^a-zA-Z']+").filter(_.nonEmpty)) 29 | .countByValue 30 | .map(kv => kv._1 + ": " + kv._2) 31 | .saveAsTextFile(args("output")) 32 | sc.run() 33 | } 34 | } 35 | 36 | /** 37 | * Test an entire pipeline end-to-end 38 | * 39 | * Pros: 40 | * - Complete test of the entire pipeline 41 | * - Covers argument parsing and I/O handling 42 | * - May also reveal serialization issues 43 | * 44 | * Cons: 45 | * - Hard to handcraft input and expected data 46 | * - Hard to cover edge cases for complex pipelines 47 | * - Can be slow in some frameworks 48 | * 49 | * Supported in: Scalding, Scio 50 | * 51 | * Recommendation: 52 | * This is a good approach to test small and simple pipelines since it offers the best code 53 | * coverage. It can also be used for pipelines with complex argument parsing and I/O handling, 54 | * e.g. ones with dynamic I/O based on arguments. 55 | * 56 | * Very complex pipelines with lots of steps may be broken down into smaller logical blocks and 57 | * tested separately using the transform test approach. 58 | */ 59 | class T01EndToEndTest extends PipelineSpec { 60 | 61 | val input = Seq("a b c d e", "a b a b") 62 | val expected = Seq("a: 3", "b: 3", "c: 1", "d: 1", "e: 1") 63 | 64 | "WordCount1" should "work" in { 65 | JobTest[com.spotify.bdrc.testing.WordCount1.type] 66 | .args("--input=in.txt", "--output=out.txt") 67 | .input(TextIO("in.txt"), input) 68 | .output(TextIO("out.txt"))(output => output should containInAnyOrder(expected)) 69 | .run() 70 | } 71 | 72 | } 73 | -------------------------------------------------------------------------------- /src/test/scala/com/spotify/bdrc/testing/T02TransformTest.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Spotify AB. 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 com.spotify.bdrc.testing 19 | 20 | import com.spotify.scio._ 21 | import com.spotify.scio.testing._ 22 | import com.spotify.scio.values.SCollection 23 | 24 | object WordCount2 { 25 | def main(cmdlineArgs: Array[String]): Unit = { 26 | val (sc, args) = ContextAndArgs(cmdlineArgs) 27 | val input = sc.textFile(args("input")) 28 | val wc = countWords(input) 29 | val output = formatOutput(wc) 30 | output.saveAsTextFile(args("output")) 31 | } 32 | 33 | def countWords(input: SCollection[String]): SCollection[(String, Long)] = 34 | input 35 | .flatMap(_.split("[^a-zA-Z']+").filter(_.nonEmpty)) 36 | .countByValue 37 | 38 | def formatOutput(input: SCollection[(String, Long)]): SCollection[String] = 39 | input 40 | .map(kv => kv._1 + ": " + kv._2) 41 | } 42 | 43 | /** 44 | * Test pipeline transforms 45 | * 46 | * Pros: 47 | * - Break down complex pipelines into smaller reusable pieces 48 | * - Easier to handcraft input and expected data than end-to-end test 49 | * 50 | * Cons: 51 | * - Does not cover argument parsing and IO handling 52 | * - May disrupt pipeline logic flow if overused 53 | * 54 | * Supported in: Scalding, Scio, Spark 55 | * 56 | * Recommendation: 57 | * Complex pipelines can be broken into logical blocks and tested using this approach. Individual 58 | * transforms should have clear roles in the pipeline, e.g. parsing input, formatting output, 59 | * aggregating data, training model, predicting labels, etc. It should also be easy to craft input 60 | * and expected data for these transforms and cover all code bases and edge cases. 61 | * 62 | * The level of granularity of each transform is also important. A transform should be small enough 63 | * for readability but big enough to avoid disruption to the main pipeline flow. Things to 64 | * consider are: number of inputs and outputs, group or join operations, etc. 65 | */ 66 | class TransformTest extends PipelineSpec { 67 | 68 | val input = Seq("a b c d e", "a b a b") 69 | val expected = Seq("a: 3", "b: 3", "c: 1", "d: 1", "e: 1") 70 | val intermediate = Seq(("a", 3L), ("b", 3L), ("c", 1L), ("d", 1L), ("e", 1L)) 71 | 72 | "countWords" should "work" in { 73 | runWithContext { sc => 74 | val in = sc.parallelize(input) 75 | WordCount2.countWords(in) should containInAnyOrder(intermediate) 76 | } 77 | } 78 | 79 | "formatOutput" should "work" in { 80 | runWithContext { sc => 81 | val in = sc.parallelize(intermediate) 82 | WordCount2.formatOutput(in) should containInAnyOrder(expected) 83 | } 84 | } 85 | 86 | } 87 | -------------------------------------------------------------------------------- /src/test/scala/com/spotify/bdrc/testing/T03FunctionTest.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Spotify AB. 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 com.spotify.bdrc.testing 19 | 20 | import com.spotify.scio._ 21 | import org.scalatest.flatspec.AnyFlatSpec 22 | import org.scalatest.matchers.should.Matchers 23 | 24 | object WordCount3 { 25 | def main(cmdlineArgs: Array[String]): Unit = { 26 | val (sc, args) = ContextAndArgs(cmdlineArgs) 27 | sc.textFile(args("input")) 28 | .flatMap(split) 29 | .countByValue 30 | .map(format) 31 | .saveAsTextFile(args("output")) 32 | } 33 | 34 | def split(input: String): Seq[String] = input.split("[^a-zA-Z']+").filter(_.nonEmpty) 35 | def format(kv: (String, Long)): String = kv._1 + ": " + kv._2 36 | } 37 | 38 | /** 39 | * Test individual functions used in a pipeline 40 | * 41 | * Pros: 42 | * - Fastest to test 43 | * - Easy to cover edge cases 44 | * 45 | * Cons: 46 | * - Limited scope of coverage 47 | * - May disrupt pipeline logic flow if overused 48 | * 49 | * Supported in: any framework 50 | * 51 | * Recommendation: 52 | * This is recommended for commonly reused functions or those with complex business logic, e.g. 53 | * numerical computation, log clean up and filtering, value group operations after groupByKey. 54 | * 55 | * The level of granularity of each function is also important. Typical candidates are multi-line 56 | * functions that are used more than once. Functions with complex logic and hard to test at a 57 | * higher level (transform or end-to-end), e.g. user session analysis after grouping by user key, 58 | * can also be tested with this approach. 59 | */ 60 | class FunctionTest extends AnyFlatSpec with Matchers { 61 | 62 | "split" should "work" in { 63 | WordCount3.split("a b,c d\te\n\nf") should equal(Seq("a", "b", "c", "d", "e", "f")) 64 | } 65 | 66 | "format" should "work" in { 67 | WordCount3.format(("a", 10L)) should equal("a: 10") 68 | } 69 | 70 | } 71 | -------------------------------------------------------------------------------- /src/test/scala/com/spotify/bdrc/testing/T04PropertyBasedTest.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Spotify AB. 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 com.spotify.bdrc.testing 19 | 20 | import com.google.common.collect.MinMaxPriorityQueue 21 | import org.scalacheck.Prop._ 22 | import org.scalacheck.{Gen, Properties} 23 | import org.scalatest.propspec.AnyPropSpec 24 | import org.scalatest.matchers.should.Matchers 25 | import org.scalatestplus.scalacheck.ScalaCheckDrivenPropertyChecks 26 | 27 | import scala.collection.JavaConverters._ 28 | 29 | object Utils { 30 | 31 | def top[T: Ordering](xs: Seq[T], num: Int): Seq[T] = { 32 | if (xs.isEmpty) { 33 | Seq.empty[T] 34 | } else { 35 | val size = math.min(num, xs.size) 36 | val ord = implicitly[Ordering[T]] 37 | MinMaxPriorityQueue 38 | .orderedBy(ord.reverse) 39 | .expectedSize(size) 40 | .maximumSize(size) 41 | .create[T](xs.asJava) 42 | .asScala 43 | .toSeq 44 | .sorted(ord.reverse) 45 | } 46 | } 47 | 48 | def split(input: String): Seq[String] = 49 | input 50 | .split("[^a-zA-Z']+") 51 | .filter(_.nonEmpty) 52 | .map(_.toLowerCase) 53 | 54 | def cosineSim(v1: Seq[Double], v2: Seq[Double]): Double = { 55 | require(v1.length == v2.length) 56 | var s1 = 0.0 57 | var s2 = 0.0 58 | var dp = 0.0 59 | var i = 0 60 | while (i < v1.length) { 61 | s1 += v1(i) * v1(i) 62 | s2 += v2(i) * v2(i) 63 | dp += v1(i) * v2(i) 64 | i += 1 65 | } 66 | dp / math.sqrt(s1 * s2) 67 | } 68 | 69 | } 70 | 71 | /** 72 | * Property-based testing using ScalaCheck 73 | * 74 | * http://scalacheck.org/ 75 | * 76 | * Pros: 77 | * - No need to handcraft input data 78 | * - May reveal rare edge cases, e.g. null input, extreme values, empty lists 79 | * 80 | * Cons: 81 | * - Hard to test business logic 82 | * - Some properties may be hard to verify 83 | * - Can be slow for expensive computations 84 | * 85 | * Supported in: any framework 86 | * 87 | * Recommendation: 88 | * This is useful for functions simple input and output types, especially those of heavy 89 | * mathematically computation, e.g. linear algebra, hash functions, set operations. 90 | * 91 | * However, since input data are randomly generated based on type signature, it might produce edge 92 | * cases irrelevant to the business logic, e.g. Double.MinValue, strings with Unicode characters. 93 | * You might also have to construct your own generator if certain distribution of input data is 94 | * expected, e.g. positive integers, strings from a finite set. 95 | * 96 | * See AlgebirdSpec.scala for more examples of testing Algebird features using ScalaCheck 97 | * https://github.com/spotify/scio/blob/master/scio-examples/src/test/scala/com/spotify/scio/examples/extra/AlgebirdSpec.scala 98 | */ 99 | class PropertyBasedTest extends AnyPropSpec with ScalaCheckDrivenPropertyChecks with Matchers { 100 | 101 | property("top") { 102 | forAll { xs: Seq[Long] => Utils.top(xs, 5) shouldBe xs.sorted.reverse.take(5) } 103 | } 104 | 105 | property("split") { 106 | forAll { line: String => Utils.split(line).forall(_.matches("[a-z']+")) } 107 | } 108 | 109 | // Generator for List[Double] of 100 doubles between -100.0 and 100.0 110 | val genVector = Gen.listOfN(100, Gen.choose(-100.0, 100.0)) 111 | 112 | property("cosineSim") { 113 | forAll(genVector, genVector) { (v1, v2) => 114 | val s1 = Utils.cosineSim(v1, v2) 115 | val s2 = Utils.cosineSim(v2, v1) 116 | 117 | s1 should (be >= -1.0 and be <= 1.0) 118 | s1 shouldBe s2 119 | Utils.cosineSim(v1, v1) shouldBe 1.0 120 | Utils.cosineSim(v1, v1.map(-_)) shouldBe -1.0 121 | } 122 | } 123 | 124 | } 125 | -------------------------------------------------------------------------------- /src/test/scala/com/spotify/bdrc/testing/T05MixedTest.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 Spotify AB. 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 com.spotify.bdrc.testing 19 | 20 | import com.spotify.scio._ 21 | import com.spotify.scio.io.TextIO 22 | import com.spotify.scio.testing.PipelineSpec 23 | import com.spotify.scio.values.SCollection 24 | 25 | object WordCount4 { 26 | def main(cmdlineArgs: Array[String]): Unit = { 27 | val (sc, args) = ContextAndArgs(cmdlineArgs) 28 | val input = sc.textFile(args("input")) 29 | val wc = countWords(input) 30 | val output = formatOutput(wc) 31 | output.saveAsTextFile(args("output")) 32 | sc.run() 33 | } 34 | 35 | // transforms 36 | def countWords(input: SCollection[String]): SCollection[(String, Long)] = 37 | input.flatMap(split).countByValue 38 | def formatOutput(input: SCollection[(String, Long)]): SCollection[String] = 39 | input.map(format) 40 | 41 | // functions 42 | def split(input: String): Seq[String] = input.split("[^a-zA-Z']+").filter(_.nonEmpty) 43 | def format(kv: (String, Long)): String = kv._1 + ": " + kv._2 44 | } 45 | 46 | /** 47 | * Mixed function, transform and end-to-end tests 48 | * 49 | * Property-based tests require an object that extends Properties and therefore are not included. 50 | */ 51 | class MixedTest extends PipelineSpec { 52 | 53 | val input = Seq("a b c d e", "a b a b") 54 | val expected = Seq("a: 3", "b: 3", "c: 1", "d: 1", "e: 1") 55 | val intermediate = Seq(("a", 3L), ("b", 3L), ("c", 1L), ("d", 1L), ("e", 1L)) 56 | 57 | // Function tests 58 | 59 | "split" should "work" in { 60 | WordCount3.split("a b,c d\te\n\nf") should equal(Seq("a", "b", "c", "d", "e", "f")) 61 | } 62 | 63 | "format" should "work" in { 64 | WordCount3.format(("a", 10L)) should equal("a: 10") 65 | } 66 | 67 | // Transform tests 68 | 69 | "countWords" should "work" in { 70 | runWithContext { sc => 71 | val in = sc.parallelize(input) 72 | WordCount4.countWords(in) should containInAnyOrder(intermediate) 73 | } 74 | } 75 | 76 | "formatOutput" should "work" in { 77 | runWithContext { sc => 78 | val in = sc.parallelize(intermediate) 79 | WordCount4.formatOutput(in) should containInAnyOrder(expected) 80 | } 81 | } 82 | 83 | // End-to-end test 84 | 85 | "WordCount1" should "work" in { 86 | JobTest[com.spotify.bdrc.testing.WordCount4.type] 87 | .args("--input=in.txt", "--output=out.txt") 88 | .input(TextIO("in.txt"), input) 89 | .output(TextIO("out.txt"))(output => output should containInAnyOrder(expected)) 90 | .run() 91 | } 92 | 93 | } 94 | --------------------------------------------------------------------------------