├── .gitignore ├── LICENSE.txt ├── README.md ├── pom.xml └── src └── main ├── resources └── log4j.properties └── scala └── com └── kwartile └── lib └── cc ├── CCDriver.scala ├── CCGraphXDriver.scala ├── CliquesGenerator.scala └── ConnectedComponent.scala /.gitignore: -------------------------------------------------------------------------------- 1 | .metadata 2 | .cache-main 3 | .classpath 4 | .project 5 | .settings 6 | *.class 7 | *.log 8 | .idea/ 9 | target/ 10 | .DS_Store 11 | connected-component.iml 12 | -------------------------------------------------------------------------------- /LICENSE.txt: -------------------------------------------------------------------------------- 1 | MIT License 2 | 3 | Copyright (c) 2018 Kwartile, Inc. 4 | 5 | Permission is hereby granted, free of charge, to any person obtaining a copy 6 | of this software and associated documentation files (the "Software"), to deal 7 | in the Software without restriction, including without limitation the rights 8 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 9 | copies of the Software, and to permit persons to whom the Software is 10 | furnished to do so, subject to the following conditions: 11 | 12 | The above copyright notice and this permission notice shall be included in all 13 | copies or substantial portions of the Software. 14 | 15 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 16 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 17 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 18 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 19 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 20 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE 21 | SOFTWARE. 22 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | ## Connected component using Map-reduce on Apache Spark 2 | 3 | #### Description 4 | Computing Connected Components of a graph is a well studied problem in Graph Theory and there have been many state of the art algorithms that perform pretty well in a single machine environment. But many of these algorithms perform poorly when we apply them in a distributed setting with hundreds of billions of nodes and edges. Our first choice was to use, GraphX, the graph computing engine that comes with Apache Spark. Although, GraphX implementation of the algorithm works reasonably well on smaller graphs (we tested up to ~10 million nodes and ~100 million edges), but its performance quickly degraded as we tried to scale to higher numbers. 5 | 6 | We implemented connected component algorithm described in the paper [Connected Components in Map Reduce and Beyond](http://dl.acm.org/citation.cfm?id=2670997). We liked its approach for two reasons - (1) the algorithm was well suited for our technical stack (Apache Spark on HDFS) (2) more over, other than typical computational complexity, the algorithm also took communication complexity and data skew into account. The proposed algorithm is iterative but in practice with our dataset and scale, it was able to converge pretty fast with less than ten iterations. 7 | 8 | --- 9 | #### Implementation 10 | We implemented the algorithm on Apache Spark on HDFS using Scala. We also provide a sample graph generator and a driver program. You can tune the parameters of this generator to change the characteristics of the generated graph. The generator saves the generated graph on HDFS. You can use the driver program to read the generated graph and run the algorithm. The results of the algorithm is also stored on HDFS. Alternatively, you can call directly call the API to run the algorithm. 11 | 12 | In the implementation, we represent a node by a unique Long number. Input to the algorithm is a List of Cliques. A Clique is a list of nodes that are connected together. For example, the cliques can be: 13 | ``` 14 | 1: List(1L, 2L, 3L) 15 | 2: List(3L, 4L) 16 | 3: List(1L, 5L) 17 | 4: List(2L) 18 | 5: List(6L) 19 | 6: List(7L, 8L) 20 | 7: List(6L, 8L) 21 | 8: List(9L) 22 | ``` 23 | 24 | In this case, we have 8 cliques as the input. As you can see that cliques 1, 2, 3, 4 form one connected component, cliques 5, 6, 7 form the second connected component, and clique 8 forms the third connected component. 25 | 26 | The main API to drive the algorithm is 27 | ``` 28 | ConnectedComponent.run(cliques:RDD[List[Long]], maxIterationCount: Int): (RDD([Long, Long)], Boolean, Int) 29 | ``` 30 | The API expects you to provide RDD of cliques and maximum number of iterations. It returns ```RDD[(Long, Long)]``` i.e. a RDD of 2-tuple. The second element of the tuple is the minimum node in a connected component and the first element is another node in the same component. 31 | 32 | We first build a List of nodePairs (```RDD[(Long, Long)]```), from the list of given cliques. We then apply the Large Star and Small Star operations on the list of node pairs. 33 | 34 | We implemented the Large Star algorithm as follows: 35 | ``` 36 | LargeStar 37 | Input: List of nodePair(a, b) 38 | Output: List of new nodePairs and change in totalConnectivityChangeCount 39 | 40 | 1: For every nodePair(a, b) emit nodePair(a, b) and nodePair(b, a). We call the first element of the tuple-2 as self and the second element as its neighbor 41 | 2: Reduce on self to get a list of its neighbors. 42 | 3: For every self, apply Large Star operation on its neighbors. The operation results in a list of new nodePairs. 43 | 4: Count the change in connectivity, connectivtyChangeCount, by subtracting the length of the list of neighbors in step 3 from the new list of neighbors in step 4 44 | 5: Sum this change for every self to get total change in connectivity, totalConnectivityChangeCount 45 | 6: Return the list of new nodePairs and totalConnectivityChangeCount 46 | ``` 47 | We implemented the Small Star algorithm as follows: 48 | ``` 49 | SmallStar 50 | Input: List of nodePair(a, b) 51 | Output: List of new nodePairs and change in totalConnectivityChangeCount 52 | 53 | 1: For every nodePair(a, b) emit nodePair(a, b) if a > b else emit nodePair(b, a) 54 | 2: Rest of the steps are same as that of Large Star. 55 | ``` 56 | We call the Large Star and Small Star alternatively till the sum of the ```totalConnectivityChangeCount``` becomes zero. The outputs are RDD of nodePairs, a flag to indicate whether the algorithm converged within the given number of iterations, and count of iterations it took the algorithm to converge. In our experiments with various datasets, we observed that the algorithm was able to converge within 5 iterations. 57 | 58 | The second element of the resultant nodePair is the minimum node in the connected component. To get all the nodes in a components, you will need to run reduce operation with second element as the key. For example, to get all the connected components, you may use the following: 59 | ``` 60 | val (cc, didConverge, iterCount) = ConnectedComponent.run(cliques, maxIterCount) 61 | If (didConverge) { 62 | val allComponents = cc.map(x => { 63 | val minNode = x._2 64 | val otherNode = x._1 65 | (minNode, List(otherNode)) 66 | }).reduceByKey((a, b) => b ::: a) 67 | } 68 | ``` 69 | 70 | --- 71 | #### Conclusion 72 | We tested our implementation on various data sizes - scaling up to ~100 billion nodes and ~800 billion edges. In all the cases, the algorithm converged in no more than 6 iterations. We indeed had to to try various Spark related configurations, including executor memory size, driver memory size, yarn memory overhead, network timeout, and number of partitions to successfully run the implementation. 73 | 74 | We would love to hear your feedback. Please drop us a note at labs@kwartile.com. 75 | -------------------------------------------------------------------------------- /pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 4.0.0 6 | 7 | com.kwartile.lib 8 | connected-component 9 | 1.0-SNAPSHOT 10 | 11 | 2.11.11 12 | 13 | 14 | 15 | 16 | maven-repo 17 | Maven Repository 18 | http://repo.maven.apache.org/maven2 19 | 20 | true 21 | 22 | 23 | false 24 | 25 | 26 | 27 | cloudera-repo 28 | Cloudera Repository 29 | https://repository.cloudera.com/artifactory/cloudera-repos 30 | 31 | 32 | 33 | 34 | org.scala-lang 35 | scala-library 36 | ${scala.version} 37 | provided 38 | 39 | 40 | org.apache.spark 41 | spark-core_2.11 42 | 2.1.0.cloudera1 43 | compile 44 | 45 | 46 | 47 | org.apache.spark 48 | spark-sql_2.11 49 | 2.1.0.cloudera1 50 | compile 51 | 52 | 53 | 54 | org.apache.spark 55 | spark-graphx_2.11 56 | 2.1.0.cloudera1 57 | compile 58 | 59 | 60 | junit 61 | junit 62 | 3.8.1 63 | test 64 | 65 | 66 | com.github.scopt 67 | scopt_2.10 68 | 3.5.0 69 | 70 | 71 | 72 | src/main/scala 73 | src/test/scala 74 | 75 | 76 | net.alchim31.maven 77 | scala-maven-plugin 78 | 3.2.0 79 | 80 | 81 | 82 | compile 83 | 84 | 85 | 86 | 87 | -dependencyfile 88 | ${project.build.directory}/.scala_dependencies 89 | 90 | 91 | 92 | 93 | 94 | 95 | org.apache.maven.plugins 96 | maven-surefire-plugin 97 | 2.18.1 98 | 99 | false 100 | true 101 | 102 | 103 | 104 | **/*Test.* 105 | **/*Suite.* 106 | 107 | 108 | 109 | 110 | maven-assembly-plugin 111 | 2.4.1 112 | 113 | 114 | jar-with-dependencies 115 | 116 | 117 | 118 | 119 | make-assembly 120 | package 121 | 122 | single 123 | 124 | 125 | 126 | 127 | 128 | 129 | 130 | 131 | -------------------------------------------------------------------------------- /src/main/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | log4j.rootLogger=${root.logger} 2 | root.logger=INFO,console 3 | log4j.appender.console=org.apache.log4j.ConsoleAppender 4 | log4j.appender.console.target=System.err 5 | log4j.appender.console.layout=org.apache.log4j.PatternLayout 6 | log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{2}: %m%n 7 | shell.log.level=ERROR 8 | log4j.logger.org.eclipse.jetty=WARN 9 | log4j.logger.org.spark-project.jetty=INFO 10 | log4j.logger.org.spark-project.jetty.util.component.AbstractLifeCycle=ERROR 11 | log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO 12 | log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO 13 | log4j.logger.org.apache.parquet=ERROR 14 | log4j.logger.parquet=ERROR 15 | log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=FATAL 16 | log4j.logger.org.apache.hadoop.hive.ql.exec.FunctionRegistry=ERROR 17 | log4j.logger.org.apache.spark.repl.Main=${shell.log.level} 18 | log4j.logger.org.apache.spark.api.python.PythonGatewayServer=${shell.log.level} -------------------------------------------------------------------------------- /src/main/scala/com/kwartile/lib/cc/CCDriver.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (c) 2017 Kwartile, Inc., http://www.kwartile.com 3 | * Permission is hereby granted, free of charge, to any person obtaining a copy 4 | * of this software and associated documentation files (the "Software"), to deal 5 | * in the Software without restriction, including without limitation the rights 6 | * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | * copies of the Software, and to permit persons to whom the Software is 8 | * furnished to do so, subject to the following conditions: 9 | * 10 | * The above copyright notice and this permission notice shall be included in all 11 | * copies or substantial portions of the Software. 12 | * 13 | * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE 19 | * SOFTWARE. 20 | */ 21 | 22 | package com.kwartile.lib.cc 23 | 24 | import org.apache.spark.{RangePartitioner, SparkConf, SparkContext} 25 | 26 | /** 27 | * Driver program to run Connected Component. 28 | */ 29 | 30 | object CCDriver extends Serializable { 31 | 32 | /** 33 | * 34 | * @param args name of the file containing cliques. One line in the file represents one clique. 35 | * e.g.: 36 | * 1 9 4 5 37 | * 2 234 23 1 38 | * 6 3 39 | */ 40 | def main(args: Array[String]) = { 41 | 42 | val sparkConf = new SparkConf() 43 | .setAppName("ConnectedComponent") 44 | .setMaster("local[1]") 45 | 46 | val sc = new SparkContext(sparkConf) 47 | val cliqueFile = args(0) 48 | val cliquesRec = sc.textFile(args(0)) 49 | val cliques = cliquesRec.map(x => { 50 | val nodes = x.split("\\s+").map(y => y.toLong).toList 51 | nodes 52 | }) 53 | 54 | val (cc, didConverge, iterCount) = ConnectedComponent.run(cliques, 20) 55 | 56 | if (didConverge) { 57 | println("Converged in " + iterCount + " iterations") 58 | val cc2 = cc.map(x => { 59 | (x._2, List(x._1)) 60 | }) 61 | 62 | /** 63 | * Get all the nodes in the connected component/ 64 | * We are using a rangePartitioner because the CliquesGenerator produces data skew. 65 | */ 66 | val rangePartitioner = new RangePartitioner(cc2.getNumPartitions, cc2) 67 | val connectedComponents = cc2.reduceByKey(rangePartitioner, (a, b) => {b ::: a}) 68 | 69 | //connectedComponents.mapPartitionsWithIndex((index, iter) => { 70 | // iter.toList.map(x => (index, x._1, x._2.size)).iterator 71 | // }).collect.foreach(println) 72 | 73 | println("connected components") 74 | connectedComponents.map(x => (x._2.length).toString + " " + x._1 + " " + x._2.sorted.mkString(" ")).saveAsTextFile(cliqueFile + "_cc_out") 75 | } 76 | else { 77 | println("Max iteration reached. Could not converge") 78 | } 79 | } 80 | } 81 | -------------------------------------------------------------------------------- /src/main/scala/com/kwartile/lib/cc/CCGraphXDriver.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (c) 2017 Kwartile, Inc., http://www.kwartile.com 3 | * Permission is hereby granted, free of charge, to any person obtaining a copy 4 | * of this software and associated documentation files (the "Software"), to deal 5 | * in the Software without restriction, including without limitation the rights 6 | * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | * copies of the Software, and to permit persons to whom the Software is 8 | * furnished to do so, subject to the following conditions: 9 | * 10 | * The above copyright notice and this permission notice shall be included in all 11 | * copies or substantial portions of the Software. 12 | * 13 | * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE 19 | * SOFTWARE. 20 | */ 21 | 22 | package com.kwartile.lib.cc 23 | 24 | import org.apache.spark.graphx.{Edge, Graph} 25 | import org.apache.spark.{SparkConf, SparkContext} 26 | 27 | import scala.annotation.tailrec 28 | 29 | 30 | /** 31 | * GraphX based implementation of Connected Component. 32 | * you can use this to compare performance of the m/r based algorithm. 33 | */ 34 | object CCGraphXDriver { 35 | 36 | @tailrec 37 | private def buildEdges(node: Long, neighbors:List[Long], partialPairs: List[Edge[Int]]) : List[Edge[Int]] = { 38 | if (neighbors.length == 0) { 39 | if (partialPairs != null) 40 | List(Edge(node, node, 1)) ::: partialPairs 41 | else 42 | List(Edge(node, node, 1)) 43 | } else if (neighbors.length == 1) { 44 | val neighbor = neighbors(0) 45 | if (node > neighbor) 46 | if (partialPairs != null) List(Edge(node, neighbor, 1)) ::: partialPairs else List(Edge(node, neighbor, 1)) 47 | else 48 | if (partialPairs != null) List(Edge(neighbor, node, 1)) ::: partialPairs else List(Edge(neighbor, node, 1)) 49 | } else { 50 | val newPartialPairs = neighbors.map(neighbor => { 51 | if (node > neighbor) 52 | List(Edge(node, neighbor, 1)) 53 | else 54 | List(Edge(neighbor, node, 1)) 55 | }).flatMap(x=>x) 56 | 57 | if (partialPairs != null) 58 | buildEdges(neighbors.head, neighbors.tail, newPartialPairs ::: partialPairs) 59 | else 60 | buildEdges(neighbors.head, neighbors.tail, newPartialPairs) 61 | } 62 | } 63 | 64 | private def buildEdges(nodes:List[Long]) : List[Edge[Int]] = { 65 | buildEdges(nodes.head, nodes.tail, null.asInstanceOf[List[Edge[Int]]]) 66 | } 67 | 68 | def main(args: Array[String]) = { 69 | val sparkConf = new SparkConf().setAppName("GraphXConnectedComponent") 70 | 71 | val sc = new SparkContext(sparkConf) 72 | 73 | val cliqueFile = args(0) 74 | val cliquesRec = sc.textFile(args(0)) 75 | val cliques = cliquesRec.map(x => { 76 | val nodes = x.split("\\s+").map(y => y.toLong).toList 77 | nodes 78 | }) 79 | 80 | val edges = cliques.map(aClique => { 81 | buildEdges(aClique) 82 | }).flatMap(x=>x) 83 | 84 | val graph = Graph.fromEdges(edges, 1) 85 | val cc = graph.connectedComponents().vertices 86 | println ("Count of Connected component: " + cc.count) 87 | } 88 | } 89 | -------------------------------------------------------------------------------- /src/main/scala/com/kwartile/lib/cc/CliquesGenerator.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (c) 2017 Kwartile, Inc., http://www.kwartile.com 3 | * Permission is hereby granted, free of charge, to any person obtaining a copy 4 | * of this software and associated documentation files (the "Software"), to deal 5 | * in the Software without restriction, including without limitation the rights 6 | * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | * copies of the Software, and to permit persons to whom the Software is 8 | * furnished to do so, subject to the following conditions: 9 | * 10 | * The above copyright notice and this permission notice shall be included in all 11 | * copies or substantial portions of the Software. 12 | * 13 | * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE 19 | * SOFTWARE. 20 | */ 21 | 22 | package com.kwartile.lib.cc 23 | 24 | import org.apache.commons.cli.{BasicParser, Options} 25 | import org.apache.spark.{SparkConf, SparkContext} 26 | import org.apache.spark.rdd.RDD 27 | 28 | import scala.collection.mutable.ListBuffer 29 | import scala.util.Random 30 | 31 | /** 32 | * Generates test data. Saves the data in HDFS. 33 | * Every line of the file represents a clique (subgraph) 34 | * 35 | */ 36 | 37 | object CliquesGenerator extends Serializable { 38 | 39 | /** 40 | * Generate test data in parallel. Useful when you want to create large dataset to test 41 | * @param sc spark context 42 | * @param cliquesCount number clique counts you want to generate 43 | * @param maxNodesPerComponent maximum number of nodes per clique. 44 | * @param numPartition partition count 45 | * @return RDD of the cliques 46 | */ 47 | def generateInParallel(sc: SparkContext, cliquesCount: Long, maxNodesPerComponent: Int, numPartition: Int): 48 | (RDD[List[List[Long]]], RDD[List[List[Long]]]) = { 49 | 50 | require(cliquesCount > 1000 && cliquesCount % 1000 == 0 && maxNodesPerComponent > 0 && maxNodesPerComponent < 1000) 51 | val partitionCount = Math.min(numPartition, cliquesCount / numPartition) 52 | val numElementsPerPartition = cliquesCount / numPartition 53 | val distData = sc.parallelize(0 to partitionCount.toInt, partitionCount.toInt) 54 | .mapPartitionsWithIndex((x, _) => { 55 | val cliqueBuffer = new ListBuffer[List[Long]] 56 | val rand = new scala.util.Random(19345) 57 | val partRange = (x * numElementsPerPartition to numElementsPerPartition * x + numElementsPerPartition - 1).toList 58 | val connectedComponentsBuffer = new ListBuffer[List[Long]] 59 | partRange.map(i => { 60 | val allComponentsBuffer = new ListBuffer[List[Long]] 61 | val idRange = (1000L * i to 1000L * i + rand.nextInt(maxNodesPerComponent) + 2).toList 62 | val numComponents = 2 + rand.nextInt(Math.min(maxNodesPerComponent, 100)) 63 | val numNodesPerComponent = 2 + rand.nextInt(Math.min(maxNodesPerComponent, 20)) 64 | var connectingElement = idRange.head 65 | for (i <- 0 to numComponents - 1) { 66 | val comp = (connectingElement :: Random.shuffle(idRange).take(numNodesPerComponent)).toSet.toList 67 | cliqueBuffer += comp 68 | allComponentsBuffer += comp 69 | connectingElement = Random.shuffle(comp).head 70 | } 71 | val cc = allComponentsBuffer.toList.flatten.toList.distinct 72 | connectedComponentsBuffer += cc 73 | }) 74 | List((cliqueBuffer.toList, connectedComponentsBuffer.toList)).iterator 75 | 76 | }).cache 77 | val cliques = distData.map(x => x._1) 78 | val cc = distData.map(x => x._2) 79 | distData.unpersist(false) 80 | (cliques, cc) 81 | } 82 | 83 | /** 84 | * Generate test data 85 | * @param sc spark context 86 | * @param cliquesCount number clique counts you want to generate 87 | * @param maxNodesPerComponent maximum number of nodes per clique. 88 | * @return 89 | */ 90 | def generate(sc: SparkContext, cliquesCount: Int, maxNodesPerComponent: Int): (List[List[Long]], List[List[Long]]) = { 91 | 92 | require(cliquesCount > 0 && maxNodesPerComponent > 0 && maxNodesPerComponent < 1000) 93 | 94 | val cliqueBuffer = new ListBuffer[List[Long]] 95 | val rand = new scala.util.Random(19345) 96 | val connectedComponentsBuffer = new ListBuffer[List[Long]] 97 | for (i <- 0 to cliquesCount) { 98 | val allComponentsBuffer = new ListBuffer[List[Long]] 99 | val idRange = (1000L * i to 1000L * i + rand.nextInt(maxNodesPerComponent) + 2).toList 100 | val numComponents = 2 + rand.nextInt(Math.min(maxNodesPerComponent, 100)) 101 | val numNodesPerComponent = 2 + rand.nextInt(Math.min(maxNodesPerComponent, 20)) 102 | var connectingElement = idRange.head 103 | for (i <- 0 to numComponents-1) { 104 | val comp = (connectingElement :: Random.shuffle(idRange).take(numNodesPerComponent)).toSet.toList 105 | cliqueBuffer += comp 106 | allComponentsBuffer += comp 107 | connectingElement = Random.shuffle(comp).head 108 | } 109 | val cc = allComponentsBuffer.toList.flatten.toList.distinct 110 | connectedComponentsBuffer += cc 111 | } 112 | (cliqueBuffer.toList, connectedComponentsBuffer.toList) 113 | } 114 | 115 | def main(args: Array[String]) = { 116 | 117 | val argsOptions = new Options 118 | argsOptions.addOption("cliquesCount", "cliquesCount", true, " Count of cliques") 119 | argsOptions.addOption("maxNodesPerComponent", "maxNodesPerComponent", true, " maximum number of nodes per component") 120 | argsOptions.addOption("outputFile", "outputFile", true, " name of the output file") 121 | argsOptions.addOption("numPartition", "numPartition", true, " number of partitions") 122 | 123 | val clParser = new BasicParser 124 | val clArgs = clParser.parse(argsOptions, args) 125 | val cliquesCount = clArgs.getOptionValue("cliquesCount").toLong 126 | val maxNodesPerComponent = clArgs.getOptionValue("maxNodesPerComponent").toInt 127 | val outputFile = clArgs.getOptionValue("outputFile") 128 | val numPartition = clArgs.getOptionValue("numPartition").toInt 129 | 130 | val sparkConf = new SparkConf().setAppName("CliqueGenerator") 131 | 132 | val sc = new SparkContext(sparkConf) 133 | val (cliques, cc) = CliquesGenerator.generateInParallel(sc, cliquesCount, maxNodesPerComponent, numPartition) 134 | 135 | val cliquesStr = cliques.map(x => x.map(i => i.mkString(" "))).flatMap(x=>x) 136 | cliquesStr.saveAsTextFile(outputFile) 137 | val ccStr = cc.map(x => x.map(i => i.mkString(" "))).flatMap(x=>x) 138 | ccStr.saveAsTextFile(outputFile + "_cc") 139 | } 140 | } 141 | -------------------------------------------------------------------------------- /src/main/scala/com/kwartile/lib/cc/ConnectedComponent.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright (c) 2017 Kwartile, Inc., http://www.kwartile.com 3 | * Permission is hereby granted, free of charge, to any person obtaining a copy 4 | * of this software and associated documentation files (the "Software"), to deal 5 | * in the Software without restriction, including without limitation the rights 6 | * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | * copies of the Software, and to permit persons to whom the Software is 8 | * furnished to do so, subject to the following conditions: 9 | * 10 | * The above copyright notice and this permission notice shall be included in all 11 | * copies or substantial portions of the Software. 12 | * 13 | * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE 19 | * SOFTWARE. 20 | */ 21 | 22 | /** 23 | * Map-reduce implementation of Connected Component 24 | * Given lists of subgraphs, returns all the nodes that are connected. 25 | */ 26 | 27 | package com.kwartile.lib.cc 28 | 29 | import org.apache.spark.rdd.RDD 30 | import org.apache.spark.storage.StorageLevel 31 | import scala.annotation.tailrec 32 | import scala.collection.mutable 33 | 34 | 35 | object ConnectedComponent extends Serializable { 36 | 37 | /** 38 | * Applies Small Star operation on RDD of nodePairs 39 | * @param nodePairs on which to apply Small Star operations 40 | * @return new nodePairs after the operation and conncectivy change count 41 | */ 42 | private def smallStar(nodePairs: RDD[(Long, Long)]): (RDD[(Long, Long)], Int) = { 43 | 44 | /** 45 | * generate RDD of (self, List(neighbors)) where self > neighbors 46 | * E.g.: nodePairs (1, 4), (6, 1), (3, 2), (6, 5) 47 | * will result into (4, List(1)), (6, List(1)), (3, List(2)), (6, List(5)) 48 | */ 49 | val neighbors = nodePairs.map(x => { 50 | val (self, neighbor) = (x._1, x._2) 51 | if (self > neighbor) 52 | (self, neighbor) 53 | else 54 | (neighbor, self) 55 | }) 56 | 57 | /** 58 | * reduce on self to get list of all its neighbors. 59 | * E.g: (4, List(1)), (6, List(1)), (3, List(2)), (6, List(5)) 60 | * will result into (4, List(1)), (6, List(1, 5)), (3, List(2)) 61 | * Note: 62 | * (1) you may need to tweak number of partitions. 63 | * (2) also, watch out for data skew. In that case, consider using rangePartitioner 64 | */ 65 | val empty = mutable.HashSet[Long]() 66 | val allNeighbors = neighbors.aggregateByKey(empty)( 67 | (lb, v) => lb += v, 68 | (lb1, lb2) => lb1 ++ lb2 69 | ) 70 | 71 | /** 72 | * Apply Small Star operation on (self, List(neighbor)) to get newNodePairs and count the change in connectivity 73 | */ 74 | 75 | val newNodePairsWithChangeCount = allNeighbors.map(x => { 76 | val self = x._1 77 | val neighbors = x._2.toList 78 | val minNode = argMin(self :: neighbors) 79 | val newNodePairs = (self :: neighbors).map(neighbor => { 80 | (neighbor, minNode) 81 | }).filter(x => { 82 | val neighbor = x._1 83 | val minNode = x._2 84 | (neighbor <= self && neighbor != minNode) || (self == neighbor) 85 | }) 86 | val uniqueNewNodePairs = newNodePairs.toSet.toList 87 | 88 | /** 89 | * We count the change by taking a diff of the new node pairs with the old node pairs 90 | */ 91 | val connectivityChangeCount = (uniqueNewNodePairs diff neighbors.map((self, _))).length 92 | (uniqueNewNodePairs, connectivityChangeCount) 93 | }).persist(StorageLevel.MEMORY_AND_DISK_SER) 94 | 95 | /** 96 | * Sum all the changeCounts 97 | */ 98 | val totalConnectivityCountChange = newNodePairsWithChangeCount.mapPartitions(iter => { 99 | val (v, l) = iter.toSeq.unzip 100 | val sum = l.foldLeft(0)(_ + _) 101 | Iterator(sum) 102 | }).sum.toInt 103 | 104 | val newNodePairs = newNodePairsWithChangeCount.map(x => x._1).flatMap(x => x) 105 | newNodePairsWithChangeCount.unpersist(false) 106 | (newNodePairs, totalConnectivityCountChange) 107 | } 108 | 109 | /** 110 | * Apply Large Star operation on a RDD of nodePairs 111 | * @param nodePairs on which to apply Large Star operations 112 | * @return new nodePairs after the operation and conncectivy change count 113 | */ 114 | private def largeStar(nodePairs: RDD[(Long, Long)]): (RDD[(Long, Long)], Int) = { 115 | 116 | /** 117 | * generate RDD of (self, List(neighbors)) 118 | * E.g.: nodePairs (1, 4), (6, 1), (3, 2), (6, 5) 119 | * will result into (4, List(1)), (1, List(4)), (6, List(1)), (1, List(6)), (3, List(2)), (2, List(3)), (6, List(5)), (5, List(6)) 120 | */ 121 | 122 | val neighbors = nodePairs.flatMap(x => { 123 | val (self, neighbor) = (x._1, x._2) 124 | if (self == neighbor) 125 | List((self, neighbor)) 126 | else 127 | List((self, neighbor), (neighbor, self)) 128 | }) 129 | 130 | /** 131 | * reduce on self to get list of all its neighbors. 132 | * E.g: (4, List(1)), (1, List(4)), (6, List(1)), (1, List(6)), (3, List(2)), (2, List(3)), (6, List(5)), (5, List(6)) 133 | * will result into (4, List(1)), (1, List(4, 6)), (6, List(1, 5)), (3, List(2)), (2, List(3)), (5, List(6)) 134 | * Note: 135 | * (1) you may need to tweak number of partitions. 136 | * (2) also, watch out for data skew. In that case, consider using rangePartitioner 137 | */ 138 | 139 | val localAdd = (s: mutable.HashSet[Long], v: Long) => s += v 140 | val partitionAdd = (s1: mutable.HashSet[Long], s2: mutable.HashSet[Long]) => s1 ++= s2 141 | val allNeighbors = neighbors.aggregateByKey(mutable.HashSet.empty[Long]/*, rangePartitioner*/)(localAdd, partitionAdd) 142 | 143 | /** 144 | * Apply Large Star operation on (self, List(neighbor)) to get newNodePairs and count the change in connectivity 145 | */ 146 | 147 | val newNodePairsWithChangeCount = allNeighbors.map(x => { 148 | val self = x._1 149 | val neighbors = x._2.toList 150 | val minNode = argMin(self :: neighbors) 151 | val newNodePairs = (self :: neighbors).map(neighbor => { 152 | (neighbor, minNode) 153 | }).filter(x => { 154 | val neighbor = x._1 155 | val minNode = x._2 156 | neighbor >= self 157 | }) 158 | 159 | val uniqueNewNodePairs = newNodePairs.toSet.toList 160 | val connectivityChangeCount = (uniqueNewNodePairs diff neighbors.map((self, _))).length 161 | (uniqueNewNodePairs, connectivityChangeCount) 162 | }).persist(StorageLevel.MEMORY_AND_DISK_SER) 163 | 164 | val totalConnectivityCountChange = newNodePairsWithChangeCount.mapPartitions(iter => { 165 | val (v, l) = iter.toSeq.unzip 166 | val sum = l.foldLeft(0)(_ + _) 167 | Iterator(sum) 168 | }).sum.toInt 169 | 170 | /** 171 | * Sum all the changeCounts 172 | */ 173 | val newNodePairs = newNodePairsWithChangeCount.map(x => x._1).flatMap(x => x) 174 | newNodePairsWithChangeCount.unpersist(false) 175 | (newNodePairs, totalConnectivityCountChange) 176 | } 177 | 178 | private def argMin(nodes: List[Long]): Long = { 179 | nodes.min(Ordering.by((node: Long) => node)) 180 | } 181 | 182 | /** 183 | * Build nodePairs given a list of nodes. A list of nodes represents a subgraph. 184 | * @param nodes that are part of a subgraph 185 | * @return nodePairs for a subgraph 186 | */ 187 | private def buildPairs(nodes:List[Long]) : List[(Long, Long)] = { 188 | buildPairs(nodes.head, nodes.tail, null.asInstanceOf[List[(Long, Long)]]) 189 | } 190 | 191 | @tailrec 192 | private def buildPairs(node: Long, neighbors:List[Long], partialPairs: List[(Long, Long)]) : List[(Long, Long)] = { 193 | if (neighbors.isEmpty) { 194 | if (partialPairs != null) 195 | List((node, node)) ::: partialPairs 196 | else 197 | List((node, node)) 198 | } else if (neighbors.length == 1) { 199 | val neighbor = neighbors(0) 200 | if (node > neighbor) 201 | if (partialPairs != null) List((node, neighbor)) ::: partialPairs else List((node, neighbor)) 202 | else 203 | if (partialPairs != null) List((neighbor, node)) ::: partialPairs else List((neighbor, node)) 204 | } else { 205 | val newPartialPairs = neighbors.map(neighbor => { 206 | if (node > neighbor) 207 | List((node, neighbor)) 208 | else 209 | List((neighbor, node)) 210 | }).flatMap(x=>x) 211 | 212 | if (partialPairs != null) 213 | buildPairs(neighbors.head, neighbors.tail, newPartialPairs ::: partialPairs) 214 | else 215 | buildPairs(neighbors.head, neighbors.tail, newPartialPairs) 216 | } 217 | } 218 | 219 | /** 220 | * Implements alternatingAlgo. Converges when the changeCount is either 0 or does not change from the previous iteration 221 | * @param nodePairs for a graph 222 | * @param largeStarConnectivityChangeCount change count that resulted from the previous iteration 223 | * @param smallStarConnectivityChangeCount change count that resulted from the previous iteration 224 | * @param didConverge flag to indicate the alorigth converged 225 | * @param currIterationCount counter to capture number of iterations 226 | * @param maxIterationCount maximum number iterations to try before giving up 227 | * @return RDD of nodePairs 228 | */ 229 | 230 | @tailrec 231 | private def alternatingAlgo(nodePairs: RDD[(Long, Long)], 232 | largeStarConnectivityChangeCount: Int, smallStarConnectivityChangeCount: Int, didConverge: Boolean, 233 | currIterationCount: Int, maxIterationCount: Int): (RDD[(Long, Long)], Boolean, Int) = { 234 | 235 | val iterationCount = currIterationCount + 1 236 | if (didConverge) 237 | (nodePairs, true, currIterationCount) 238 | else if (currIterationCount >= maxIterationCount) { 239 | (nodePairs, false, currIterationCount) 240 | } 241 | else { 242 | 243 | val (nodePairsLargeStar, currLargeStarConnectivityChangeCount) = largeStar(nodePairs) 244 | 245 | val (nodePairsSmallStar, currSmallStarConnectivityChangeCount) = smallStar(nodePairsLargeStar) 246 | 247 | if ((currLargeStarConnectivityChangeCount == largeStarConnectivityChangeCount && 248 | currSmallStarConnectivityChangeCount == smallStarConnectivityChangeCount) || 249 | (currSmallStarConnectivityChangeCount == 0 && currLargeStarConnectivityChangeCount == 0)) { 250 | alternatingAlgo(nodePairsSmallStar, currLargeStarConnectivityChangeCount, 251 | currSmallStarConnectivityChangeCount, true, iterationCount, maxIterationCount) 252 | } 253 | else { 254 | alternatingAlgo(nodePairsSmallStar, currLargeStarConnectivityChangeCount, 255 | currSmallStarConnectivityChangeCount, false, iterationCount, maxIterationCount) 256 | } 257 | } 258 | } 259 | 260 | /** 261 | * Driver function 262 | * @param cliques list of nodes representing subgraphs (or cliques) 263 | * @param maxIterationCount maximum number iterations to try before giving up 264 | * @return Connected Components as nodePairs where second member of the nodePair is the minimum node in the component 265 | */ 266 | def run(cliques:RDD[List[Long]], maxIterationCount: Int): (RDD[(Long, Long)], Boolean, Int) = { 267 | 268 | val nodePairs = cliques.map(aClique => { 269 | buildPairs(aClique) 270 | }).flatMap(x=>x) 271 | 272 | val (cc, didConverge, iterCount) = alternatingAlgo(nodePairs, 9999999, 9999999, false, 0, maxIterationCount) 273 | 274 | if (didConverge) { 275 | (cc, didConverge, iterCount) 276 | } else { 277 | (null.asInstanceOf[RDD[(Long, Long)]], didConverge, iterCount) 278 | } 279 | } 280 | } 281 | 282 | 283 | --------------------------------------------------------------------------------