├── .gitignore ├── .travis.yml ├── LICENSE ├── README.md ├── build.gradle ├── gradle.properties ├── gradle └── wrapper │ ├── gradle-wrapper.jar │ └── gradle-wrapper.properties ├── gradlew ├── gradlew.bat ├── settings.gradle └── src ├── main └── scala │ └── org │ └── chojin │ └── spark │ └── lineage │ ├── Config.scala │ ├── QueryParser.scala │ ├── ReportProcessor.scala │ ├── SparkSqlLineageListener.scala │ ├── inputs │ └── package.scala │ ├── outputs │ └── package.scala │ ├── report │ ├── Metadata.scala │ └── Report.scala │ └── reporter │ ├── DynamodbReporter.scala │ ├── InMemoryReporter.scala │ ├── KinesisReporter.scala │ └── Reporter.scala └── test ├── resources ├── lineage.properties └── log4j.properties └── scala └── org └── chojin └── spark └── lineage ├── ConfigSpec.scala ├── SparkSqlLineageListenerSpec.scala └── reporter └── DynamodbReporterSpec.scala /.gitignore: -------------------------------------------------------------------------------- 1 | # Ignore Gradle project-specific cache directory 2 | # 3 | .gradle 4 | 5 | # Ignore Gradle build output directory 6 | build 7 | metastore_db 8 | out 9 | *.log 10 | .idea 11 | *.py 12 | -------------------------------------------------------------------------------- /.travis.yml: -------------------------------------------------------------------------------- 1 | language: scala 2 | scala: 3 | - 2.11.12 4 | jdk: 5 | - openjdk8 6 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | Apache License 2 | Version 2.0, January 2004 3 | http://www.apache.org/licenses/ 4 | 5 | TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION 6 | 7 | 1. Definitions. 8 | 9 | "License" shall mean the terms and conditions for use, reproduction, 10 | and distribution as defined by Sections 1 through 9 of this document. 11 | 12 | "Licensor" shall mean the copyright owner or entity authorized by 13 | the copyright owner that is granting the License. 14 | 15 | "Legal Entity" shall mean the union of the acting entity and all 16 | other entities that control, are controlled by, or are under common 17 | control with that entity. For the purposes of this definition, 18 | "control" means (i) the power, direct or indirect, to cause the 19 | direction or management of such entity, whether by contract or 20 | otherwise, or (ii) ownership of fifty percent (50%) or more of the 21 | outstanding shares, or (iii) beneficial ownership of such entity. 22 | 23 | "You" (or "Your") shall mean an individual or Legal Entity 24 | exercising permissions granted by this License. 25 | 26 | "Source" form shall mean the preferred form for making modifications, 27 | including but not limited to software source code, documentation 28 | source, and configuration files. 29 | 30 | "Object" form shall mean any form resulting from mechanical 31 | transformation or translation of a Source form, including but 32 | not limited to compiled object code, generated documentation, 33 | and conversions to other media types. 34 | 35 | "Work" shall mean the work of authorship, whether in Source or 36 | Object form, made available under the License, as indicated by a 37 | copyright notice that is included in or attached to the work 38 | (an example is provided in the Appendix below). 39 | 40 | "Derivative Works" shall mean any work, whether in Source or Object 41 | form, that is based on (or derived from) the Work and for which the 42 | editorial revisions, annotations, elaborations, or other modifications 43 | represent, as a whole, an original work of authorship. For the purposes 44 | of this License, Derivative Works shall not include works that remain 45 | separable from, or merely link (or bind by name) to the interfaces of, 46 | the Work and Derivative Works thereof. 47 | 48 | "Contribution" shall mean any work of authorship, including 49 | the original version of the Work and any modifications or additions 50 | to that Work or Derivative Works thereof, that is intentionally 51 | submitted to Licensor for inclusion in the Work by the copyright owner 52 | or by an individual or Legal Entity authorized to submit on behalf of 53 | the copyright owner. For the purposes of this definition, "submitted" 54 | means any form of electronic, verbal, or written communication sent 55 | to the Licensor or its representatives, including but not limited to 56 | communication on electronic mailing lists, source code control systems, 57 | and issue tracking systems that are managed by, or on behalf of, the 58 | Licensor for the purpose of discussing and improving the Work, but 59 | excluding communication that is conspicuously marked or otherwise 60 | designated in writing by the copyright owner as "Not a Contribution." 61 | 62 | "Contributor" shall mean Licensor and any individual or Legal Entity 63 | on behalf of whom a Contribution has been received by Licensor and 64 | subsequently incorporated within the Work. 65 | 66 | 2. Grant of Copyright License. Subject to the terms and conditions of 67 | this License, each Contributor hereby grants to You a perpetual, 68 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 69 | copyright license to reproduce, prepare Derivative Works of, 70 | publicly display, publicly perform, sublicense, and distribute the 71 | Work and such Derivative Works in Source or Object form. 72 | 73 | 3. Grant of Patent License. Subject to the terms and conditions of 74 | this License, each Contributor hereby grants to You a perpetual, 75 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 76 | (except as stated in this section) patent license to make, have made, 77 | use, offer to sell, sell, import, and otherwise transfer the Work, 78 | where such license applies only to those patent claims licensable 79 | by such Contributor that are necessarily infringed by their 80 | Contribution(s) alone or by combination of their Contribution(s) 81 | with the Work to which such Contribution(s) was submitted. If You 82 | institute patent litigation against any entity (including a 83 | cross-claim or counterclaim in a lawsuit) alleging that the Work 84 | or a Contribution incorporated within the Work constitutes direct 85 | or contributory patent infringement, then any patent licenses 86 | granted to You under this License for that Work shall terminate 87 | as of the date such litigation is filed. 88 | 89 | 4. Redistribution. You may reproduce and distribute copies of the 90 | Work or Derivative Works thereof in any medium, with or without 91 | modifications, and in Source or Object form, provided that You 92 | meet the following conditions: 93 | 94 | (a) You must give any other recipients of the Work or 95 | Derivative Works a copy of this License; and 96 | 97 | (b) You must cause any modified files to carry prominent notices 98 | stating that You changed the files; and 99 | 100 | (c) You must retain, in the Source form of any Derivative Works 101 | that You distribute, all copyright, patent, trademark, and 102 | attribution notices from the Source form of the Work, 103 | excluding those notices that do not pertain to any part of 104 | the Derivative Works; and 105 | 106 | (d) If the Work includes a "NOTICE" text file as part of its 107 | distribution, then any Derivative Works that You distribute must 108 | include a readable copy of the attribution notices contained 109 | within such NOTICE file, excluding those notices that do not 110 | pertain to any part of the Derivative Works, in at least one 111 | of the following places: within a NOTICE text file distributed 112 | as part of the Derivative Works; within the Source form or 113 | documentation, if provided along with the Derivative Works; or, 114 | within a display generated by the Derivative Works, if and 115 | wherever such third-party notices normally appear. The contents 116 | of the NOTICE file are for informational purposes only and 117 | do not modify the License. You may add Your own attribution 118 | notices within Derivative Works that You distribute, alongside 119 | or as an addendum to the NOTICE text from the Work, provided 120 | that such additional attribution notices cannot be construed 121 | as modifying the License. 122 | 123 | You may add Your own copyright statement to Your modifications and 124 | may provide additional or different license terms and conditions 125 | for use, reproduction, or distribution of Your modifications, or 126 | for any such Derivative Works as a whole, provided Your use, 127 | reproduction, and distribution of the Work otherwise complies with 128 | the conditions stated in this License. 129 | 130 | 5. Submission of Contributions. Unless You explicitly state otherwise, 131 | any Contribution intentionally submitted for inclusion in the Work 132 | by You to the Licensor shall be under the terms and conditions of 133 | this License, without any additional terms or conditions. 134 | Notwithstanding the above, nothing herein shall supersede or modify 135 | the terms of any separate license agreement you may have executed 136 | with Licensor regarding such Contributions. 137 | 138 | 6. Trademarks. This License does not grant permission to use the trade 139 | names, trademarks, service marks, or product names of the Licensor, 140 | except as required for reasonable and customary use in describing the 141 | origin of the Work and reproducing the content of the NOTICE file. 142 | 143 | 7. Disclaimer of Warranty. Unless required by applicable law or 144 | agreed to in writing, Licensor provides the Work (and each 145 | Contributor provides its Contributions) on an "AS IS" BASIS, 146 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 147 | implied, including, without limitation, any warranties or conditions 148 | of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A 149 | PARTICULAR PURPOSE. You are solely responsible for determining the 150 | appropriateness of using or redistributing the Work and assume any 151 | risks associated with Your exercise of permissions under this License. 152 | 153 | 8. Limitation of Liability. In no event and under no legal theory, 154 | whether in tort (including negligence), contract, or otherwise, 155 | unless required by applicable law (such as deliberate and grossly 156 | negligent acts) or agreed to in writing, shall any Contributor be 157 | liable to You for damages, including any direct, indirect, special, 158 | incidental, or consequential damages of any character arising as a 159 | result of this License or out of the use or inability to use the 160 | Work (including but not limited to damages for loss of goodwill, 161 | work stoppage, computer failure or malfunction, or any and all 162 | other commercial damages or losses), even if such Contributor 163 | has been advised of the possibility of such damages. 164 | 165 | 9. Accepting Warranty or Additional Liability. While redistributing 166 | the Work or Derivative Works thereof, You may choose to offer, 167 | and charge a fee for, acceptance of support, warranty, indemnity, 168 | or other liability obligations and/or rights consistent with this 169 | License. However, in accepting such obligations, You may act only 170 | on Your own behalf and on Your sole responsibility, not on behalf 171 | of any other Contributor, and only if You agree to indemnify, 172 | defend, and hold each Contributor harmless for any liability 173 | incurred by, or claims asserted against, such Contributor by reason 174 | of your accepting any such warranty or additional liability. 175 | 176 | END OF TERMS AND CONDITIONS 177 | 178 | APPENDIX: How to apply the Apache License to your work. 179 | 180 | To apply the Apache License to your work, attach the following 181 | boilerplate notice, with the fields enclosed by brackets "[]" 182 | replaced with your own identifying information. (Don't include 183 | the brackets!) The text should be enclosed in the appropriate 184 | comment syntax for the file format. We also recommend that a 185 | file or class name and description of purpose be included on the 186 | same "printed page" as the copyright notice for easier 187 | identification within third-party archives. 188 | 189 | Copyright [yyyy] [name of copyright owner] 190 | 191 | Licensed under the Apache License, Version 2.0 (the "License"); 192 | you may not use this file except in compliance with the License. 193 | You may obtain a copy of the License at 194 | 195 | http://www.apache.org/licenses/LICENSE-2.0 196 | 197 | Unless required by applicable law or agreed to in writing, software 198 | distributed under the License is distributed on an "AS IS" BASIS, 199 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 200 | See the License for the specific language governing permissions and 201 | limitations under the License. 202 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | spark-lineage 2 | ============= 3 | 4 | - [Installation](#installation) 5 | - [Configuration](#configuration) 6 | - [JSON Format](#json-format) 7 | - [Reporters](#reporters) 8 | * [Kinesis](#kinesis) 9 | * [DynamoDB](#dynamodb) 10 | - [Contributing](#contributing) 11 | 12 | [![Build Status](https://travis-ci.com/thesquelched/spark-lineage.svg?branch=master)](https://travis-ci.com/thesquelched/spark-lineage) 13 | 14 | Spark SQL listener to report lineage data to a variety of outputs, e.g. Amazon Kinesis. Heavily inspired by the 15 | [Spark Atlas Connector](https://github.com/hortonworks-spark/spark-atlas-connector), but intended to be more generic to 16 | help those who can't or won't use Atlas. 17 | 18 | For a Spark SQL query that produces an output (e.g. writes data to a filesystem), the listener produces a message 19 | containing the following: 20 | 21 | * Output details, e.g. type, output location, and format 22 | * For each (top-level) output field, a list of inputs that contribute to it. Each input contains: 23 | * Type, e.g. `hive` 24 | * List of input fields that affect the output field, and how they affect it (e.g. via a join, filter, aggregation, 25 | projection, etc.) 26 | * Metadata, e.g. spark application name 27 | 28 | Installation 29 | ------------ 30 | 31 | Build the jar: 32 | 33 | ```bash 34 | $ ./gradlew shadowJar 35 | ``` 36 | 37 | Copy the output from `build/lib/jars` to your cluster. Note that you will need to have the appropriate AWS SDK jars in 38 | your spark classpath if you mean to use the associated reporters. 39 | 40 | 41 | Configuration 42 | ------------- 43 | 44 | The listener is configured via a file, `lineage.properties`, that needs to be in the class path of the spark driver, 45 | e.g. by placing it in the spark config directory. Here's an example using the `KinesisReporter`: 46 | 47 | ``` 48 | org.chojin.spark.lineage.reporters=org.chojin.spark.lineage.reporters.KinesisReporter 49 | org.chojin.spark.lineage.reporter.kinesis.stream=my-kinesis-stream 50 | org.chojin.spark.lineage.reporter.kinesis.region=us-east-1 51 | org.chojin.spark.lineage.reporter.kinesis.shard=0 52 | org.chojin.spark.lineage.reporter.kinesis.compression=gzip 53 | ``` 54 | 55 | Then, configure `spark.sql.queryExecutionListeners` to include the `SparkSqlLineageListener`: 56 | 57 | ```bash 58 | $ spark-shell \ 59 | --jars file:///path/to/spark-lineage-0.0.3-SNAPSHOT.jar \ 60 | --conf spark.sql.queryExecutionListeners=org.chojin.spark.lineage.SparkSqlLineageListener 61 | ``` 62 | 63 | JSON Format 64 | ----------- 65 | 66 | Consider the following spark example: 67 | 68 | ```scala 69 | val foo = spark.table("mydb.foo") 70 | val bar = spark.table("mydb.bar") 71 | 72 | foo 73 | .join(bar, foo.col("bar_fk") === bar.col("pk")) 74 | .groupBy('my_flag) 75 | .agg(sum('amount) as 'amount_sum) 76 | .write 77 | .mode("overwrite") 78 | .parquet("s3:///bucket/path/to/report") 79 | ``` 80 | 81 | Once evaluated, the following JSON record is produced: 82 | 83 | ```json 84 | { 85 | "output": { 86 | "path": "s3://bucket/path/to/report", 87 | "type": "fs", 88 | "format": "Parquet" 89 | }, 90 | "metadata": { 91 | "appName": "Spark shell" 92 | }, 93 | "outputKey": "fs-s3://bucket/path/to/report", 94 | "fields": { 95 | "my_flag": [ 96 | { 97 | "type": "hive", 98 | "name": "mydb.foo", 99 | "fields": [ 100 | { 101 | "name": "bar_fk", 102 | "how": "join" 103 | } 104 | ] 105 | }, 106 | { 107 | "type": "hive", 108 | "name": "mydb.bar", 109 | "fields": [ 110 | { 111 | "name": "pk", 112 | "how": "join" 113 | }, 114 | { 115 | "name": "groupby", 116 | "how": "my_flag" 117 | } 118 | ] 119 | } 120 | ], 121 | "amount_sum": [ 122 | { 123 | "type": "hive", 124 | "name": "mydb.foo", 125 | "fields": [ 126 | { 127 | "name": "bar_fk", 128 | "how": "join" 129 | }, 130 | { 131 | "name": "amount", 132 | "how": "aggregate" 133 | } 134 | ] 135 | }, 136 | { 137 | "type": "hive", 138 | "name": "mydb.bar", 139 | "fields": [ 140 | { 141 | "name": "pk", 142 | "how": "join" 143 | }, 144 | { 145 | "name": "groupby", 146 | "how": "my_flag" 147 | } 148 | ] 149 | } 150 | ] 151 | } 152 | } 153 | ``` 154 | 155 | Reporters 156 | --------- 157 | 158 | One or more reporter classes can be specified as a comma-separated list in the parameter 159 | `org.chojin.spark.lineage.reporters`. 160 | 161 | ### Kinesis 162 | 163 | Class name: `org.chojin.spark.lineage.reporters.KinesisReporter` 164 | 165 | Write an optionally compressed JSON blob to an AWS Kinesis stream. 166 | 167 | #### Options 168 | 169 | Options start with the prefix `org.chojin.spark.lineage.reporter.kinesis.` 170 | 171 | * `stream` - stream name 172 | * `region` - region (e.g. `us-east-2`) 173 | * `shard` - shard/partition to which to write; multiple shards are not supported at this time 174 | * `compression` - optional compression to apply to the JSON blob; any standard spark compression codec is supported, 175 | e.g. `gzip`, `deflate` 176 | 177 | ### DynamoDB 178 | 179 | Class name: `org.chojin.spark.lineage.reporters.DynamodbReporter` 180 | 181 | Write a structured record to an AWS DynamoDB table. 182 | 183 | #### Options 184 | 185 | Options start with the prefix `org.chojin.spark.lineage.reporter.dynamodb.` 186 | 187 | * `table` - table name 188 | * `region` - region (e.g. `us-east-2`) 189 | * `json` - if `true`, write a single attribute, `json`, containing the lineage info as a binary blob 190 | * `compression` - optional compression to apply to the JSON blob (only if `json=true`; any standard spark compression 191 | codec is supported, e.g. `gzip`, `deflate` 192 | 193 | With `json=false`, and using the spark example above, the following DynamoDB record is produced: 194 | 195 | ```json 196 | { 197 | "Item": { 198 | "output": { 199 | "M": { 200 | "path": { 201 | "S": "s3://bucket/path/to/report" 202 | }, 203 | "type": { 204 | "S": "fs" 205 | }, 206 | "format": { 207 | "S": "Parquet" 208 | } 209 | } 210 | }, 211 | "metadata": { 212 | "M": { 213 | "appName": { 214 | "S": "Spark shell" 215 | } 216 | } 217 | }, 218 | "outputKey": { 219 | "S": "fs-s3://bucket/path/to/report" 220 | }, 221 | "fields": { 222 | "M": { 223 | "my_flag": { 224 | "L": [ 225 | { 226 | "M": { 227 | "fields": { 228 | "L": [ 229 | { 230 | "M": { 231 | "how": { 232 | "S": "join" 233 | }, 234 | "name": { 235 | "S": "bar_fk" 236 | } 237 | } 238 | } 239 | ] 240 | }, 241 | "type": { 242 | "S": "hive" 243 | }, 244 | "name": { 245 | "S": "mydb.foo" 246 | } 247 | } 248 | }, 249 | { 250 | "M": { 251 | "fields": { 252 | "L": [ 253 | { 254 | "M": { 255 | "how": { 256 | "S": "join" 257 | }, 258 | "name": { 259 | "S": "pk" 260 | } 261 | } 262 | }, 263 | { 264 | "M": { 265 | "how": { 266 | "S": "groupby" 267 | }, 268 | "name": { 269 | "S": "my_flag" 270 | } 271 | } 272 | } 273 | ] 274 | }, 275 | "type": { 276 | "S": "hive" 277 | }, 278 | "name": { 279 | "S": "mydb.bar" 280 | } 281 | } 282 | } 283 | ] 284 | }, 285 | "amount_sum": { 286 | "L": [ 287 | { 288 | "M": { 289 | "fields": { 290 | "L": [ 291 | { 292 | "M": { 293 | "how": { 294 | "S": "join" 295 | }, 296 | "name": { 297 | "S": "bar_fk" 298 | } 299 | } 300 | }, 301 | { 302 | "M": { 303 | "how": { 304 | "S": "aggregate" 305 | }, 306 | "name": { 307 | "S": "amount" 308 | } 309 | } 310 | } 311 | ] 312 | }, 313 | "type": { 314 | "S": "hive" 315 | }, 316 | "name": { 317 | "S": "mydb.foo" 318 | } 319 | } 320 | }, 321 | { 322 | "M": { 323 | "fields": { 324 | "L": [ 325 | { 326 | "M": { 327 | "how": { 328 | "S": "join" 329 | }, 330 | "name": { 331 | "S": "pk" 332 | } 333 | } 334 | }, 335 | { 336 | "M": { 337 | "how": { 338 | "S": "groupby" 339 | }, 340 | "name": { 341 | "S": "my_flag" 342 | } 343 | } 344 | } 345 | ] 346 | }, 347 | "type": { 348 | "S": "hive" 349 | }, 350 | "name": { 351 | "S": "mydb.bar" 352 | } 353 | } 354 | } 355 | ] 356 | } 357 | } 358 | } 359 | }, 360 | "ScannedCount": 1, 361 | "ConsumedCapacity": null 362 | } 363 | ``` 364 | 365 | Contributing 366 | ------------ 367 | 368 | This is just a side project for me, and I'm more than willing to review pull requests. I just ask that you do the following: 369 | 370 | - Add unit tests 371 | - Try to respect code conventions (e.g. spaces not tabs) 372 | - Provide a useful commit message 373 | -------------------------------------------------------------------------------- /build.gradle: -------------------------------------------------------------------------------- 1 | plugins { 2 | id 'scala' 3 | id 'idea' 4 | id 'com.github.johnrengelman.shadow' version '5.1.0' 5 | id "com.github.maiflai.scalatest" version '0.25' 6 | } 7 | 8 | compileScala.targetCompatibility = 1.8 9 | compileScala.sourceCompatibility = 1.8 10 | 11 | ext { 12 | sparkVersion = '2.4.5' 13 | scalaVersion = '2.11.12' 14 | scalaVersionMinor = scalaVersion.replaceFirst('\\.[0-9]+$', '') 15 | awsVersion = '1.11.650' 16 | } 17 | 18 | repositories { 19 | jcenter() 20 | } 21 | 22 | dependencies { 23 | implementation "org.scala-lang:scala-library:$scalaVersion" 24 | implementation "org.clapper:grizzled-slf4j_$scalaVersionMinor:1.3.4" 25 | implementation "org.json4s:json4s-ext_$scalaVersionMinor:3.5.3" 26 | 27 | compileOnly "org.apache.spark:spark-sql_$scalaVersionMinor:$sparkVersion" 28 | compileOnly "org.apache.spark:spark-hive_$scalaVersionMinor:$sparkVersion" 29 | 30 | compileOnly "com.amazonaws:aws-java-sdk-kinesis:$awsVersion" 31 | compileOnly "com.amazonaws:aws-java-sdk-dynamodb:$awsVersion" 32 | 33 | testImplementation "org.scalatest:scalatest_$scalaVersionMinor:3.0.8" 34 | testImplementation "org.mockito:mockito-scala-scalatest_$scalaVersionMinor:1.6.2" 35 | testRuntime 'org.pegdown:pegdown:1.6.0' 36 | } 37 | 38 | sourceSets { 39 | test.compileClasspath += configurations.compileOnly 40 | test.runtimeClasspath += configurations.compileOnly 41 | } 42 | 43 | test { 44 | testLogging { 45 | events 'passed', 'skipped', 'failed' 46 | } 47 | } 48 | 49 | jar { 50 | manifest { 51 | attributes("Implementation-Title": project.name, 52 | "Implementation-Version": project.version) 53 | } 54 | } 55 | 56 | shadowJar { 57 | classifier = null 58 | mergeServiceFiles() 59 | } 60 | 61 | -------------------------------------------------------------------------------- /gradle.properties: -------------------------------------------------------------------------------- 1 | group=org.chojin 2 | version=0.0.3-SNAPSHOT 3 | -------------------------------------------------------------------------------- /gradle/wrapper/gradle-wrapper.jar: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/thesquelched/spark-lineage/c9b7178cc223f042f603caba8e8c9a437c428a78/gradle/wrapper/gradle-wrapper.jar -------------------------------------------------------------------------------- /gradle/wrapper/gradle-wrapper.properties: -------------------------------------------------------------------------------- 1 | #Thu Oct 10 10:55:43 CDT 2019 2 | distributionUrl=https\://services.gradle.org/distributions/gradle-5.6.2-all.zip 3 | distributionBase=GRADLE_USER_HOME 4 | distributionPath=wrapper/dists 5 | zipStorePath=wrapper/dists 6 | zipStoreBase=GRADLE_USER_HOME 7 | -------------------------------------------------------------------------------- /gradlew: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env sh 2 | 3 | # 4 | # Copyright 2015 the original author or authors. 5 | # 6 | # Licensed under the Apache License, Version 2.0 (the "License"); 7 | # you may not use this file except in compliance with the License. 8 | # You may obtain a copy of the License at 9 | # 10 | # https://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | # 18 | 19 | ############################################################################## 20 | ## 21 | ## Gradle start up script for UN*X 22 | ## 23 | ############################################################################## 24 | 25 | # Attempt to set APP_HOME 26 | # Resolve links: $0 may be a link 27 | PRG="$0" 28 | # Need this for relative symlinks. 29 | while [ -h "$PRG" ] ; do 30 | ls=`ls -ld "$PRG"` 31 | link=`expr "$ls" : '.*-> \(.*\)$'` 32 | if expr "$link" : '/.*' > /dev/null; then 33 | PRG="$link" 34 | else 35 | PRG=`dirname "$PRG"`"/$link" 36 | fi 37 | done 38 | SAVED="`pwd`" 39 | cd "`dirname \"$PRG\"`/" >/dev/null 40 | APP_HOME="`pwd -P`" 41 | cd "$SAVED" >/dev/null 42 | 43 | APP_NAME="Gradle" 44 | APP_BASE_NAME=`basename "$0"` 45 | 46 | # Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script. 47 | DEFAULT_JVM_OPTS='"-Xmx64m" "-Xms64m"' 48 | 49 | # Use the maximum available, or set MAX_FD != -1 to use that value. 50 | MAX_FD="maximum" 51 | 52 | warn () { 53 | echo "$*" 54 | } 55 | 56 | die () { 57 | echo 58 | echo "$*" 59 | echo 60 | exit 1 61 | } 62 | 63 | # OS specific support (must be 'true' or 'false'). 64 | cygwin=false 65 | msys=false 66 | darwin=false 67 | nonstop=false 68 | case "`uname`" in 69 | CYGWIN* ) 70 | cygwin=true 71 | ;; 72 | Darwin* ) 73 | darwin=true 74 | ;; 75 | MINGW* ) 76 | msys=true 77 | ;; 78 | NONSTOP* ) 79 | nonstop=true 80 | ;; 81 | esac 82 | 83 | CLASSPATH=$APP_HOME/gradle/wrapper/gradle-wrapper.jar 84 | 85 | # Determine the Java command to use to start the JVM. 86 | if [ -n "$JAVA_HOME" ] ; then 87 | if [ -x "$JAVA_HOME/jre/sh/java" ] ; then 88 | # IBM's JDK on AIX uses strange locations for the executables 89 | JAVACMD="$JAVA_HOME/jre/sh/java" 90 | else 91 | JAVACMD="$JAVA_HOME/bin/java" 92 | fi 93 | if [ ! -x "$JAVACMD" ] ; then 94 | die "ERROR: JAVA_HOME is set to an invalid directory: $JAVA_HOME 95 | 96 | Please set the JAVA_HOME variable in your environment to match the 97 | location of your Java installation." 98 | fi 99 | else 100 | JAVACMD="java" 101 | which java >/dev/null 2>&1 || die "ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH. 102 | 103 | Please set the JAVA_HOME variable in your environment to match the 104 | location of your Java installation." 105 | fi 106 | 107 | # Increase the maximum file descriptors if we can. 108 | if [ "$cygwin" = "false" -a "$darwin" = "false" -a "$nonstop" = "false" ] ; then 109 | MAX_FD_LIMIT=`ulimit -H -n` 110 | if [ $? -eq 0 ] ; then 111 | if [ "$MAX_FD" = "maximum" -o "$MAX_FD" = "max" ] ; then 112 | MAX_FD="$MAX_FD_LIMIT" 113 | fi 114 | ulimit -n $MAX_FD 115 | if [ $? -ne 0 ] ; then 116 | warn "Could not set maximum file descriptor limit: $MAX_FD" 117 | fi 118 | else 119 | warn "Could not query maximum file descriptor limit: $MAX_FD_LIMIT" 120 | fi 121 | fi 122 | 123 | # For Darwin, add options to specify how the application appears in the dock 124 | if $darwin; then 125 | GRADLE_OPTS="$GRADLE_OPTS \"-Xdock:name=$APP_NAME\" \"-Xdock:icon=$APP_HOME/media/gradle.icns\"" 126 | fi 127 | 128 | # For Cygwin or MSYS, switch paths to Windows format before running java 129 | if [ "$cygwin" = "true" -o "$msys" = "true" ] ; then 130 | APP_HOME=`cygpath --path --mixed "$APP_HOME"` 131 | CLASSPATH=`cygpath --path --mixed "$CLASSPATH"` 132 | JAVACMD=`cygpath --unix "$JAVACMD"` 133 | 134 | # We build the pattern for arguments to be converted via cygpath 135 | ROOTDIRSRAW=`find -L / -maxdepth 1 -mindepth 1 -type d 2>/dev/null` 136 | SEP="" 137 | for dir in $ROOTDIRSRAW ; do 138 | ROOTDIRS="$ROOTDIRS$SEP$dir" 139 | SEP="|" 140 | done 141 | OURCYGPATTERN="(^($ROOTDIRS))" 142 | # Add a user-defined pattern to the cygpath arguments 143 | if [ "$GRADLE_CYGPATTERN" != "" ] ; then 144 | OURCYGPATTERN="$OURCYGPATTERN|($GRADLE_CYGPATTERN)" 145 | fi 146 | # Now convert the arguments - kludge to limit ourselves to /bin/sh 147 | i=0 148 | for arg in "$@" ; do 149 | CHECK=`echo "$arg"|egrep -c "$OURCYGPATTERN" -` 150 | CHECK2=`echo "$arg"|egrep -c "^-"` ### Determine if an option 151 | 152 | if [ $CHECK -ne 0 ] && [ $CHECK2 -eq 0 ] ; then ### Added a condition 153 | eval `echo args$i`=`cygpath --path --ignore --mixed "$arg"` 154 | else 155 | eval `echo args$i`="\"$arg\"" 156 | fi 157 | i=$((i+1)) 158 | done 159 | case $i in 160 | (0) set -- ;; 161 | (1) set -- "$args0" ;; 162 | (2) set -- "$args0" "$args1" ;; 163 | (3) set -- "$args0" "$args1" "$args2" ;; 164 | (4) set -- "$args0" "$args1" "$args2" "$args3" ;; 165 | (5) set -- "$args0" "$args1" "$args2" "$args3" "$args4" ;; 166 | (6) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" ;; 167 | (7) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" ;; 168 | (8) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" "$args7" ;; 169 | (9) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" "$args7" "$args8" ;; 170 | esac 171 | fi 172 | 173 | # Escape application args 174 | save () { 175 | for i do printf %s\\n "$i" | sed "s/'/'\\\\''/g;1s/^/'/;\$s/\$/' \\\\/" ; done 176 | echo " " 177 | } 178 | APP_ARGS=$(save "$@") 179 | 180 | # Collect all arguments for the java command, following the shell quoting and substitution rules 181 | eval set -- $DEFAULT_JVM_OPTS $JAVA_OPTS $GRADLE_OPTS "\"-Dorg.gradle.appname=$APP_BASE_NAME\"" -classpath "\"$CLASSPATH\"" org.gradle.wrapper.GradleWrapperMain "$APP_ARGS" 182 | 183 | # by default we should be in the correct project dir, but when run from Finder on Mac, the cwd is wrong 184 | if [ "$(uname)" = "Darwin" ] && [ "$HOME" = "$PWD" ]; then 185 | cd "$(dirname "$0")" 186 | fi 187 | 188 | exec "$JAVACMD" "$@" 189 | -------------------------------------------------------------------------------- /gradlew.bat: -------------------------------------------------------------------------------- 1 | @rem 2 | @rem Copyright 2015 the original author or authors. 3 | @rem 4 | @rem Licensed under the Apache License, Version 2.0 (the "License"); 5 | @rem you may not use this file except in compliance with the License. 6 | @rem You may obtain a copy of the License at 7 | @rem 8 | @rem https://www.apache.org/licenses/LICENSE-2.0 9 | @rem 10 | @rem Unless required by applicable law or agreed to in writing, software 11 | @rem distributed under the License is distributed on an "AS IS" BASIS, 12 | @rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | @rem See the License for the specific language governing permissions and 14 | @rem limitations under the License. 15 | @rem 16 | 17 | @if "%DEBUG%" == "" @echo off 18 | @rem ########################################################################## 19 | @rem 20 | @rem Gradle startup script for Windows 21 | @rem 22 | @rem ########################################################################## 23 | 24 | @rem Set local scope for the variables with windows NT shell 25 | if "%OS%"=="Windows_NT" setlocal 26 | 27 | set DIRNAME=%~dp0 28 | if "%DIRNAME%" == "" set DIRNAME=. 29 | set APP_BASE_NAME=%~n0 30 | set APP_HOME=%DIRNAME% 31 | 32 | @rem Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script. 33 | set DEFAULT_JVM_OPTS="-Xmx64m" "-Xms64m" 34 | 35 | @rem Find java.exe 36 | if defined JAVA_HOME goto findJavaFromJavaHome 37 | 38 | set JAVA_EXE=java.exe 39 | %JAVA_EXE% -version >NUL 2>&1 40 | if "%ERRORLEVEL%" == "0" goto init 41 | 42 | echo. 43 | echo ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH. 44 | echo. 45 | echo Please set the JAVA_HOME variable in your environment to match the 46 | echo location of your Java installation. 47 | 48 | goto fail 49 | 50 | :findJavaFromJavaHome 51 | set JAVA_HOME=%JAVA_HOME:"=% 52 | set JAVA_EXE=%JAVA_HOME%/bin/java.exe 53 | 54 | if exist "%JAVA_EXE%" goto init 55 | 56 | echo. 57 | echo ERROR: JAVA_HOME is set to an invalid directory: %JAVA_HOME% 58 | echo. 59 | echo Please set the JAVA_HOME variable in your environment to match the 60 | echo location of your Java installation. 61 | 62 | goto fail 63 | 64 | :init 65 | @rem Get command-line arguments, handling Windows variants 66 | 67 | if not "%OS%" == "Windows_NT" goto win9xME_args 68 | 69 | :win9xME_args 70 | @rem Slurp the command line arguments. 71 | set CMD_LINE_ARGS= 72 | set _SKIP=2 73 | 74 | :win9xME_args_slurp 75 | if "x%~1" == "x" goto execute 76 | 77 | set CMD_LINE_ARGS=%* 78 | 79 | :execute 80 | @rem Setup the command line 81 | 82 | set CLASSPATH=%APP_HOME%\gradle\wrapper\gradle-wrapper.jar 83 | 84 | @rem Execute Gradle 85 | "%JAVA_EXE%" %DEFAULT_JVM_OPTS% %JAVA_OPTS% %GRADLE_OPTS% "-Dorg.gradle.appname=%APP_BASE_NAME%" -classpath "%CLASSPATH%" org.gradle.wrapper.GradleWrapperMain %CMD_LINE_ARGS% 86 | 87 | :end 88 | @rem End local scope for the variables with windows NT shell 89 | if "%ERRORLEVEL%"=="0" goto mainEnd 90 | 91 | :fail 92 | rem Set variable GRADLE_EXIT_CONSOLE if you need the _script_ return code instead of 93 | rem the _cmd.exe /c_ return code! 94 | if not "" == "%GRADLE_EXIT_CONSOLE%" exit 1 95 | exit /b 1 96 | 97 | :mainEnd 98 | if "%OS%"=="Windows_NT" endlocal 99 | 100 | :omega 101 | -------------------------------------------------------------------------------- /settings.gradle: -------------------------------------------------------------------------------- 1 | /* 2 | * This file was generated by the Gradle 'init' task. 3 | * 4 | * The settings file is used to specify which projects to include in your build. 5 | * 6 | * Detailed information about configuring a multi-project build in Gradle can be found 7 | * in the user manual at https://docs.gradle.org/5.6.2/userguide/multi_project_builds.html 8 | */ 9 | 10 | rootProject.name = 'spark-lineage' 11 | -------------------------------------------------------------------------------- /src/main/scala/org/chojin/spark/lineage/Config.scala: -------------------------------------------------------------------------------- 1 | package org.chojin.spark.lineage 2 | 3 | import java.util.Properties 4 | 5 | import grizzled.slf4j.Logger 6 | 7 | import scala.collection.JavaConversions._ 8 | 9 | object Config { 10 | private lazy val LOGGER = Logger[this.type] 11 | 12 | private final val prefix = "org.chojin.spark.lineage" 13 | private lazy val properties = { 14 | Option(getClass.getResourceAsStream("/lineage.properties")) 15 | .map({ stream => 16 | val props = new Properties() 17 | props.load(stream) 18 | stream.close() 19 | 20 | props 21 | }) 22 | .getOrElse(new Properties()) 23 | } 24 | 25 | def get(name: String): String = properties.getProperty(name) 26 | 27 | def getList(name: String): Seq[String] = Option.apply(properties.getProperty(name)) 28 | .flatMap(p => if (p.isEmpty) None else Some(p)) 29 | .map(p => p.split("\\s*,\\s*").toSeq) 30 | .getOrElse(Seq()) 31 | 32 | def createInstanceOf[T](suffix: String): T = { 33 | val propPrefix = s"$prefix.$suffix" 34 | val className = get(propPrefix) 35 | 36 | createInstance(className, propPrefix) 37 | } 38 | 39 | def createInstancesOf[T](suffix: String): List[T] = { 40 | val propPrefix = s"$prefix.$suffix" 41 | 42 | getList(s"${propPrefix}s").map(className => { 43 | createInstance[T](className, propPrefix) 44 | }).toList 45 | } 46 | 47 | private def createInstance[T](className: String, prefix: String): T = { 48 | try { 49 | def clazz = getClass.getClassLoader.loadClass(className) 50 | val configKey = clazz.getSimpleName.replaceFirst("Reporter$", "").toLowerCase 51 | 52 | val clazzPrefix = s"$prefix.$configKey" 53 | 54 | val props = properties 55 | .toMap 56 | .filter({ case (k, _) => k.startsWith(s"$clazzPrefix.")}) 57 | .map({ case (k, v) => k.substring(clazzPrefix.length + 1) -> v}) 58 | 59 | LOGGER.debug(s"Properties -> $props") 60 | 61 | clazz 62 | .getConstructor(classOf[Map[String, String]]) 63 | .newInstance(props) 64 | .asInstanceOf[T] 65 | } catch { 66 | case e: Throwable => { 67 | LOGGER.error(s"Unable to create instance of $className", e) 68 | throw e 69 | } 70 | } 71 | } 72 | } 73 | -------------------------------------------------------------------------------- /src/main/scala/org/chojin/spark/lineage/QueryParser.scala: -------------------------------------------------------------------------------- 1 | package org.chojin.spark.lineage 2 | 3 | import grizzled.slf4j.Logger 4 | import org.apache.spark.sql.catalyst.catalog.HiveTableRelation 5 | import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference} 6 | import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LogicalPlan, Project, Union} 7 | import org.apache.spark.sql.execution.QueryExecution 8 | import org.apache.spark.sql.execution.datasources.{InsertIntoHadoopFsRelationCommand, LogicalRelation} 9 | import org.chojin.spark.lineage.inputs.How.How 10 | import org.chojin.spark.lineage.inputs.{Field, HiveInput, How, Input} 11 | import org.chojin.spark.lineage.outputs.FsOutput 12 | import org.chojin.spark.lineage.report.{Metadata, Report} 13 | 14 | object QueryParser { 15 | private lazy val LOGGER = Logger[this.type] 16 | 17 | def findSource(attr: AttributeReference, plan: LogicalPlan, how: How): Seq[Input] = findSource(attr.toAttribute, plan, how) 18 | 19 | def findSource(attr: Attribute, plan: LogicalPlan, how: How): Seq[Input] = { 20 | plan.collect { 21 | //case u: Union => u.children.flatMap(child => child.output.flatMap(childAttr => findSource(childAttr, child, how))) 22 | case r: HiveTableRelation if r.outputSet.contains(attr) => Seq(HiveInput(r.tableMeta.qualifiedName, Set(Field(attr.name, how)))) 23 | case r: LogicalRelation if r.outputSet.contains(attr) => Seq(HiveInput(r.catalogTable.get.qualifiedName, Set(Field(attr.name, how)))) 24 | case j: Join => { 25 | val conds = j.condition.map { cond => 26 | cond.collect { 27 | case ar: AttributeReference => j.children.flatMap({ child => findSource(ar, child, How.JOIN) }) 28 | case al: Alias => al.collect { 29 | case ar: AttributeReference => j.children.flatMap({ child => findSource(ar, child, How.JOIN) }) 30 | }.flatten 31 | }.flatten 32 | } 33 | .getOrElse(Seq()) 34 | .filter({ input => input.fields.map({ f => f.how == How.JOIN }).reduce((a, b) => a || b) }) 35 | 36 | j.children.flatMap({ child => findSource(attr, child, null) ++ conds }) 37 | } 38 | case p: Project => { 39 | p.projectList.flatMap { proj => 40 | proj.collect { 41 | case ar: AttributeReference if ar.toAttribute == attr => findSource(ar, p.child, How.PROJECTION) 42 | case al: Alias if al.name == attr.name => al.collect { 43 | case ar: AttributeReference => findSource(ar, p.child, How.PROJECTION) 44 | }.flatten 45 | }.flatten 46 | } 47 | } 48 | case f: Filter => f.condition.collect { 49 | case ar: AttributeReference => findSource(ar, f.child, How.FILTER) 50 | case _ => Seq() 51 | }.flatten 52 | case a: Aggregate => { 53 | val groupings = a.groupingExpressions.collect { 54 | case ar: AttributeReference => findSource(ar, a.child, How.GROUPBY) 55 | case al: Alias => al.collect { 56 | case ar: AttributeReference => findSource(ar, a.child, How.GROUPBY) 57 | }.flatten 58 | }.flatten 59 | 60 | val aggregates = a.aggregateExpressions.diff(a.groupingExpressions).collect { 61 | case ar: AttributeReference if ar.toAttribute == attr => findSource(ar, a.child, How.AGGREGATE) ++ groupings 62 | case al: Alias if al.name == attr.name => al.collect { 63 | case ar: AttributeReference => findSource(ar, a.child, How.AGGREGATE) ++ groupings 64 | }.flatten 65 | }.flatten 66 | 67 | aggregates ++ groupings 68 | } 69 | }.flatten 70 | } 71 | 72 | def parseQuery(qe: QueryExecution): Option[Report] = { 73 | qe.logical.collectFirst { 74 | case c: InsertIntoHadoopFsRelationCommand => { 75 | val output = FsOutput(c.outputPath.toString, c.fileFormat.toString) 76 | 77 | val sources = c.query match { 78 | case u: Union => u.children.flatMap(child => child.output.map(childAttr => childAttr -> findSource(childAttr, child, null))) 79 | case query => query.output.map { 80 | attr => attr -> findSource(attr, query, null) 81 | } 82 | } 83 | 84 | val fields: Seq[(String, List[Input])] = sources 85 | .map{ case (field, rawInputs) => 86 | val inputs = rawInputs 87 | .groupBy { case HiveInput(name, _, _) => (HiveInput, name) } 88 | .map { case (key, value) => 89 | key match { 90 | case (HiveInput, table) => 91 | HiveInput( 92 | table, 93 | value 94 | .map(_.fields.filter({ col => col.how != null })) 95 | .reduce((a, b) => a ++ b)) 96 | } 97 | } 98 | 99 | field.name -> inputs.toList 100 | } 101 | .groupBy { case (k, _) => k } 102 | .map { case (k, v) => k -> v.flatMap(_._2).toList } 103 | .toSeq 104 | 105 | LOGGER.debug(s"Fields: $fields") 106 | 107 | val metadata = Metadata(qe.sparkSession.sparkContext.appName) 108 | 109 | Report(metadata, output, Map(fields: _*)) 110 | } 111 | case other => 112 | LOGGER.info(s"Unable to produce report for node ${other.getClass.getSimpleName}") 113 | null 114 | } 115 | } 116 | } 117 | -------------------------------------------------------------------------------- /src/main/scala/org/chojin/spark/lineage/ReportProcessor.scala: -------------------------------------------------------------------------------- 1 | package org.chojin.spark.lineage 2 | 3 | import java.util.concurrent.{LinkedBlockingQueue, TimeUnit} 4 | 5 | import grizzled.slf4j.Logger 6 | import org.apache.spark.sql.execution.QueryExecution 7 | import org.chojin.spark.lineage.reporter.Reporter 8 | 9 | class ReportProcessor(private val reporters: List[Reporter]) { 10 | private lazy val LOGGER = Logger[this.type] 11 | 12 | private val queue = new LinkedBlockingQueue[QueryExecution](1000) 13 | 14 | private lazy val thread = new Thread { 15 | override def run(): Unit = processReports() 16 | } 17 | 18 | def runInBackground(): Unit = { 19 | thread.setName(getClass.getName.concat("-process")) 20 | thread.setDaemon(true) 21 | 22 | Option(Thread.currentThread().getContextClassLoader) match { 23 | case Some(loader) if getClass.getClassLoader != loader => thread.setContextClassLoader(loader) 24 | case _ => LOGGER.debug("Context loader not set") 25 | } 26 | 27 | thread.start() 28 | } 29 | 30 | def offer(qe: QueryExecution, async: Boolean = true): Unit = { 31 | if (!queue.offer(qe, 10L, TimeUnit.SECONDS)) { 32 | LOGGER.warn("Unable to query execution to queue; dropping on the floor") 33 | LOGGER.debug(s"Skipped query plan:\n${qe.logical.treeString(verbose = true)}") 34 | } 35 | 36 | if (!async) { 37 | processReport() 38 | } 39 | } 40 | 41 | def processReport() = { 42 | if (reporters.isEmpty) { 43 | LOGGER.warn("No lineage reporters found") 44 | } else { 45 | reporters.foreach(reporter => LOGGER.info(s"Using reporter $reporter")) 46 | } 47 | 48 | LOGGER.debug("Polling for report to process") 49 | Option(queue.poll(500L, TimeUnit.MILLISECONDS)).foreach({qe => { 50 | LOGGER.info("Processing report") 51 | LOGGER.debug(s"Query execution: $qe") 52 | 53 | QueryParser.parseQuery(qe).foreach(report => { 54 | LOGGER.debug(s"Produced report: ${report.prettyPrint}") 55 | 56 | reporters.foreach(reporter => reporter.report(report)) 57 | 58 | LOGGER.info("Successfully processed report") 59 | }) 60 | }}) 61 | } 62 | 63 | def processReports(): Unit = { 64 | LOGGER.info("Starting report processor thread") 65 | 66 | val reportersStr = reporters.map(reporter => s" $reporter").mkString("\n") 67 | LOGGER.info(s"Found ${reporters.length} reporters:\n$reportersStr") 68 | 69 | var running = true 70 | 71 | while(running) { 72 | try { 73 | processReport() 74 | } catch { 75 | case _: InterruptedException => { 76 | LOGGER.info("Caught InterruptedException; exiting...") 77 | running = false 78 | } 79 | case e: Throwable => { 80 | LOGGER.error("Caught exception while processing report", e) 81 | } 82 | } 83 | } 84 | } 85 | } 86 | -------------------------------------------------------------------------------- /src/main/scala/org/chojin/spark/lineage/SparkSqlLineageListener.scala: -------------------------------------------------------------------------------- 1 | package org.chojin.spark.lineage 2 | 3 | import grizzled.slf4j.Logger 4 | import org.apache.spark.sql.execution.QueryExecution 5 | import org.apache.spark.sql.util.QueryExecutionListener 6 | import org.chojin.spark.lineage.reporter.Reporter 7 | 8 | case class SparkSqlLineageListener(reporters: List[Reporter], async: Boolean = true) extends QueryExecutionListener { 9 | private lazy val LOGGER = Logger[this.type] 10 | private lazy val processor = new ReportProcessor(reporters) 11 | 12 | def this() = this(Config.createInstancesOf("reporter")) 13 | 14 | if (async) { 15 | processor.runInBackground() 16 | } 17 | 18 | override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = { 19 | LOGGER.debug(s"Logical plan:\n${qe.logical}") 20 | 21 | LOGGER.info("Offering query execution to report processor") 22 | processor.offer(qe, async) 23 | } 24 | 25 | override def onFailure(funcName: String, qe: QueryExecution, exception: Exception): Unit = { 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /src/main/scala/org/chojin/spark/lineage/inputs/package.scala: -------------------------------------------------------------------------------- 1 | package org.chojin.spark.lineage 2 | 3 | import org.chojin.spark.lineage.inputs.How.How 4 | 5 | package object inputs { 6 | 7 | sealed trait Input { 8 | val typeName: String 9 | val fields: Set[Field] 10 | 11 | def toMap(): Map[String, Any] 12 | } 13 | 14 | object How extends Enumeration { 15 | type How = Value 16 | val PROJECTION, FILTER, JOIN, AGGREGATE, GROUPBY = Value 17 | } 18 | 19 | sealed case class Field(name: String, how: How) { 20 | override def toString: String = s"Column(name: $name, how: ${how}" 21 | 22 | def toMap(): Map[String, String] = Map( 23 | "name" -> name, 24 | "how" -> how.toString.toLowerCase 25 | ) 26 | } 27 | 28 | case class HiveInput(name: String, fields: Set[Field], typeName: String = "hive") extends Input { 29 | override def toString: String = s"HiveInput(name: $name, columns: ${fields.mkString(", ")})" 30 | 31 | override def toMap(): Map[String, Any] = Map( 32 | "type" -> typeName, 33 | "name" -> name, 34 | "fields" -> fields.map(_.toMap()) 35 | ) 36 | } 37 | 38 | } 39 | 40 | -------------------------------------------------------------------------------- /src/main/scala/org/chojin/spark/lineage/outputs/package.scala: -------------------------------------------------------------------------------- 1 | package org.chojin.spark.lineage 2 | 3 | package object outputs { 4 | sealed trait Output { 5 | val typeName: String 6 | def toMap(): Map[String, Any] 7 | } 8 | 9 | case class FsOutput(path: String, format: String, typeName: String = "fs") extends Output { 10 | override def toMap: Map[String, Any] = Map( 11 | "type" -> typeName, 12 | "format" -> format, 13 | "path" -> path 14 | ) 15 | } 16 | } 17 | 18 | -------------------------------------------------------------------------------- /src/main/scala/org/chojin/spark/lineage/report/Metadata.scala: -------------------------------------------------------------------------------- 1 | package org.chojin.spark.lineage.report 2 | 3 | case class Metadata(appName: String) { 4 | def toMap(): Map[String, String] = Map( 5 | "appName" -> appName 6 | ) 7 | } 8 | -------------------------------------------------------------------------------- /src/main/scala/org/chojin/spark/lineage/report/Report.scala: -------------------------------------------------------------------------------- 1 | package org.chojin.spark.lineage.report 2 | 3 | import org.chojin.spark.lineage.inputs.{HiveInput, How, Input} 4 | import org.chojin.spark.lineage.outputs.Output 5 | import org.json4s._ 6 | import org.json4s.ext.EnumNameSerializer 7 | import org.json4s.jackson.Serialization 8 | import org.json4s.jackson.Serialization.write 9 | 10 | case class Report(metadata: Metadata, output: Output, fields: Map[String, List[Input]]) { 11 | implicit val formats = Serialization.formats(NoTypeHints) + new EnumNameSerializer(How) 12 | 13 | override def equals(other: Any): Boolean = other match { 14 | case Report(otherMeta, otherOutput, otherInput) => ( 15 | metadata == otherMeta 16 | && output == otherOutput 17 | && fields.mapValues(_.toSet).toSet == otherInput.mapValues(_.toSet).toSet 18 | ) 19 | case _ => false 20 | } 21 | 22 | def prettyPrint = { 23 | val fieldsStr = fields.map { case (k, v) => 24 | val valStr = v.map({ 25 | case HiveInput(name, columns, _) => { 26 | s""" HiveInput( 27 | | name: $name 28 | | columns: 29 | | ${columns.mkString("\n ")}""".stripMargin 30 | } 31 | case field => s" $field" 32 | }).mkString("\n") 33 | s" $k:\n$valStr" 34 | }.mkString("\n") 35 | 36 | s"""|Report( 37 | | metadata: $metadata, 38 | | output: $output, 39 | | fields: 40 | |$fieldsStr 41 | |""".stripMargin 42 | } 43 | 44 | def toJson(): String = { 45 | write(this) 46 | } 47 | 48 | def toMap(): Map[String, Any] = Map( 49 | "metadata" -> metadata.toMap(), 50 | "output" -> output.toMap(), 51 | "fields" -> fields.map({ case (k, v) => k -> v.map(_.toMap()) }) 52 | ) 53 | } 54 | -------------------------------------------------------------------------------- /src/main/scala/org/chojin/spark/lineage/reporter/DynamodbReporter.scala: -------------------------------------------------------------------------------- 1 | package org.chojin.spark.lineage.reporter 2 | 3 | import java.nio.ByteBuffer 4 | 5 | import com.amazonaws.services.dynamodbv2.model.{AttributeValue, PutItemRequest} 6 | import com.amazonaws.services.dynamodbv2.{AmazonDynamoDB, AmazonDynamoDBClientBuilder} 7 | import org.chojin.spark.lineage.outputs.FsOutput 8 | import org.chojin.spark.lineage.report.Report 9 | 10 | import scala.collection.JavaConversions._ 11 | 12 | case class DynamodbReporter(table: String, 13 | region: Option[String], 14 | compression: Option[String], 15 | json: Boolean = false, 16 | _client: Option[AmazonDynamoDB] = None) extends Reporter { 17 | def this(props: Map[String, String]) = this( 18 | table=props("table"), 19 | region=props.get("region"), 20 | compression=props.get("compression"), 21 | json=props.get("json").exists(value => Set("true", "yes", "1").contains(value.toLowerCase)) 22 | ) 23 | 24 | private lazy val client = _client.getOrElse(region match { 25 | case Some(r) => AmazonDynamoDBClientBuilder.standard().withRegion(r).build() 26 | case None => AmazonDynamoDBClientBuilder.defaultClient() 27 | }) 28 | 29 | override def report(report: Report): Unit = { 30 | val outputKey = report.output match { 31 | case FsOutput(path, _, typeName) => s"$typeName-$path" 32 | } 33 | 34 | val baseItem = Map( 35 | "outputKey" -> new AttributeValue().withS(outputKey) 36 | ) 37 | 38 | val reportItem = if (json) Map("json" -> compress(report.toJson().getBytes())) else report.toMap() 39 | val item = baseItem ++ reportItem.map({ case (k, v) => k -> makeAttr(v) }) 40 | 41 | client.putItem( 42 | new PutItemRequest() 43 | .withTableName(table) 44 | .withItem(item)) 45 | } 46 | 47 | private def makeAttr(item: Any): AttributeValue = item match { 48 | case s: String => new AttributeValue().withS(s) 49 | case m: Map[String, Any] => new AttributeValue().withM(m.map({ case (k, v) => k -> makeAttr(v) })) 50 | case i: Iterable[Any] => new AttributeValue().withL(i.map(makeAttr).toList) 51 | case b: Array[Byte] => new AttributeValue().withB(ByteBuffer.wrap(b)) 52 | } 53 | } 54 | -------------------------------------------------------------------------------- /src/main/scala/org/chojin/spark/lineage/reporter/InMemoryReporter.scala: -------------------------------------------------------------------------------- 1 | package org.chojin.spark.lineage.reporter 2 | 3 | import org.chojin.spark.lineage.report.Report 4 | 5 | import scala.collection.mutable.ListBuffer 6 | 7 | case class InMemoryReporter(compression: Option[String]) extends Reporter { 8 | private val reports = new ListBuffer[Report]() 9 | 10 | override def report(report: Report): Unit = { 11 | reports += report 12 | } 13 | 14 | def getReports(): List[Report] = reports.toList 15 | 16 | def clear(): Unit = reports.clear() 17 | } 18 | -------------------------------------------------------------------------------- /src/main/scala/org/chojin/spark/lineage/reporter/KinesisReporter.scala: -------------------------------------------------------------------------------- 1 | package org.chojin.spark.lineage.reporter 2 | 3 | import java.nio.ByteBuffer 4 | 5 | import com.amazonaws.services.kinesis.AmazonKinesisClientBuilder 6 | import com.amazonaws.services.kinesis.model.PutRecordRequest 7 | import org.chojin.spark.lineage.report.Report 8 | 9 | case class KinesisReporter(stream: String, 10 | region: Option[String], 11 | shard: String, 12 | compression: Option[String]) extends Reporter { 13 | def this(props: Map[String, String]) = this( 14 | stream=props("stream"), 15 | region=props.get("region"), 16 | shard=props("shard"), 17 | compression=props.get("compression")) 18 | 19 | private lazy val client = region match { 20 | case Some(r) => AmazonKinesisClientBuilder.standard().withRegion(r).build 21 | case None => AmazonKinesisClientBuilder.defaultClient() 22 | } 23 | 24 | override def report(report: Report): Unit = { 25 | val payload = compress(report.toJson().getBytes()) 26 | 27 | client.putRecord( 28 | new PutRecordRequest() 29 | .withStreamName(stream) 30 | .withPartitionKey(shard) 31 | .withData(ByteBuffer.wrap(payload))) 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /src/main/scala/org/chojin/spark/lineage/reporter/Reporter.scala: -------------------------------------------------------------------------------- 1 | package org.chojin.spark.lineage.reporter 2 | 3 | import java.io.ByteArrayOutputStream 4 | 5 | import org.apache.hadoop.conf.Configuration 6 | import org.apache.hadoop.io.compress.CompressionCodecFactory 7 | import org.chojin.spark.lineage.report.Report 8 | 9 | trait Reporter { 10 | val compression: Option[String] 11 | 12 | def report(report: Report): Unit 13 | 14 | private lazy val codec = compression.map(name => 15 | new CompressionCodecFactory(new Configuration()).getCodecByName(name)) 16 | 17 | protected def compress(data: Array[Byte]): Array[Byte] = codec.map(codec => { 18 | val bos = new ByteArrayOutputStream() 19 | val os = codec.createOutputStream(bos) 20 | 21 | os.write(data) 22 | os.flush() 23 | os.close() 24 | 25 | bos.toByteArray 26 | }).getOrElse(data) 27 | } 28 | -------------------------------------------------------------------------------- /src/test/resources/lineage.properties: -------------------------------------------------------------------------------- 1 | org.chojin.spark.lineage.reporters=org.chojin.spark.lineage.reporter.DynamodbReporter,org.chojin.spark.lineage.reporter.KinesisReporter 2 | org.chojin.spark.lineage.reporter.dynamodb.table=my-table 3 | org.chojin.spark.lineage.reporter.dynamodb.region=us-west-2 4 | org.chojin.spark.lineage.reporter.kinesis.stream=my-stream 5 | org.chojin.spark.lineage.reporter.kinesis.region=us-west-2 6 | org.chojin.spark.lineage.reporter.kinesis.shard=0 7 | org.chojin.spark.lineage.reporter.kinesis.compression=gzip 8 | -------------------------------------------------------------------------------- /src/test/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | log4j.rootLogger=ERROR, console 2 | log4j.appender.console=org.apache.log4j.ConsoleAppender 3 | log4j.appender.console.target=System.out 4 | log4j.appender.console.layout=org.apache.log4j.PatternLayout 5 | log4j.appender.console.layout.ConversionPattern=%p %c{1}: %m%n 6 | 7 | log4j.logger.org.chojin=DEBUG 8 | -------------------------------------------------------------------------------- /src/test/scala/org/chojin/spark/lineage/ConfigSpec.scala: -------------------------------------------------------------------------------- 1 | package org.chojin.spark.lineage 2 | 3 | import org.chojin.spark.lineage.reporter.{DynamodbReporter, KinesisReporter, Reporter} 4 | import org.mockito.{ArgumentMatchersSugar, MockitoSugar} 5 | import org.scalatest.{WordSpec, Matchers => ScalaTestMatchers} 6 | 7 | class ConfigSpec extends WordSpec with MockitoSugar with ScalaTestMatchers with ArgumentMatchersSugar { 8 | 9 | "ConfigSpec" should { 10 | 11 | "createInstancesOf" in { 12 | val reporters: List[Reporter] = Config.createInstancesOf("reporter") 13 | 14 | reporters shouldEqual List( 15 | DynamodbReporter("my-table", Some("us-west-2"), None), 16 | KinesisReporter("my-stream", Some("us-west-2"), "0", Some("gzip")) 17 | ) 18 | } 19 | } 20 | } 21 | -------------------------------------------------------------------------------- /src/test/scala/org/chojin/spark/lineage/SparkSqlLineageListenerSpec.scala: -------------------------------------------------------------------------------- 1 | package org.chojin.spark.lineage 2 | 3 | import java.io.File 4 | import java.nio.file.{Files, Path} 5 | 6 | import org.apache.commons.io.FileUtils 7 | import org.apache.spark.sql.SparkSession 8 | import org.apache.spark.sql.functions._ 9 | import org.apache.spark.sql.types.StringType 10 | import org.chojin.spark.lineage.inputs.{Field, HiveInput, How} 11 | import org.chojin.spark.lineage.outputs.FsOutput 12 | import org.chojin.spark.lineage.report.{Metadata, Report} 13 | import org.chojin.spark.lineage.reporter.InMemoryReporter 14 | import org.scalatest._ 15 | 16 | class SparkSqlLineageListenerSpec extends FunSuite with BeforeAndAfterAll with BeforeAndAfterEach with Matchers with Inside { 17 | private var spark: SparkSession = _ 18 | 19 | private val tempDir: Path = Files.createTempDirectory("listener-test-") 20 | private val reporter: InMemoryReporter = InMemoryReporter(compression = None) 21 | private val listener: SparkSqlLineageListener = SparkSqlLineageListener(List(reporter), async = false) 22 | private val outputPath = tempDir.resolve("test.parquet") 23 | 24 | override protected def beforeAll(): Unit = { 25 | super.beforeAll() 26 | 27 | spark = SparkSession 28 | .builder() 29 | .master("local") 30 | .appName("test") 31 | .config("spark.ui.enabled", "false") 32 | .enableHiveSupport() 33 | .getOrCreate() 34 | 35 | spark.sql("drop database if exists test cascade") 36 | spark.sql("create database test") 37 | 38 | Seq("foo", "bar", "baz").foreach { name => 39 | val path = tempDir.resolve(s"test/$name/day=2019-10-01/data.csv") 40 | path.getParent.toFile.mkdirs() 41 | 42 | Files.write(path, "1,a,10\n2,b,20\n3,c,30\n".getBytes()) 43 | 44 | spark.sql( 45 | s""" 46 | |CREATE EXTERNAL TABLE test.$name ( 47 | | pk BIGINT, 48 | | name STRING, 49 | | value BIGINT 50 | |) 51 | |PARTITIONED BY (day STRING) 52 | |ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' 53 | |STORED AS TEXTFILE 54 | |LOCATION '${path.getParent.getParent.toString}' 55 | |""".stripMargin) 56 | spark.sql( 57 | s""" 58 | |ALTER TABLE test.$name 59 | |ADD PARTITION (day='2019-10-01') 60 | |LOCATION '${path.getParent.toString}' 61 | |""".stripMargin) 62 | } 63 | 64 | spark.listenerManager.register(listener) 65 | } 66 | 67 | override protected def beforeEach(): Unit = { 68 | super.beforeEach() 69 | 70 | reporter.clear() 71 | } 72 | 73 | override protected def afterEach(): Unit = { 74 | FileUtils.deleteDirectory(outputPath.toFile) 75 | 76 | super.afterEach() 77 | } 78 | 79 | override protected def afterAll(): Unit = { 80 | spark.listenerManager.unregister(listener) 81 | spark.sessionState.catalog.reset() 82 | spark.stop() 83 | 84 | SparkSession.clearActiveSession() 85 | SparkSession.clearDefaultSession() 86 | 87 | FileUtils.deleteDirectory(tempDir.toFile) 88 | FileUtils.deleteDirectory(new File("spark-warehouse")) 89 | 90 | super.afterAll() 91 | } 92 | 93 | test("hive projection") { 94 | val ss = spark 95 | import ss.implicits._ 96 | 97 | spark.table("test.foo") 98 | .select('pk, concat('name, 'value.cast(StringType)) as 'new_value) 99 | .write 100 | .parquet(outputPath.toString) 101 | 102 | val expected = Report( 103 | Metadata("test"), 104 | FsOutput(s"file:$outputPath", "Parquet"), 105 | Map( 106 | "pk" -> List( 107 | HiveInput( 108 | "test.foo", 109 | Set(Field("pk", How.PROJECTION)))), 110 | "new_value" -> List( 111 | HiveInput( 112 | "test.foo", 113 | Set(Field("name", How.PROJECTION), Field("value", How.PROJECTION)))))) 114 | 115 | reporter.getReports() should contain theSameElementsInOrderAs List(expected) 116 | } 117 | 118 | test("hive filter") { 119 | val ss = spark 120 | import ss.implicits._ 121 | 122 | spark.table("test.foo") 123 | .filter('name =!= "c") 124 | .select('pk, concat('pk, 'value.cast(StringType)) as 'new_value) 125 | .write 126 | .parquet(outputPath.toString) 127 | 128 | val expected = Report( 129 | Metadata("test"), 130 | FsOutput(s"file:$outputPath", "Parquet"), 131 | Map( 132 | "pk" -> List( 133 | HiveInput( 134 | "test.foo", 135 | Set( 136 | Field("pk", How.PROJECTION), 137 | Field("name", How.FILTER)))), 138 | "new_value" -> List( 139 | HiveInput( 140 | "test.foo", 141 | Set( 142 | Field("pk", How.PROJECTION), 143 | Field("value", How.PROJECTION), 144 | Field("name", How.FILTER)))))) 145 | 146 | reporter.getReports() should contain theSameElementsInOrderAs List(expected) 147 | } 148 | 149 | test("hive aggregate") { 150 | val ss = spark 151 | import ss.implicits._ 152 | 153 | spark.table("test.foo") 154 | .groupBy('name) 155 | .count 156 | .write 157 | .parquet(outputPath.toString) 158 | 159 | val expected = Report( 160 | Metadata("test"), 161 | FsOutput(s"file:$outputPath", "Parquet"), 162 | Map( 163 | "count" -> List(HiveInput("test.foo", Set(Field("name", How.GROUPBY)))), 164 | "name" -> List(HiveInput("test.foo", Set(Field("name", How.GROUPBY)))))) 165 | 166 | reporter.getReports() should contain theSameElementsInOrderAs List(expected) 167 | } 168 | 169 | test("hive aggregate sum") { 170 | val ss = spark 171 | import ss.implicits._ 172 | 173 | spark.table("test.foo") 174 | .groupBy('name) 175 | .agg(sum('value) as 'sum) 176 | .write 177 | .parquet(outputPath.toString) 178 | 179 | val expected = Report( 180 | Metadata("test"), 181 | FsOutput(s"file:$outputPath", "Parquet"), 182 | Map( 183 | "sum" -> List( 184 | HiveInput( 185 | "test.foo", 186 | Set( 187 | Field("name", How.GROUPBY), 188 | Field("value", How.AGGREGATE)))), 189 | "name" -> List(HiveInput("test.foo", Set(Field("name", How.GROUPBY)))))) 190 | 191 | reporter.getReports() should contain theSameElementsInOrderAs List(expected) 192 | } 193 | 194 | test("hive join") { 195 | val ss = spark 196 | import ss.implicits._ 197 | 198 | val foo = spark.table("test.foo").select('pk, 'name, 'value) 199 | val bar = spark.table("test.bar") 200 | 201 | 202 | foo.join(bar, Seq("pk")) 203 | .select(foo.col("*"), bar.col("name") as 'bar_name) 204 | .write 205 | .parquet(outputPath.toString) 206 | 207 | val expected = Report( 208 | Metadata("test"), 209 | FsOutput(s"file:$outputPath", "Parquet"), 210 | Map( 211 | "pk" -> List( 212 | HiveInput("test.foo", Set(Field("pk", How.JOIN), Field("pk", How.PROJECTION))), 213 | HiveInput("test.bar", Set(Field("pk", How.JOIN)))), 214 | "name" -> List( 215 | HiveInput( 216 | "test.foo", 217 | Set(Field("pk", How.JOIN), Field("name", How.PROJECTION))), 218 | HiveInput( 219 | "test.bar", 220 | Set(Field("pk", How.JOIN)))), 221 | "value" -> List( 222 | HiveInput( 223 | "test.foo", 224 | Set(Field("pk", How.JOIN), Field("value", How.PROJECTION))), 225 | HiveInput( 226 | "test.bar", 227 | Set(Field("pk", How.JOIN)))), 228 | "bar_name" -> List( 229 | HiveInput( 230 | "test.foo", 231 | Set(Field("pk", How.JOIN))), 232 | HiveInput( 233 | "test.bar", 234 | Set(Field("pk", How.JOIN), Field("name", How.PROJECTION)))))) 235 | 236 | reporter.getReports() should contain theSameElementsInOrderAs List(expected) 237 | } 238 | 239 | test("hive union") { 240 | val ss = spark 241 | import ss.implicits._ 242 | 243 | Seq("foo", "bar") 244 | .map(name => spark.table(s"test.$name").select('pk, concat('name, 'value.cast(StringType)) as 'new_value)) 245 | .reduce((a, b) => a.union(b)) 246 | .write 247 | .parquet(outputPath.toString) 248 | 249 | val expected = Report( 250 | Metadata("test"), 251 | FsOutput(s"file:$outputPath", "Parquet"), 252 | Map( 253 | "pk" -> List( 254 | HiveInput( 255 | "test.foo", 256 | Set(Field("pk", How.PROJECTION))), 257 | HiveInput( 258 | "test.bar", 259 | Set(Field("pk", How.PROJECTION))) 260 | ), 261 | "new_value" -> List( 262 | HiveInput( 263 | "test.foo", 264 | Set(Field("name", How.PROJECTION), Field("value", How.PROJECTION))), 265 | HiveInput( 266 | "test.bar", 267 | Set(Field("name", How.PROJECTION), Field("value", How.PROJECTION))) 268 | ))) 269 | 270 | reporter.getReports() should contain theSameElementsInOrderAs List(expected) 271 | } 272 | } 273 | -------------------------------------------------------------------------------- /src/test/scala/org/chojin/spark/lineage/reporter/DynamodbReporterSpec.scala: -------------------------------------------------------------------------------- 1 | package org.chojin.spark.lineage.reporter 2 | 3 | import scala.collection.JavaConversions._ 4 | import com.amazonaws.services.dynamodbv2.AmazonDynamoDB 5 | import com.amazonaws.services.dynamodbv2.model.{AttributeValue, PutItemRequest} 6 | import org.chojin.spark.lineage.inputs.{Field, HiveInput, How} 7 | import org.chojin.spark.lineage.outputs.FsOutput 8 | import org.chojin.spark.lineage.report.{Metadata, Report} 9 | import org.mockito.{ArgumentMatchersSugar, MockitoSugar} 10 | import org.mockito.captor.ArgCaptor 11 | import org.scalatest.{WordSpec, Matchers => ScalaTestMatchers} 12 | 13 | class DynamodbReporterSpec extends WordSpec with MockitoSugar with ScalaTestMatchers with ArgumentMatchersSugar { 14 | val report = Report( 15 | Metadata("my-app"), 16 | FsOutput( 17 | path = "s3:///bucket/path/to/data", 18 | format = "parquet"), 19 | Map( 20 | "one" -> List( 21 | HiveInput( 22 | name = "db.table1", 23 | fields = Set( 24 | Field(name = "pk", how = How.JOIN), 25 | Field(name = "one", how = How.PROJECTION))), 26 | HiveInput( 27 | name = "db.table2", 28 | fields = Set( 29 | Field(name = "pk", how = How.JOIN)))), 30 | "two" -> List( 31 | HiveInput( 32 | name = "db.table1", 33 | fields = Set( 34 | Field(name = "pk", how = How.JOIN))), 35 | HiveInput( 36 | name = "db.table2", 37 | fields = Set( 38 | Field(name = "pk", how = How.JOIN), 39 | Field(name = "two", how = How.PROJECTION)))) 40 | ) 41 | ) 42 | 43 | "report" should { 44 | "put a dynamo record" in { 45 | val dynamo = mock[AmazonDynamoDB] 46 | val reporter = DynamodbReporter( 47 | table = "mytable", 48 | region = None, 49 | compression = None, 50 | _client = Some(dynamo)) 51 | 52 | reporter.report(report) 53 | 54 | val captor = ArgCaptor[PutItemRequest] 55 | verify(dynamo).putItem(captor) 56 | 57 | captor.value.getTableName shouldEqual "mytable" 58 | 59 | val item = captor.value.getItem 60 | 61 | item("outputKey") shouldEqual new AttributeValue().withS( 62 | s"fs-${report.output.asInstanceOf[FsOutput].path}") 63 | 64 | item("metadata") shouldEqual new AttributeValue().withM( 65 | Map("appName" -> new AttributeValue().withS("my-app"))) 66 | 67 | item("output") shouldEqual new AttributeValue().withM( 68 | Map( 69 | "type" -> new AttributeValue().withS("fs"), 70 | "format" -> new AttributeValue().withS("parquet"), 71 | "path" -> new AttributeValue().withS(report.output.asInstanceOf[FsOutput].path) 72 | )) 73 | 74 | item("fields") shouldEqual new AttributeValue().withM( 75 | Map( 76 | "one" -> new AttributeValue().withL(List( 77 | new AttributeValue().withM(Map( 78 | "type" -> new AttributeValue().withS("hive"), 79 | "name" -> new AttributeValue().withS("db.table1"), 80 | "fields" -> new AttributeValue().withL(List( 81 | new AttributeValue().withM(Map( 82 | "name" -> new AttributeValue().withS("pk"), 83 | "how" -> new AttributeValue().withS("join") 84 | )), 85 | new AttributeValue().withM(Map( 86 | "name" -> new AttributeValue().withS("one"), 87 | "how" -> new AttributeValue().withS("projection") 88 | )) 89 | )) 90 | )), 91 | new AttributeValue().withM(Map( 92 | "type" -> new AttributeValue().withS("hive"), 93 | "name" -> new AttributeValue().withS("db.table2"), 94 | "fields" -> new AttributeValue().withL(List( 95 | new AttributeValue().withM(Map( 96 | "name" -> new AttributeValue().withS("pk"), 97 | "how" -> new AttributeValue().withS("join") 98 | )) 99 | )) 100 | )) 101 | )), 102 | "two" -> new AttributeValue().withL(List( 103 | new AttributeValue().withM(Map( 104 | "type" -> new AttributeValue().withS("hive"), 105 | "name" -> new AttributeValue().withS("db.table1"), 106 | "fields" -> new AttributeValue().withL(List( 107 | new AttributeValue().withM(Map( 108 | "name" -> new AttributeValue().withS("pk"), 109 | "how" -> new AttributeValue().withS("join") 110 | )) 111 | )) 112 | )), 113 | new AttributeValue().withM(Map( 114 | "type" -> new AttributeValue().withS("hive"), 115 | "name" -> new AttributeValue().withS("db.table2"), 116 | "fields" -> new AttributeValue().withL(List( 117 | new AttributeValue().withM(Map( 118 | "name" -> new AttributeValue().withS("pk"), 119 | "how" -> new AttributeValue().withS("join") 120 | )), 121 | new AttributeValue().withM(Map( 122 | "name" -> new AttributeValue().withS("two"), 123 | "how" -> new AttributeValue().withS("projection") 124 | )) 125 | )) 126 | )) 127 | )) 128 | )) 129 | } 130 | } 131 | } 132 | --------------------------------------------------------------------------------