├── project ├── build.properties ├── assembly.sbt └── build.sbt ├── .gitignore ├── src ├── main │ ├── resources │ │ ├── log4j.properties │ │ └── application.conf │ └── scala │ │ └── com │ │ └── godatadriven │ │ ├── join │ │ ├── JoinType.scala │ │ ├── JoinStrategy.scala │ │ ├── NormalJoin.scala │ │ └── IterativeBroadcastJoin.scala │ │ ├── common │ │ ├── SparkUtil.scala │ │ └── Config.scala │ │ ├── generator │ │ ├── UniformDataGenerator.scala │ │ ├── SkewedDataGenerator.scala │ │ └── DataGenerator.scala │ │ └── RunBenchmark.scala └── test │ └── scala │ └── com │ └── godatadriven │ └── TestSkewedDataGenerator.scala ├── .travis.yml ├── README.md ├── run.sh └── LICENSE /project/build.properties: -------------------------------------------------------------------------------- 1 | sbt.version = 0.13.16 -------------------------------------------------------------------------------- /project/assembly.sbt: -------------------------------------------------------------------------------- 1 | addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.14.5") 2 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | *.class 2 | *.log 3 | .idea/ 4 | metastore_db/ 5 | target 6 | skewed-data/ 7 | *.parquet 8 | *.csv 9 | -------------------------------------------------------------------------------- /src/main/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | # Change this to set Spark log level 2 | log4j.logger.org.apache.spark=WARN 3 | -------------------------------------------------------------------------------- /project/build.sbt: -------------------------------------------------------------------------------- 1 | addSbtPlugin("org.scoverage" % "sbt-scoverage" % "1.3.5") 2 | 3 | addSbtPlugin("org.scoverage" % "sbt-coveralls" % "1.1.0") -------------------------------------------------------------------------------- /.travis.yml: -------------------------------------------------------------------------------- 1 | language: scala 2 | 3 | scala: 4 | - 2.11.8 5 | 6 | script: 7 | - sbt clean coverage test 8 | 9 | after_success: 10 | - sbt coverageReport coveralls 11 | -------------------------------------------------------------------------------- /src/main/scala/com/godatadriven/join/JoinType.scala: -------------------------------------------------------------------------------- 1 | package com.godatadriven.join 2 | 3 | 4 | sealed trait JoinType 5 | 6 | final case class IterativeBroadcastJoinType() extends JoinType 7 | 8 | final case class SortMergeJoinType() extends JoinType 9 | 10 | 11 | -------------------------------------------------------------------------------- /src/main/scala/com/godatadriven/join/JoinStrategy.scala: -------------------------------------------------------------------------------- 1 | package com.godatadriven.join 2 | 3 | import org.apache.spark.sql.{DataFrame, SparkSession} 4 | 5 | trait JoinStrategy { 6 | 7 | def join(spark: SparkSession, 8 | dfLarge: DataFrame, 9 | dfMedium: DataFrame): DataFrame 10 | 11 | } 12 | -------------------------------------------------------------------------------- /src/main/resources/application.conf: -------------------------------------------------------------------------------- 1 | 2 | broadcast { 3 | passes = 2 4 | } 5 | 6 | generator { 7 | partitions = 200 8 | keys = 100000 9 | multiplier = 1000 10 | uniform { 11 | mediumTableName = table_medium.parquet 12 | largeTableName = table_large.parquet 13 | } 14 | skewed { 15 | mediumTableName = table_medium.parquet 16 | largeTableName = table_large.parquet 17 | } 18 | } 19 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # The iterative broadcast join 2 | The iterative broadcast join example code. 3 | 4 | ## How to run the code 5 | 6 | First generate a dataset: 7 | ``` 8 | sbt "run generate" 9 | ``` 10 | By generating the data first in a separate job, we ensure that we use the same data and the benchmark isn't affected by the data generation process. Then you can run the benchmark: 11 | ``` 12 | sbt "run benchmark" 13 | ``` 14 | -------------------------------------------------------------------------------- /run.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | for rows in 1000000 2000000 3000000 4000000 5000000 6000000 7000000 8000000 9000000 10000000 11000000 12000000 13000000 14000000 15000000 16000000 17000000 18000000 19000000 20000000 21000000 22000000 23000000 24000000 25000000 26000000 27000000 2800000 29000000 30000000 4 | do 5 | sbt -Dgenerator.keys=$rows "run generator" 6 | time sbt -Djoin.type=std "run benchmark" 7 | time sbt -Djoin.type=itr "run benchmark" 8 | done 9 | -------------------------------------------------------------------------------- /src/main/scala/com/godatadriven/common/SparkUtil.scala: -------------------------------------------------------------------------------- 1 | package com.godatadriven 2 | 3 | import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession} 4 | 5 | object SparkUtil { 6 | 7 | def dfWrite(df: DataFrame, name: String): Unit = 8 | df 9 | .write 10 | .mode(SaveMode.Overwrite) 11 | .parquet(name) 12 | 13 | def dfRead(spark: SparkSession, name: String): DataFrame = 14 | spark 15 | .read 16 | .load(name) 17 | 18 | } 19 | -------------------------------------------------------------------------------- /src/main/scala/com/godatadriven/join/NormalJoin.scala: -------------------------------------------------------------------------------- 1 | package com.godatadriven.join 2 | 3 | import org.apache.spark.sql.{DataFrame, SparkSession} 4 | 5 | object NormalJoin extends JoinStrategy { 6 | 7 | override def join(spark: SparkSession, dfLarge: DataFrame, dfMedium: DataFrame): DataFrame = { 8 | // Explicitly disable the broadcastjoin 9 | spark.conf.set("spark.sql.autoBroadcastJoinThreshold", -1) 10 | 11 | dfLarge 12 | .join( 13 | dfMedium, 14 | Seq("key"), 15 | "left_outer" 16 | ) 17 | .select( 18 | dfLarge("key"), 19 | dfMedium("label") 20 | ) 21 | } 22 | } 23 | -------------------------------------------------------------------------------- /src/test/scala/com/godatadriven/TestSkewedDataGenerator.scala: -------------------------------------------------------------------------------- 1 | package com.godatadriven 2 | 3 | import com.godatadriven.generator.SkewedDataGenerator 4 | import org.scalatest.FunSuiteLike 5 | 6 | 7 | class TestSkewedDataGenerator extends FunSuiteLike { 8 | 9 | test("The generated sequence should be decreasing") { 10 | val listOfNumbers = SkewedDataGenerator 11 | .generateSkewedSequence(Math.pow(10, 6).toInt) 12 | .sortBy(_._1) 13 | .map(_._2) 14 | 15 | val listOfNumbersSorted = listOfNumbers.sorted.reverse 16 | 17 | val zipped = listOfNumbers.zip(listOfNumbersSorted) 18 | 19 | assert(zipped.forall(pair => pair._1 == pair._2)) 20 | } 21 | 22 | test("The data should be skewed based on the key and count") { 23 | val key = 19 24 | val count = 25 25 | val res = SkewedDataGenerator.skewDistribution(key, count) 26 | 27 | assert(res.length == count) 28 | assert(res.forall(_ == key)) 29 | } 30 | 31 | } 32 | -------------------------------------------------------------------------------- /src/main/scala/com/godatadriven/common/Config.scala: -------------------------------------------------------------------------------- 1 | package com.godatadriven.common 2 | 3 | import com.typesafe.config.ConfigFactory 4 | 5 | object Config { 6 | private val conf = ConfigFactory.load 7 | 8 | var numberOfBroadcastPasses: Int = conf.getInt("broadcast.passes") 9 | 10 | var broadcastIterationTableName: String = "tmp_broadcast_table.parquet" 11 | 12 | // The number of partitions 13 | var numberOfPartitions: Int = conf.getInt("generator.partitions") 14 | 15 | // The number of rows 16 | var numberOfKeys: Int = conf.getInt("generator.keys") 17 | 18 | // The number of times the keys get duplicated, 19 | // This controls the skewness 20 | var keysMultiplier: Int = conf.getInt("generator.multiplier") 21 | 22 | def getMediumTableName(generatorType: String): String = { 23 | conf.getString(s"generator.$generatorType.mediumTableName") 24 | } 25 | 26 | def getLargeTableName(generatorType: String): String = { 27 | conf.getString(s"generator.$generatorType.largeTableName") 28 | } 29 | } -------------------------------------------------------------------------------- /src/main/scala/com/godatadriven/generator/UniformDataGenerator.scala: -------------------------------------------------------------------------------- 1 | package com.godatadriven.generator 2 | 3 | import com.godatadriven.common.Config 4 | import org.apache.spark.sql.{SaveMode, SparkSession} 5 | 6 | import scala.util.Random 7 | 8 | object UniformDataGenerator extends DataGenerator { 9 | 10 | 11 | def buildTestset(spark: SparkSession, 12 | numberOfKeys: Int = Config.numberOfKeys, 13 | keysMultiplier: Int = Config.keysMultiplier, 14 | numberOfPartitions: Int = Config.numberOfPartitions): Unit = { 15 | 16 | import spark.implicits._ 17 | 18 | val numRows = numberOfRows(numberOfKeys, keysMultiplier) 19 | 20 | println(s"Generating $numRows rows") 21 | 22 | val df = spark 23 | .range(keysMultiplier) 24 | .repartition(numberOfPartitions) 25 | .mapPartitions(rows => { 26 | val r = new Random() 27 | val count = numRows / keysMultiplier 28 | rows.map(_ => (0 until count.toInt) 29 | .map(_ => r.nextInt(numberOfKeys))).flatten 30 | }) 31 | .map(Key) 32 | .repartition(numberOfPartitions) 33 | 34 | assert(df.count() == numberOfRows()) 35 | 36 | df 37 | .write 38 | .mode(SaveMode.Overwrite) 39 | .save(Config.getLargeTableName("uniform")) 40 | 41 | createMediumTable(spark, Config.getMediumTableName("uniform"), numberOfPartitions) 42 | } 43 | 44 | def getName: String = "UniformDataGenerator" 45 | 46 | def getMediumTableName: String = Config.getMediumTableName("uniform") 47 | 48 | def getLargeTableName: String = Config.getLargeTableName("uniform") 49 | 50 | case class Key(key: Int) 51 | 52 | case class KeyLabel(key: Int, label: String, pass: Int) 53 | 54 | } 55 | -------------------------------------------------------------------------------- /src/main/scala/com/godatadriven/join/IterativeBroadcastJoin.scala: -------------------------------------------------------------------------------- 1 | package com.godatadriven.join 2 | 3 | import com.godatadriven.SparkUtil 4 | import com.godatadriven.common.Config 5 | import org.apache.spark.sql.functions._ 6 | import org.apache.spark.sql.{DataFrame, SparkSession} 7 | 8 | import scala.annotation.tailrec 9 | 10 | object IterativeBroadcastJoin extends JoinStrategy { 11 | 12 | @tailrec 13 | private def iterativeBroadcastJoin(spark: SparkSession, 14 | result: DataFrame, 15 | broadcast: DataFrame, 16 | iteration: Int = 0): DataFrame = 17 | if (iteration < Config.numberOfBroadcastPasses) { 18 | val tableName = s"tmp_broadcast_table_itr_$iteration.parquet" 19 | 20 | val out = result.join( 21 | broadcast.filter(col("pass") === lit(iteration)), 22 | Seq("key"), 23 | "left_outer" 24 | ).select( 25 | result("key"), 26 | 27 | // Join in the label 28 | coalesce( 29 | result("label"), 30 | broadcast("label") 31 | ).as("label") 32 | ) 33 | 34 | SparkUtil.dfWrite(out, tableName) 35 | 36 | iterativeBroadcastJoin( 37 | spark, 38 | SparkUtil.dfRead(spark, tableName), 39 | broadcast, 40 | iteration + 1 41 | ) 42 | } else result 43 | 44 | override def join(spark: SparkSession, 45 | dfLarge: DataFrame, 46 | dfMedium: DataFrame): DataFrame = { 47 | broadcast(dfMedium) 48 | iterativeBroadcastJoin( 49 | spark, 50 | dfLarge 51 | .select("key") 52 | .withColumn("label", lit(null)), 53 | dfMedium 54 | ) 55 | } 56 | 57 | } 58 | -------------------------------------------------------------------------------- /src/main/scala/com/godatadriven/generator/SkewedDataGenerator.scala: -------------------------------------------------------------------------------- 1 | package com.godatadriven.generator 2 | 3 | import com.godatadriven.common.Config 4 | import org.apache.spark.sql.{SaveMode, SparkSession} 5 | 6 | object SkewedDataGenerator extends DataGenerator { 7 | 8 | def buildTestset(spark: SparkSession, 9 | numberOfKeys: Int = Config.numberOfKeys, 10 | keysMultiplier: Int = Config.keysMultiplier, 11 | numberOfPartitions: Int = Config.numberOfPartitions): Unit = { 12 | 13 | import spark.implicits._ 14 | 15 | println(s"Generating ${numberOfRows(numberOfKeys, keysMultiplier)} rows") 16 | 17 | val df = spark 18 | .sparkContext 19 | .parallelize(generateSkewedSequence(numberOfKeys), numberOfPartitions) 20 | .flatMap(list => (0 until keysMultiplier).map(_ => list)) 21 | .repartition(numberOfPartitions) 22 | .flatMap(pair => skewDistribution(pair._1, pair._2)) 23 | .toDS() 24 | .map(Key) 25 | .repartition(numberOfPartitions) 26 | 27 | assert(df.count() == numberOfRows()) 28 | 29 | df 30 | .write 31 | .mode(SaveMode.Overwrite) 32 | .save(Config.getLargeTableName("skewed")) 33 | 34 | createMediumTable(spark, Config.getMediumTableName("skewed"), numberOfPartitions) 35 | } 36 | 37 | /** 38 | * Will generate a sequence of the input sample 39 | * 40 | * @param key The sample and the 41 | * @param count count the number of repetitions 42 | * @return 43 | */ 44 | def skewDistribution(key: Int, count: Int): Seq[Int] = Seq.fill(count)(key) 45 | 46 | def getName: String = "SkewedDataGenerator" 47 | 48 | def getMediumTableName: String = Config.getMediumTableName("skewed") 49 | 50 | def getLargeTableName: String = Config.getLargeTableName("skewed") 51 | 52 | case class Key(key: Int) 53 | 54 | case class KeyLabel(key: Int, label: String, pass: Int) 55 | 56 | } 57 | -------------------------------------------------------------------------------- /src/main/scala/com/godatadriven/generator/DataGenerator.scala: -------------------------------------------------------------------------------- 1 | package com.godatadriven.generator 2 | 3 | import com.godatadriven.common.Config 4 | import com.godatadriven.generator.UniformDataGenerator.KeyLabel 5 | import org.apache.spark.sql.{SaveMode, SparkSession} 6 | 7 | import scala.util.Random 8 | 9 | trait DataGenerator { 10 | 11 | 12 | def numberOfRows(numberOfKeys: Int = Config.numberOfKeys, 13 | keysMultiplier: Int = Config.keysMultiplier): Long = 14 | generateSkewedSequence(numberOfKeys).map(_._2).sum * keysMultiplier.toLong 15 | 16 | /** 17 | * Generates a sequence of numbers, for example num = 22 would generate: 18 | * Array(22, 10, 6, 4, 3, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1) 19 | * 20 | * @param numberOfKeys number of elements in the sequence 21 | * @return tuple of (key, num), where key is the unique key and num is the times it will be repeated 22 | */ 23 | def generateSkewedSequence(numberOfKeys: Int): List[(Int, Int)] = 24 | (0 to numberOfKeys).par.map(i => 25 | (i, Math.ceil( 26 | (numberOfKeys.toDouble - i.toDouble) / (i.toDouble + 1.0) 27 | ).toInt) 28 | ).toList 29 | 30 | def createMediumTable(spark: SparkSession, tableName: String, numberOfPartitions: Int): Unit = { 31 | 32 | import spark.implicits._ 33 | 34 | val df = spark 35 | .read 36 | .parquet("table_large.parquet") 37 | .as[Int] 38 | .distinct() 39 | .mapPartitions(rows => { 40 | val r = new Random() 41 | rows.map(key => 42 | KeyLabel( 43 | key, 44 | s"Description for entry $key, that can be anything", 45 | // Already preallocate the pass of the broadcast iteration here 46 | Math.floor(r.nextDouble() * Config.numberOfBroadcastPasses).toInt 47 | ) 48 | ) 49 | }) 50 | .repartition(numberOfPartitions) 51 | 52 | assert(df.count() == Config.numberOfKeys) 53 | 54 | df 55 | .write 56 | .mode(SaveMode.Overwrite) 57 | .parquet(tableName) 58 | } 59 | 60 | def buildTestset(spark: SparkSession, 61 | numberOfKeys: Int = Config.numberOfKeys, 62 | keysMultiplier: Int = Config.keysMultiplier, 63 | numberOfPartitions: Int = Config.numberOfPartitions): Unit 64 | 65 | def getName: String 66 | 67 | def getMediumTableName: String 68 | 69 | def getLargeTableName: String 70 | 71 | } 72 | -------------------------------------------------------------------------------- /src/main/scala/com/godatadriven/RunBenchmark.scala: -------------------------------------------------------------------------------- 1 | package com.godatadriven 2 | 3 | import com.godatadriven.common.Config 4 | import com.godatadriven.generator.{DataGenerator, SkewedDataGenerator, UniformDataGenerator} 5 | import com.godatadriven.join._ 6 | import org.apache.spark.sql.{SaveMode, SparkSession} 7 | 8 | 9 | object RunBenchmark extends App { 10 | 11 | def time[R](block: => R): R = { 12 | val t0 = System.nanoTime() 13 | val result = block // call-by-name 14 | val t1 = System.nanoTime() 15 | println("Elapsed time: " + (t1 - t0) / 1000 / 1000 / 1000 + " sec") 16 | result 17 | } 18 | 19 | 20 | def runTest(generator: DataGenerator, 21 | joinType: JoinType, 22 | tableNameOutput: String) { 23 | 24 | val rows = generator.numberOfRows() 25 | 26 | val name = s"${generator.getName}: $joinType, passes=${Config.numberOfBroadcastPasses}, keys=${Config.numberOfKeys}, multiplier=${Config.keysMultiplier}, rows=$rows" 27 | 28 | println(name) 29 | 30 | 31 | val spark = getSparkSession(name) 32 | 33 | time { 34 | 35 | val out = joinType match { 36 | case _: SortMergeJoinType => NormalJoin.join( 37 | spark, 38 | spark 39 | .read 40 | .load(generator.getLargeTableName), 41 | spark 42 | .read 43 | .load(generator.getMediumTableName) 44 | ) 45 | case _: IterativeBroadcastJoinType => IterativeBroadcastJoin.join( 46 | spark, 47 | spark 48 | .read 49 | .load(generator.getLargeTableName), 50 | spark 51 | .read 52 | .load(generator.getMediumTableName) 53 | ) 54 | } 55 | 56 | out.write 57 | .mode(SaveMode.Overwrite) 58 | .parquet(tableNameOutput) 59 | } 60 | 61 | spark.stop() 62 | } 63 | 64 | 65 | def runBenchmark(dataGenerator: DataGenerator, 66 | iterations: Int = 8, 67 | outputTable: String = "result.parquet"): Unit = { 68 | val originalMultiplier = Config.keysMultiplier 69 | 70 | (0 to iterations) 71 | .map(step => originalMultiplier + (step * originalMultiplier)) 72 | .foreach(multiplier => { 73 | 74 | val keys = Config.numberOfKeys 75 | Config.keysMultiplier = multiplier 76 | 77 | // Generate uniform data and benchmark 78 | val rows = dataGenerator.numberOfRows() 79 | 80 | val spark = getSparkSession(s"${dataGenerator.getName}: Generate dataset with $keys keys, $rows rows") 81 | dataGenerator.buildTestset( 82 | spark, 83 | keysMultiplier = multiplier 84 | ) 85 | spark.stop() 86 | 87 | Config.numberOfBroadcastPasses = 2 88 | 89 | runTest( 90 | dataGenerator, 91 | new IterativeBroadcastJoinType, 92 | outputTable 93 | ) 94 | 95 | Config.numberOfBroadcastPasses = 3 96 | 97 | runTest( 98 | dataGenerator, 99 | new IterativeBroadcastJoinType, 100 | outputTable 101 | ) 102 | 103 | runTest( 104 | dataGenerator, 105 | new SortMergeJoinType, 106 | outputTable 107 | ) 108 | }) 109 | 110 | // Reset global Config 111 | Config.keysMultiplier = originalMultiplier 112 | } 113 | 114 | def getSparkSession(appName: String = "Spark Application"): SparkSession = { 115 | val spark = SparkSession 116 | .builder 117 | .appName(appName) 118 | .getOrCreate() 119 | 120 | spark.conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") 121 | spark.conf.set("parquet.enable.dictionary", "false") 122 | spark.conf.set("spark.default.parallelism", Config.numberOfPartitions) 123 | spark.conf.set("spark.sql.shuffle.partitions", Config.numberOfPartitions) 124 | 125 | // Tell Spark to don't be too chatty 126 | spark.sparkContext.setLogLevel("WARN") 127 | 128 | spark 129 | } 130 | 131 | // runBenchmark(UniformDataGenerator) 132 | runBenchmark(SkewedDataGenerator) 133 | } 134 | -------------------------------------------------------------------------------- /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 | --------------------------------------------------------------------------------